From e0ce93040abf8e1b744b51fef9e5a2797aefdb9f Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Mon, 31 Jul 2023 10:10:35 +0800 Subject: [PATCH 001/699] [improve][txn]Add log when can't send TcClientConnectRequest to broker. (#20880) Main Issue: https://github.com/apache/pulsar/issues/20851 ### Motivation When the Proto version does not allow us to send TcClientConnectRequest to the broker, we should add a log to debug it. ### Modifications Add a waining log. --- .../apache/pulsar/client/impl/TransactionMetaStoreHandler.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java index ebbfca0c3cb3f..0b5174a015118 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java @@ -167,6 +167,8 @@ public CompletableFuture connectionOpened(ClientCnx cnx) { return null; }); } else { + LOG.warn("Can not connect to the transaction coordinator because the protocol version {} is " + + "lower than 19", cnx.getRemoteEndpointProtocolVersion()); registerToConnection(cnx); future.complete(null); } From c5c680eb642c2c35b92233457a2ddbf90c6a5222 Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Mon, 31 Jul 2023 20:40:29 +0800 Subject: [PATCH 002/699] [improve][sql] Fix the wrong format of the logs (#20907) --- .../java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java | 2 +- .../apache/pulsar/sql/presto/PulsarSqlSchemaInfoProvider.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java index 858624b156dbb..6663b349761a3 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java @@ -776,7 +776,7 @@ private void initEntryCacheSizeAllocator(PulsarConnectorConfig connectorConfig) connectorConfig.getMaxSplitQueueSizeBytes() / 2); this.messageQueueCacheSizeAllocator = new NoStrictCacheSizeAllocator( connectorConfig.getMaxSplitQueueSizeBytes() / 2); - log.info("Init cacheSizeAllocator with maxSplitEntryQueueSizeBytes {}.", + log.info("Init cacheSizeAllocator with maxSplitEntryQueueSizeBytes %d.", connectorConfig.getMaxSplitQueueSizeBytes()); } else { this.entryQueueCacheSizeAllocator = new NullCacheSizeAllocator(); diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSqlSchemaInfoProvider.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSqlSchemaInfoProvider.java index e2d030d2d7f1b..673ea2b3940d9 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSqlSchemaInfoProvider.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarSqlSchemaInfoProvider.java @@ -74,7 +74,7 @@ public CompletableFuture getSchemaByVersion(byte[] schemaVersion) { } return cache.get(BytesSchemaVersion.of(schemaVersion)); } catch (ExecutionException e) { - LOG.error("Can't get generic schema for topic {} schema version {}", + LOG.error("Can't get generic schema for topic %s schema version %s", topicName.toString(), new String(schemaVersion, StandardCharsets.UTF_8), e); return FutureUtil.failedFuture(e.getCause()); } From 736265cb29ad2251286e04bee762eec8442368ea Mon Sep 17 00:00:00 2001 From: lifepuzzlefun Date: Tue, 1 Aug 2023 13:16:25 +0800 Subject: [PATCH 003/699] [fix][broker] fix MessageDeduplication throw NPE when enable broker dedup and set namespace disable deduplication. (#20905) --- .../persistent/MessageDeduplication.java | 4 ++ .../persistent/TopicDuplicationTest.java | 37 +++++++++++++++++++ 2 files changed, 41 insertions(+) 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 ed4e70bfd2953..490be4a8876fc 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 @@ -482,6 +482,10 @@ public long getLastPublishedSequenceId(String producerName) { } public void takeSnapshot() { + if (!isEnabled()) { + return; + } + Integer interval = topic.getHierarchyTopicPolicies().getDeduplicationSnapshotIntervalSeconds().get(); long currentTimeStamp = System.currentTimeMillis(); if (interval == null || interval <= 0 diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/TopicDuplicationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/TopicDuplicationTest.java index e57092d02dd5d..16721ca1203fd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/TopicDuplicationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/TopicDuplicationTest.java @@ -492,6 +492,43 @@ public void testDisableNamespacePolicyTakeSnapshot() throws Exception { } + @Test(timeOut = 30000) + public void testDisableNamespacePolicyTakeSnapshotShouldNotThrowException() throws Exception { + cleanup(); + conf.setBrokerDeduplicationEnabled(true); + conf.setBrokerDeduplicationSnapshotFrequencyInSeconds(1); + conf.setBrokerDeduplicationSnapshotIntervalSeconds(1); + conf.setBrokerDeduplicationEntriesInterval(20000); + setup(); + + final String topicName = testTopic + UUID.randomUUID().toString(); + final String producerName = "my-producer"; + @Cleanup + Producer producer = pulsarClient + .newProducer(Schema.STRING).topic(topicName).enableBatching(false).producerName(producerName).create(); + + // disable deduplication + admin.namespaces().setDeduplicationStatus(myNamespace, false); + + int msgNum = 50; + CountDownLatch countDownLatch = new CountDownLatch(msgNum); + for (int i = 0; i < msgNum; i++) { + producer.newMessage().value("msg" + i).sendAsync().whenComplete((res, e) -> countDownLatch.countDown()); + } + countDownLatch.await(); + PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService() + .getTopicIfExists(topicName).get().get(); + ManagedCursor managedCursor = persistentTopic.getMessageDeduplication().getManagedCursor(); + + // when disable topic deduplication the cursor should be deleted. + assertNull(managedCursor); + + // this method will be called at brokerService forEachTopic. + // if topic level disable deduplication. + // this method should be skipped without throw exception. + persistentTopic.checkDeduplicationSnapshot(); + } + private void waitCacheInit(String topicName) throws Exception { pulsarClient.newConsumer().topic(topicName).subscriptionName("my-sub").subscribe().close(); TopicName topic = TopicName.get(topicName); From 36093998f0217a464113f4d0bfe51dad898bda56 Mon Sep 17 00:00:00 2001 From: Jiwe Guo Date: Wed, 2 Aug 2023 11:28:56 +0800 Subject: [PATCH 004/699] Release 3.1.0 --- 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 +- managed-ledger/pom.xml | 2 +- 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-client-1x-base/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-1x/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-2x-shaded/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/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/hdfs2/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-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-sql/pom.xml | 2 +- pulsar-sql/presto-distribution/pom.xml | 2 +- pulsar-sql/presto-pulsar-plugin/pom.xml | 2 +- pulsar-sql/presto-pulsar/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 +- 133 files changed, 135 insertions(+), 135 deletions(-) diff --git a/bouncy-castle/bc/pom.xml b/bouncy-castle/bc/pom.xml index 7ea333a4711a7..6170221e9432e 100644 --- a/bouncy-castle/bc/pom.xml +++ b/bouncy-castle/bc/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar bouncy-castle-parent - 3.1.0-SNAPSHOT + 3.1.0 .. diff --git a/bouncy-castle/bcfips-include-test/pom.xml b/bouncy-castle/bcfips-include-test/pom.xml index e8348be9292cd..678a5c3c804e3 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 - 3.1.0-SNAPSHOT + 3.1.0 .. diff --git a/bouncy-castle/bcfips/pom.xml b/bouncy-castle/bcfips/pom.xml index a07e5e19907f2..0654d746ed9bf 100644 --- a/bouncy-castle/bcfips/pom.xml +++ b/bouncy-castle/bcfips/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar bouncy-castle-parent - 3.1.0-SNAPSHOT + 3.1.0 .. diff --git a/bouncy-castle/pom.xml b/bouncy-castle/pom.xml index daefeb83b5371..be5e058fd396a 100644 --- a/bouncy-castle/pom.xml +++ b/bouncy-castle/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 3.1.0-SNAPSHOT + 3.1.0 .. diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 836e1c5cb5f0e..b4827e63cfaa1 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -31,12 +31,12 @@ org.apache.pulsar buildtools - 3.1.0-SNAPSHOT + 3.1.0 jar Pulsar Build Tools - 2023-05-03T02:53:27Z + 2023-08-02T03:24:30Z 1.8 1.8 3.1.0 diff --git a/distribution/io/pom.xml b/distribution/io/pom.xml index 568d76922bf4e..93aa65c08101f 100644 --- a/distribution/io/pom.xml +++ b/distribution/io/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 3.1.0-SNAPSHOT + 3.1.0 .. diff --git a/distribution/offloaders/pom.xml b/distribution/offloaders/pom.xml index d23ebec2ef26d..5f0965cca9585 100644 --- a/distribution/offloaders/pom.xml +++ b/distribution/offloaders/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 3.1.0-SNAPSHOT + 3.1.0 .. diff --git a/distribution/pom.xml b/distribution/pom.xml index 36a3fa1c5835a..9cd66e14d51f2 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 3.1.0-SNAPSHOT + 3.1.0 .. diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index 2702bace54e6a..129d05891e2d5 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 3.1.0-SNAPSHOT + 3.1.0 .. diff --git a/distribution/shell/pom.xml b/distribution/shell/pom.xml index 9e3134a75e5bf..d625cdcc1796b 100644 --- a/distribution/shell/pom.xml +++ b/distribution/shell/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 3.1.0-SNAPSHOT + 3.1.0 .. diff --git a/docker/pom.xml b/docker/pom.xml index afe55f0fe57f0..87a810f4f7b0a 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 3.1.0-SNAPSHOT + 3.1.0 docker-images Apache Pulsar :: Docker Images diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index e616ac132d319..b269754e6336f 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar docker-images - 3.1.0-SNAPSHOT + 3.1.0 4.0.0 pulsar-all-docker-image diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 85d86cff12523..4747f05aa8e0c 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar docker-images - 3.1.0-SNAPSHOT + 3.1.0 4.0.0 pulsar-docker-image diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml index dfb155c2d5a7d..dc8c0d376c425 100644 --- a/jclouds-shaded/pom.xml +++ b/jclouds-shaded/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 3.1.0-SNAPSHOT + 3.1.0 .. diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index a8cb560b7b376..0558edcf69b28 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 3.1.0-SNAPSHOT + 3.1.0 .. diff --git a/pom.xml b/pom.xml index 7e3538dc6a154..a5e0b9bf5b2a3 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ org.apache.pulsar pulsar - 3.1.0-SNAPSHOT + 3.1.0 Pulsar Pulsar is a distributed pub-sub messaging platform with a very @@ -92,7 +92,7 @@ flexible messaging model and an intuitive client API. UTF-8 UTF-8 - 2023-05-03T02:53:27Z + 2023-08-02T03:24:30Z true - 2.8.0 + 3.4.0 1.8.20 1.0 From b2c70523fef920d1cec6b96d277ab37c479ef23d Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Fri, 1 Sep 2023 10:28:48 +0800 Subject: [PATCH 032/699] [improve][broker] Make read compacted entries support maxReadSizeBytes limitation (#21065) --- .../mledger/impl/ManagedCursorImpl.java | 4 +- .../pulsar/compaction/CompactedTopic.java | 3 +- .../pulsar/compaction/CompactedTopicImpl.java | 15 +++++- .../compaction/CompactedTopicUtils.java | 18 +++++-- .../compaction/CompactedTopicUtilsTest.java | 5 +- .../pulsar/compaction/CompactionTest.java | 49 +++++++++++++++++++ .../compaction/StrategicCompactionTest.java | 19 +++++++ 7 files changed, 102 insertions(+), 11 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 e2b202cce1524..a2420c1c29efb 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 @@ -3490,7 +3490,7 @@ public ManagedCursorMXBean getStats() { return this.mbean; } - void updateReadStats(int readEntriesCount, long readEntriesSize) { + public void updateReadStats(int readEntriesCount, long readEntriesSize) { this.entriesReadCount += readEntriesCount; this.entriesReadSize += readEntriesSize; } @@ -3522,7 +3522,7 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { }, null); } - private int applyMaxSizeCap(int maxEntries, long maxSizeBytes) { + public int applyMaxSizeCap(int maxEntries, long maxSizeBytes) { if (maxSizeBytes == NO_MAX_SIZE_LIMIT) { return maxEntries; } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java index 660c7ea779741..8c17e0f3ca34d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java @@ -38,7 +38,8 @@ public interface CompactedTopic { */ @Deprecated void asyncReadEntriesOrWait(ManagedCursor cursor, - int numberOfEntriesToRead, + int maxEntries, + long bytesToRead, boolean isFirstRead, ReadEntriesCallback callback, Consumer consumer); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java index fe24a23b7cdee..b028b708c49e4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java @@ -43,6 +43,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.EntryImpl; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer.ReadEntriesCtx; @@ -93,7 +94,8 @@ public CompletableFuture deleteCompactedLedger(long compactedLedgerId) { @Override @Deprecated public void asyncReadEntriesOrWait(ManagedCursor cursor, - int numberOfEntriesToRead, + int maxEntries, + long bytesToRead, boolean isFirstRead, ReadEntriesCallback callback, Consumer consumer) { PositionImpl cursorPosition; @@ -110,8 +112,11 @@ public void asyncReadEntriesOrWait(ManagedCursor cursor, if (currentCompactionHorizon == null || currentCompactionHorizon.compareTo(cursorPosition) < 0) { - cursor.asyncReadEntriesOrWait(numberOfEntriesToRead, callback, readEntriesCtx, PositionImpl.LATEST); + cursor.asyncReadEntriesOrWait(maxEntries, bytesToRead, callback, readEntriesCtx, PositionImpl.LATEST); } else { + ManagedCursorImpl managedCursor = (ManagedCursorImpl) cursor; + int numberOfEntriesToRead = managedCursor.applyMaxSizeCap(maxEntries, bytesToRead); + compactedTopicContext.thenCompose( (context) -> findStartPoint(cursorPosition, context.ledger.getLastAddConfirmed(), context.cache) .thenCompose((startPoint) -> { @@ -126,6 +131,12 @@ public void asyncReadEntriesOrWait(ManagedCursor cursor, startPoint + (numberOfEntriesToRead - 1)); return readEntries(context.ledger, startPoint, endPoint) .thenAccept((entries) -> { + long entriesSize = 0; + for (Entry entry : entries) { + entriesSize += entry.getLength(); + } + managedCursor.updateReadStats(entries.size(), entriesSize); + Entry lastEntry = entries.get(entries.size() - 1); // The compaction task depends on the last snapshot and the incremental // entries to build the new snapshot. So for the compaction cursor, we diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java index 6acd33279fd40..66bcf4c3002bd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java @@ -30,6 +30,7 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.CollectionUtils; import org.apache.pulsar.broker.service.Consumer; @@ -40,13 +41,13 @@ public class CompactedTopicUtils { @Beta public static void asyncReadCompactedEntries(TopicCompactionService topicCompactionService, - ManagedCursor cursor, int numberOfEntriesToRead, + ManagedCursor cursor, int maxEntries, long bytesToRead, boolean readFromEarliest, AsyncCallbacks.ReadEntriesCallback callback, boolean wait, @Nullable Consumer consumer) { Objects.requireNonNull(topicCompactionService); Objects.requireNonNull(cursor); - checkArgument(numberOfEntriesToRead > 0); + checkArgument(maxEntries > 0); Objects.requireNonNull(callback); final PositionImpl readPosition; @@ -67,15 +68,18 @@ public static void asyncReadCompactedEntries(TopicCompactionService topicCompact || readPosition.compareTo( lastCompactedPosition.getLedgerId(), lastCompactedPosition.getEntryId()) > 0) { if (wait) { - cursor.asyncReadEntriesOrWait(numberOfEntriesToRead, bytesToRead, callback, readEntriesCtx, + cursor.asyncReadEntriesOrWait(maxEntries, bytesToRead, callback, readEntriesCtx, PositionImpl.LATEST); } else { - cursor.asyncReadEntries(numberOfEntriesToRead, bytesToRead, callback, readEntriesCtx, + cursor.asyncReadEntries(maxEntries, bytesToRead, callback, readEntriesCtx, PositionImpl.LATEST); } return CompletableFuture.completedFuture(null); } + ManagedCursorImpl managedCursor = (ManagedCursorImpl) cursor; + int numberOfEntriesToRead = managedCursor.applyMaxSizeCap(maxEntries, bytesToRead); + return topicCompactionService.readCompactedEntries(readPosition, numberOfEntriesToRead) .thenAccept(entries -> { if (CollectionUtils.isEmpty(entries)) { @@ -88,6 +92,12 @@ public static void asyncReadCompactedEntries(TopicCompactionService topicCompact return; } + long entriesSize = 0; + for (Entry entry : entries) { + entriesSize += entry.getLength(); + } + managedCursor.updateReadStats(entries.size(), entriesSize); + Entry lastEntry = entries.get(entries.size() - 1); cursor.seek(lastEntry.getPosition().getNext(), true); callback.readEntriesComplete(entries, readEntriesCtx); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicUtilsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicUtilsTest.java index 329abf9f780fa..94f2a17a2a3f4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicUtilsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicUtilsTest.java @@ -25,8 +25,8 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.Entry; -import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.mockito.Mockito; import org.testng.Assert; @@ -46,8 +46,9 @@ public void testReadCompactedEntriesWithEmptyEntries() throws ExecutionException PositionImpl initPosition = PositionImpl.get(1, 90); AtomicReference readPositionRef = new AtomicReference<>(initPosition.getNext()); - ManagedCursor cursor = Mockito.mock(ManagedCursor.class); + ManagedCursorImpl cursor = Mockito.mock(ManagedCursorImpl.class); Mockito.doReturn(readPositionRef.get()).when(cursor).getReadPosition(); + Mockito.doReturn(1).when(cursor).applyMaxSizeCap(Mockito.anyInt(), Mockito.anyLong()); Mockito.doAnswer(invocation -> { readPositionRef.set(invocation.getArgument(0)); return null; 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 c5dbd9c49aac9..afbbe6101f8fc 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 @@ -55,6 +55,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerInfo; import org.apache.bookkeeper.mledger.Position; +import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; @@ -90,6 +91,7 @@ import org.apache.pulsar.common.protocol.Markers; import org.apache.pulsar.common.util.FutureUtil; import org.awaitility.Awaitility; +import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -1877,4 +1879,51 @@ public void testReceiverQueueSize() throws Exception { consumer.close(); producer.close(); } + + @Test + public void testDispatcherMaxReadSizeBytes() throws Exception { + final String topicName = + "persistent://my-property/use/my-ns/testDispatcherMaxReadSizeBytes" + UUID.randomUUID(); + final String subName = "my-sub"; + final int receiveQueueSize = 1; + @Cleanup + PulsarClient client = newPulsarClient(lookupUrl.toString(), 100); + Producer producer = pulsarClient.newProducer(Schema.BYTES) + .topic(topicName).create(); + + for (int i = 0; i < 10; i+=2) { + producer.newMessage().key(null).value(new byte[4*1024*1024]).send(); + } + producer.flush(); + + admin.topics().triggerCompaction(topicName); + + Awaitility.await().untilAsserted(() -> { + assertEquals(admin.topics().compactionStatus(topicName).status, + LongRunningProcessStatus.Status.SUCCESS); + }); + + admin.topics().unload(topicName); + + ConsumerImpl consumer = (ConsumerImpl) client.newConsumer(Schema.BYTES) + .topic(topicName).readCompacted(true).receiverQueueSize(receiveQueueSize).subscriptionName(subName) + .subscribe(); + + + PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get(); + TopicCompactionService topicCompactionService = Mockito.spy(topic.getTopicCompactionService()); + FieldUtils.writeDeclaredField(topic, "topicCompactionService", topicCompactionService, true); + + Awaitility.await().untilAsserted(() -> { + assertEquals(consumer.getStats().getMsgNumInReceiverQueue(), + 1); + }); + + consumer.increaseAvailablePermits(2); + + Mockito.verify(topicCompactionService, Mockito.times(1)).readCompactedEntries(Mockito.any(), Mockito.same(1)); + + consumer.close(); + producer.close(); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionTest.java index 135a839bd54a8..799c2703e1ed4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionTest.java @@ -148,5 +148,24 @@ public void testNumericOrderCompaction() throws Exception { Assert.assertEquals(tableView.entrySet(), expectedCopy.entrySet()); } + @Override + public void testCompactCompressedBatching() throws Exception { + compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler, 10); + super.testCompactCompressedBatching(); + compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler, 1); + } + + @Override + public void testCompactEncryptedAndCompressedBatching() throws Exception { + compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler, 10); + super.testCompactEncryptedAndCompressedBatching(); + compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler, 1); + } + @Override + public void testCompactEncryptedBatching() throws Exception { + compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler, 10); + super.testCompactEncryptedBatching(); + compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler, 1); + } } From 59ca24ab5e0019147c44f0917c90952a149e08d2 Mon Sep 17 00:00:00 2001 From: thetumbled <52550727+thetumbled@users.noreply.github.com> Date: Thu, 31 Aug 2023 09:38:11 +0800 Subject: [PATCH 033/699] [fix] [broker] remove bundle-data in local metadata store. (#21078) Motivation: When deleting a namespace, we will delete znode under the path `/loadbalance/bundle-data` in `local metadata store` instead of `global metadata store`. Modifications: Delete bundle data znode in local metadata store. --- .../broker/resources/NamespaceResources.java | 8 ++-- .../broker/resources/PulsarResources.java | 3 +- .../resources/NamespaceResourcesTest.java | 44 +++++++++++++++++++ .../pulsar/broker/admin/AdminApi2Test.java | 31 ++++++++++--- .../broker/testcontext/PulsarTestContext.java | 3 +- 5 files changed, 79 insertions(+), 10 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java index f70d3f31d6e2f..b5ccc9a5a9077 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/NamespaceResources.java @@ -49,16 +49,18 @@ public class NamespaceResources extends BaseResources { private final IsolationPolicyResources isolationPolicies; private final PartitionedTopicResources partitionedTopicResources; private final MetadataStore configurationStore; + private final MetadataStore localStore; public static final String POLICIES_READONLY_FLAG_PATH = "/admin/flags/policies-readonly"; private static final String NAMESPACE_BASE_PATH = "/namespace"; private static final String BUNDLE_DATA_BASE_PATH = "/loadbalance/bundle-data"; - public NamespaceResources(MetadataStore configurationStore, int operationTimeoutSec) { + public NamespaceResources(MetadataStore localStore, MetadataStore configurationStore, int operationTimeoutSec) { super(configurationStore, Policies.class, operationTimeoutSec); this.configurationStore = configurationStore; isolationPolicies = new IsolationPolicyResources(configurationStore, operationTimeoutSec); partitionedTopicResources = new PartitionedTopicResources(configurationStore, operationTimeoutSec); + this.localStore = localStore; } public CompletableFuture> listNamespacesAsync(String tenant) { @@ -381,13 +383,13 @@ public CompletableFuture runWithMarkDeleteAsync(TopicName topic, // clear resource of `/loadbalance/bundle-data/{tenant}/{namespace}/` in metadata-store public CompletableFuture deleteBundleDataAsync(NamespaceName ns) { final String namespaceBundlePath = joinPath(BUNDLE_DATA_BASE_PATH, ns.toString()); - return getStore().deleteRecursive(namespaceBundlePath); + return this.localStore.deleteRecursive(namespaceBundlePath); } // clear resource of `/loadbalance/bundle-data/{tenant}/` in metadata-store public CompletableFuture deleteBundleDataTenantAsync(String tenant) { final String tenantBundlePath = joinPath(BUNDLE_DATA_BASE_PATH, tenant); - return getStore().deleteRecursive(tenantBundlePath); + return this.localStore.deleteRecursive(tenantBundlePath); } } \ No newline at end of file diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/PulsarResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/PulsarResources.java index dfcd0a4194ff5..a3c5633a6dbe8 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/PulsarResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/PulsarResources.java @@ -60,7 +60,8 @@ public PulsarResources(MetadataStore localMetadataStore, MetadataStore configura if (configurationMetadataStore != null) { tenantResources = new TenantResources(configurationMetadataStore, operationTimeoutSec); clusterResources = new ClusterResources(configurationMetadataStore, operationTimeoutSec); - namespaceResources = new NamespaceResources(configurationMetadataStore, operationTimeoutSec); + namespaceResources = new NamespaceResources(localMetadataStore, configurationMetadataStore + , operationTimeoutSec); resourcegroupResources = new ResourceGroupResources(configurationMetadataStore, operationTimeoutSec); } else { tenantResources = null; diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/resources/NamespaceResourcesTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/resources/NamespaceResourcesTest.java index 85f54a76dc3c4..deb86e1802f6f 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/resources/NamespaceResourcesTest.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/resources/NamespaceResourcesTest.java @@ -18,12 +18,34 @@ */ package org.apache.pulsar.broker.resources; +import static org.apache.pulsar.broker.resources.BaseResources.joinPath; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; + +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.metadata.api.MetadataStore; +import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; public class NamespaceResourcesTest { + + private MetadataStore localStore; + private MetadataStore configurationStore; + private NamespaceResources namespaceResources; + + private static final String BUNDLE_DATA_BASE_PATH = "/loadbalance/bundle-data"; + + @BeforeMethod + public void setup() { + localStore = mock(MetadataStore.class); + configurationStore = mock(MetadataStore.class); + namespaceResources = new NamespaceResources(localStore, configurationStore, 30); + } + @Test public void test_pathIsFromNamespace() { assertFalse(NamespaceResources.pathIsFromNamespace("/admin/clusters")); @@ -31,4 +53,26 @@ public void test_pathIsFromNamespace() { assertFalse(NamespaceResources.pathIsFromNamespace("/admin/policies/my-tenant")); assertTrue(NamespaceResources.pathIsFromNamespace("/admin/policies/my-tenant/my-ns")); } + + /** + * Test that the bundle-data node is deleted from the local stores. + */ + @Test + public void testDeleteBundleDataAsync() { + NamespaceName ns = NamespaceName.get("my-tenant/my-ns"); + String namespaceBundlePath = joinPath(BUNDLE_DATA_BASE_PATH, ns.toString()); + namespaceResources.deleteBundleDataAsync(ns); + + String tenant="my-tenant"; + String tenantBundlePath = joinPath(BUNDLE_DATA_BASE_PATH, tenant); + namespaceResources.deleteBundleDataTenantAsync(tenant); + + verify(localStore).deleteRecursive(namespaceBundlePath); + verify(localStore).deleteRecursive(tenantBundlePath); + + assertThrows(()-> verify(configurationStore).deleteRecursive(namespaceBundlePath)); + assertThrows(()-> verify(configurationStore).deleteRecursive(tenantBundlePath)); + } + + } \ No newline at end of file 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 0adf945a55552..99009a1988206 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 @@ -67,6 +67,7 @@ import org.apache.pulsar.broker.admin.AdminApiTest.MockedPulsarService; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl; +import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerWrapper; import org.apache.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; @@ -1720,6 +1721,8 @@ public void testDeleteNamespace(NamespaceAttr namespaceAttr) throws Exception { // Set conf. cleanup(); setNamespaceAttr(namespaceAttr); + this.conf.setMetadataStoreUrl("127.0.0.1:2181"); + this.conf.setConfigurationMetadataStoreUrl("127.0.0.1:2182"); setup(); String tenant = newUniqueName("test-tenant"); @@ -1740,6 +1743,28 @@ public void testDeleteNamespace(NamespaceAttr namespaceAttr) throws Exception { admin.topics().createPartitionedTopic(topic, 10); assertFalse(admin.topics().getList(namespace).isEmpty()); + final String managedLedgersPath = "/managed-ledgers/" + namespace; + final String bundleDataPath = "/loadbalance/bundle-data/" + namespace; + // Trigger bundle owned by brokers. + pulsarClient.newProducer().topic(topic).create().close(); + // Trigger bundle data write to ZK. + Awaitility.await().untilAsserted(() -> { + boolean bundleDataWereWriten = false; + for (PulsarService ps : new PulsarService[]{pulsar, mockPulsarSetup.getPulsar()}) { + ModularLoadManagerWrapper loadManager = (ModularLoadManagerWrapper) ps.getLoadManager().get(); + ModularLoadManagerImpl loadManagerImpl = (ModularLoadManagerImpl) loadManager.getLoadManager(); + ps.getBrokerService().updateRates(); + loadManagerImpl.updateLocalBrokerData(); + loadManagerImpl.writeBundleDataOnZooKeeper(); + bundleDataWereWriten = bundleDataWereWriten || ps.getLocalMetadataStore().exists(bundleDataPath).join(); + } + assertTrue(bundleDataWereWriten); + }); + + // assert znode exists in metadata store + assertTrue(pulsar.getLocalMetadataStore().exists(bundleDataPath).join()); + assertTrue(pulsar.getLocalMetadataStore().exists(managedLedgersPath).join()); + try { admin.namespaces().deleteNamespace(namespace, false); fail("should have failed due to namespace not empty"); @@ -1756,12 +1781,8 @@ public void testDeleteNamespace(NamespaceAttr namespaceAttr) throws Exception { assertFalse(admin.namespaces().getNamespaces(tenant).contains(namespace)); assertTrue(admin.namespaces().getNamespaces(tenant).isEmpty()); - - final String managedLedgersPath = "/managed-ledgers/" + namespace; + // assert znode deleted in metadata store assertFalse(pulsar.getLocalMetadataStore().exists(managedLedgersPath).join()); - - - final String bundleDataPath = "/loadbalance/bundle-data/" + namespace; assertFalse(pulsar.getLocalMetadataStore().exists(bundleDataPath).join()); } 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 40a42286fda74..aa1ff56c0681a 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 @@ -715,7 +715,8 @@ protected void initializePulsarServices(SpyConfig spyConfig, Builder builder) { if (metadataStore == null) { metadataStore = builder.configurationMetadataStore; } - NamespaceResources nsr = spyConfigPulsarResources.spy(NamespaceResources.class, metadataStore, 30); + NamespaceResources nsr = spyConfigPulsarResources.spy(NamespaceResources.class, + builder.localMetadataStore, metadataStore, 30); TopicResources tsr = spyConfigPulsarResources.spy(TopicResources.class, metadataStore); pulsarResources( spyConfigPulsarResources.spy( From b9051c63919806c86463e74f39254ee6540ed6f3 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Mon, 4 Sep 2023 17:04:51 +0800 Subject: [PATCH 034/699] [fix][broker] Fix write duplicate entries into the compacted ledger after RawReader reconnects (#21081) --- .../pulsar/compaction/TwoPhaseCompactor.java | 13 +++- .../pulsar/compaction/CompactionTest.java | 66 +++++++++++++++++++ 2 files changed, 76 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java index ba13705b8075e..e4e067ad6117e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java @@ -205,7 +205,7 @@ private CompletableFuture phaseTwoSeekThenLoop(RawReader reader, MessageId reader.seekAsync(from).thenCompose((v) -> { Semaphore outstanding = new Semaphore(MAX_OUTSTANDING); CompletableFuture loopPromise = new CompletableFuture<>(); - phaseTwoLoop(reader, to, latestForKey, ledger, outstanding, loopPromise); + phaseTwoLoop(reader, to, latestForKey, ledger, outstanding, loopPromise, MessageId.earliest); return loopPromise; }).thenCompose((v) -> closeLedger(ledger)) .thenCompose((v) -> reader.acknowledgeCumulativeAsync(lastReadId, @@ -227,7 +227,8 @@ private CompletableFuture phaseTwoSeekThenLoop(RawReader reader, MessageId } private void phaseTwoLoop(RawReader reader, MessageId to, Map latestForKey, - LedgerHandle lh, Semaphore outstanding, CompletableFuture promise) { + LedgerHandle lh, Semaphore outstanding, CompletableFuture promise, + MessageId lastCompactedMessageId) { if (promise.isDone()) { return; } @@ -236,6 +237,12 @@ private void phaseTwoLoop(RawReader reader, MessageId to, Map m.close(); return; } + + if (m.getMessageId().compareTo(lastCompactedMessageId) <= 0) { + phaseTwoLoop(reader, to, latestForKey, lh, outstanding, promise, lastCompactedMessageId); + return; + } + try { MessageId id = m.getMessageId(); Optional messageToAdd = Optional.empty(); @@ -306,7 +313,7 @@ private void phaseTwoLoop(RawReader reader, MessageId to, Map } return; } - phaseTwoLoop(reader, to, latestForKey, lh, outstanding, promise); + phaseTwoLoop(reader, to, latestForKey, lh, outstanding, promise, m.getMessageId()); } finally { m.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 afbbe6101f8fc..e13dbb734bedb 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 @@ -1926,4 +1926,70 @@ public void testDispatcherMaxReadSizeBytes() throws Exception { consumer.close(); producer.close(); } + + @Test + public void testCompactionDuplicate() throws Exception { + String topic = "persistent://my-property/use/my-ns/testCompactionDuplicate"; + final int numMessages = 1000; + final int maxKeys = 800; + + @Cleanup + Producer producer = pulsarClient.newProducer() + .topic(topic) + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); + + // trigger compaction (create __compaction cursor) + admin.topics().triggerCompaction(topic); + + Map expected = new HashMap<>(); + Random r = new Random(0); + + pulsarClient.newConsumer().topic(topic).subscriptionName("sub1").readCompacted(true).subscribe().close(); + + for (int j = 0; j < numMessages; j++) { + int keyIndex = r.nextInt(maxKeys); + String key = "key" + keyIndex; + byte[] data = ("my-message-" + key + "-" + j).getBytes(); + producer.newMessage().key(key).value(data).send(); + expected.put(key, data); + } + + producer.flush(); + + // trigger compaction + admin.topics().triggerCompaction(topic); + + Awaitility.await().untilAsserted(() -> { + assertEquals(admin.topics().compactionStatus(topic).status, + LongRunningProcessStatus.Status.RUNNING); + }); + + // Wait for phase one to complete + Thread.sleep(500); + + // Unload topic make reader of compaction reconnect + admin.topics().unload(topic); + + Awaitility.await().untilAsserted(() -> { + PersistentTopicInternalStats internalStats = admin.topics().getInternalStats(topic, false); + // Compacted topic ledger should have same number of entry equals to number of unique key. + Assert.assertEquals(internalStats.compactedLedger.entries, expected.size()); + Assert.assertTrue(internalStats.compactedLedger.ledgerId > -1); + Assert.assertFalse(internalStats.compactedLedger.offloaded); + }); + + // consumer with readCompacted enabled only get compacted entries + try (Consumer consumer = pulsarClient.newConsumer().topic(topic).subscriptionName("sub1") + .readCompacted(true).subscribe()) { + while (true) { + Message m = consumer.receive(2, TimeUnit.SECONDS); + Assert.assertEquals(expected.remove(m.getKey()), m.getData()); + if (expected.isEmpty()) { + break; + } + } + } + } } From 12fd09dcd30d02c832fc81c92f05992ce0265356 Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Mon, 4 Sep 2023 17:49:35 +0800 Subject: [PATCH 035/699] [fix][broker] Cleanup correctly heartbeat bundle ownership when handling broker deletion event (#21083) --- .../pulsar/broker/admin/impl/BrokersBase.java | 5 +- .../channel/ServiceUnitStateChannelImpl.java | 10 ++- .../broker/namespace/NamespaceService.java | 38 ++++----- .../pulsar/broker/admin/AdminApiTest.java | 2 +- .../broker/admin/v1/V1_AdminApiTest.java | 2 +- .../ExtensibleLoadManagerImplTest.java | 8 +- .../channel/ServiceUnitStateChannelTest.java | 81 ++++++++++++++----- .../namespace/NamespaceServiceTest.java | 2 +- .../broker/service/BrokerServiceTest.java | 6 +- .../service/InactiveTopicDeleteTest.java | 6 +- .../systopic/PartitionedSystemTopicTest.java | 6 +- .../pulsar/compaction/CompactionTest.java | 4 +- 12 files changed, 106 insertions(+), 64 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 be8390f15f826..3fb1941b33af5 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 @@ -396,9 +396,10 @@ private void checkDeadlockedThreads() { private CompletableFuture internalRunHealthCheck(TopicVersion topicVersion) { + String lookupServiceAddress = pulsar().getLookupServiceAddress(); NamespaceName namespaceName = (topicVersion == TopicVersion.V2) - ? NamespaceService.getHeartbeatNamespaceV2(pulsar().getAdvertisedAddress(), pulsar().getConfiguration()) - : NamespaceService.getHeartbeatNamespace(pulsar().getAdvertisedAddress(), pulsar().getConfiguration()); + ? NamespaceService.getHeartbeatNamespaceV2(lookupServiceAddress, pulsar().getConfiguration()) + : NamespaceService.getHeartbeatNamespace(lookupServiceAddress, pulsar().getConfiguration()); final String topicName = String.format("persistent://%s/%s", namespaceName, HEALTH_CHECK_TOPIC_SUFFIX); LOG.info("[{}] Running healthCheck with topic={}", clientAppId(), topicName); final String messageStr = UUID.randomUUID().toString(); 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 717ff484fe772..98aa02d4e72b4 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 @@ -41,6 +41,8 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.MetadataState.Stable; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.MetadataState.Unstable; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData.state; +import static org.apache.pulsar.broker.namespace.NamespaceService.HEARTBEAT_NAMESPACE_FMT; +import static org.apache.pulsar.broker.namespace.NamespaceService.HEARTBEAT_NAMESPACE_FMT_V2; import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; import static org.apache.pulsar.common.topics.TopicCompactionStrategy.TABLE_VIEW_TAG; import static org.apache.pulsar.metadata.api.extended.SessionEvent.SessionLost; @@ -92,6 +94,7 @@ 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.NamespaceName; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.stats.Metrics; @@ -1214,10 +1217,9 @@ private synchronized void doCleanup(String broker) { int orphanServiceUnitCleanupCnt = 0; long totalCleanupErrorCntStart = totalCleanupErrorCnt.get(); String heartbeatNamespace = - NamespaceService.getHeartbeatNamespace(pulsar.getAdvertisedAddress(), pulsar.getConfiguration()) - .toString(); - String heartbeatNamespaceV2 = NamespaceService.getHeartbeatNamespaceV2(pulsar.getAdvertisedAddress(), - pulsar.getConfiguration()).toString(); + NamespaceName.get(String.format(HEARTBEAT_NAMESPACE_FMT, config.getClusterName(), broker)).toString(); + String heartbeatNamespaceV2 = + NamespaceName.get(String.format(HEARTBEAT_NAMESPACE_FMT_V2, broker)).toString(); Map orphanSystemServiceUnits = new HashMap<>(); for (var etr : tableview.entrySet()) { 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 585d62c5b1f27..d66e3c3b65d76 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 @@ -133,8 +133,8 @@ public class NamespaceService implements AutoCloseable { public static final Pattern HEARTBEAT_NAMESPACE_PATTERN = Pattern.compile("pulsar/[^/]+/([^:]+:\\d+)"); public static final Pattern HEARTBEAT_NAMESPACE_PATTERN_V2 = Pattern.compile("pulsar/([^:]+:\\d+)"); public static final Pattern SLA_NAMESPACE_PATTERN = Pattern.compile(SLA_NAMESPACE_PROPERTY + "/[^/]+/([^:]+:\\d+)"); - public static final String HEARTBEAT_NAMESPACE_FMT = "pulsar/%s/%s:%s"; - public static final String HEARTBEAT_NAMESPACE_FMT_V2 = "pulsar/%s:%s"; + public static final String HEARTBEAT_NAMESPACE_FMT = "pulsar/%s/%s"; + public static final String HEARTBEAT_NAMESPACE_FMT_V2 = "pulsar/%s"; public static final String SLA_NAMESPACE_FMT = SLA_NAMESPACE_PROPERTY + "/%s/%s:%s"; private final ConcurrentOpenHashMap namespaceClients; @@ -164,7 +164,7 @@ public class NamespaceService implements AutoCloseable { */ public NamespaceService(PulsarService pulsar) { this.pulsar = pulsar; - host = pulsar.getAdvertisedAddress(); + this.host = pulsar.getAdvertisedAddress(); this.config = pulsar.getConfiguration(); this.loadManager = pulsar.getLoadManager(); this.bundleFactory = new NamespaceBundleFactory(pulsar, Hashing.crc32()); @@ -332,15 +332,17 @@ private CompletableFuture> internalGetWebServiceUrl(@Nullable Serv * @throws PulsarServerException if an unexpected error occurs */ public void registerBootstrapNamespaces() throws PulsarServerException { - + String lookupServiceAddress = pulsar.getLookupServiceAddress(); // ensure that we own the heartbeat namespace - if (registerNamespace(getHeartbeatNamespace(host, config), true)) { - LOG.info("added heartbeat namespace name in local cache: ns={}", getHeartbeatNamespace(host, config)); + if (registerNamespace(getHeartbeatNamespace(lookupServiceAddress, config), true)) { + LOG.info("added heartbeat namespace name in local cache: ns={}", + getHeartbeatNamespace(lookupServiceAddress, config)); } // ensure that we own the heartbeat namespace - if (registerNamespace(getHeartbeatNamespaceV2(host, config), true)) { - LOG.info("added heartbeat namespace name in local cache: ns={}", getHeartbeatNamespaceV2(host, config)); + if (registerNamespace(getHeartbeatNamespaceV2(lookupServiceAddress, config), true)) { + LOG.info("added heartbeat namespace name in local cache: ns={}", + getHeartbeatNamespaceV2(lookupServiceAddress, config)); } // we may not need strict ownership checking for bootstrap names for now @@ -1579,24 +1581,12 @@ public void unloadSLANamespace() throws Exception { LOG.info("Namespace {} unloaded successfully", namespaceName); } - public static NamespaceName getHeartbeatNamespace(String host, ServiceConfiguration config) { - Integer port = null; - if (config.getWebServicePort().isPresent()) { - port = config.getWebServicePort().get(); - } else if (config.getWebServicePortTls().isPresent()) { - port = config.getWebServicePortTls().get(); - } - return NamespaceName.get(String.format(HEARTBEAT_NAMESPACE_FMT, config.getClusterName(), host, port)); + public static NamespaceName getHeartbeatNamespace(String lookupBroker, ServiceConfiguration config) { + return NamespaceName.get(String.format(HEARTBEAT_NAMESPACE_FMT, config.getClusterName(), lookupBroker)); } - public static NamespaceName getHeartbeatNamespaceV2(String host, ServiceConfiguration config) { - Integer port = null; - if (config.getWebServicePort().isPresent()) { - port = config.getWebServicePort().get(); - } else if (config.getWebServicePortTls().isPresent()) { - port = config.getWebServicePortTls().get(); - } - return NamespaceName.get(String.format(HEARTBEAT_NAMESPACE_FMT_V2, host, port)); + public static NamespaceName getHeartbeatNamespaceV2(String lookupBroker, ServiceConfiguration config) { + return NamespaceName.get(String.format(HEARTBEAT_NAMESPACE_FMT_V2, lookupBroker)); } public static NamespaceName getSLAMonitorNamespace(String host, ServiceConfiguration config) { 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 a4be829b977f7..9fb3cb6fb2328 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 @@ -534,7 +534,7 @@ public void brokers() throws Exception { for (String ns : nsMap.keySet()) { NamespaceOwnershipStatus nsStatus = nsMap.get(ns); if (ns.equals( - NamespaceService.getHeartbeatNamespace(pulsar.getAdvertisedAddress(), pulsar.getConfiguration()) + NamespaceService.getHeartbeatNamespace(pulsar.getLookupServiceAddress(), pulsar.getConfiguration()) + "/0x00000000_0xffffffff")) { assertEquals(nsStatus.broker_assignment, BrokerAssignment.shared); assertFalse(nsStatus.is_controlled); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest.java index ab83c8fec032e..8922a90e95843 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v1/V1_AdminApiTest.java @@ -450,7 +450,7 @@ public void brokers() throws Exception { for (String ns : nsMap.keySet()) { NamespaceOwnershipStatus nsStatus = nsMap.get(ns); if (ns.equals( - NamespaceService.getHeartbeatNamespace(pulsar.getAdvertisedAddress(), pulsar.getConfiguration()) + NamespaceService.getHeartbeatNamespace(pulsar.getLookupServiceAddress(), pulsar.getConfiguration()) + "/0x00000000_0xffffffff")) { assertEquals(nsStatus.broker_assignment, BrokerAssignment.shared); assertFalse(nsStatus.is_controlled); 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 cbb3e174a0d87..9ce57a88540c3 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 @@ -1046,14 +1046,14 @@ public void testListTopic() throws Exception { @Test(timeOut = 30 * 1000) public void testGetOwnedServiceUnitsAndGetOwnedNamespaceStatus() throws Exception { NamespaceName heartbeatNamespacePulsar1V1 = - NamespaceService.getHeartbeatNamespace(pulsar1.getAdvertisedAddress(), pulsar1.getConfiguration()); + NamespaceService.getHeartbeatNamespace(pulsar1.getLookupServiceAddress(), pulsar1.getConfiguration()); NamespaceName heartbeatNamespacePulsar1V2 = - NamespaceService.getHeartbeatNamespaceV2(pulsar1.getAdvertisedAddress(), pulsar1.getConfiguration()); + NamespaceService.getHeartbeatNamespaceV2(pulsar1.getLookupServiceAddress(), pulsar1.getConfiguration()); NamespaceName heartbeatNamespacePulsar2V1 = - NamespaceService.getHeartbeatNamespace(pulsar2.getAdvertisedAddress(), pulsar2.getConfiguration()); + NamespaceService.getHeartbeatNamespace(pulsar2.getLookupServiceAddress(), pulsar2.getConfiguration()); NamespaceName heartbeatNamespacePulsar2V2 = - NamespaceService.getHeartbeatNamespaceV2(pulsar2.getAdvertisedAddress(), pulsar2.getConfiguration()); + NamespaceService.getHeartbeatNamespaceV2(pulsar2.getLookupServiceAddress(), pulsar2.getConfiguration()); NamespaceBundle bundle1 = pulsar1.getNamespaceService().getNamespaceBundleFactory() .getFullBundle(heartbeatNamespacePulsar1V1); 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 ace31b1d60bb2..de21842f4584c 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 @@ -30,6 +30,8 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.EventType.Unload; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.MAX_CLEAN_UP_DELAY_TIME_IN_SECS; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData.state; +import static org.apache.pulsar.broker.namespace.NamespaceService.HEARTBEAT_NAMESPACE_FMT; +import static org.apache.pulsar.broker.namespace.NamespaceService.HEARTBEAT_NAMESPACE_FMT_V2; import static org.apache.pulsar.metadata.api.extended.SessionEvent.ConnectionLost; import static org.apache.pulsar.metadata.api.extended.SessionEvent.Reconnected; import static org.apache.pulsar.metadata.api.extended.SessionEvent.SessionLost; @@ -87,6 +89,7 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.client.impl.TableViewImpl; +import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.policies.data.TopicType; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.metadata.api.MetadataStoreException; @@ -747,11 +750,41 @@ public void handleBrokerDeletionEventTest() String broker = lookupServiceAddress1; channel1.publishAssignEventAsync(bundle1, broker); channel2.publishAssignEventAsync(bundle2, broker); + waitUntilNewOwner(channel1, bundle1, broker); waitUntilNewOwner(channel2, bundle1, broker); waitUntilNewOwner(channel1, bundle2, broker); waitUntilNewOwner(channel2, bundle2, broker); + // Register the broker-1 heartbeat namespace bundle. + String heartbeatNamespaceBroker1V1 = NamespaceName + .get(String.format(HEARTBEAT_NAMESPACE_FMT, conf.getClusterName(), broker)).toString(); + String heartbeatNamespaceBroker1V2 = NamespaceName + .get(String.format(HEARTBEAT_NAMESPACE_FMT_V2, broker)).toString(); + String heartbeatNamespaceBroker1V1Bundle = heartbeatNamespaceBroker1V1 + "/0x00000000_0xfffffff0"; + String heartbeatNamespaceBroker1V2Bundle = heartbeatNamespaceBroker1V2 + "/0x00000000_0xfffffff0"; + channel1.publishAssignEventAsync(heartbeatNamespaceBroker1V1Bundle, broker); + channel1.publishAssignEventAsync(heartbeatNamespaceBroker1V2Bundle, broker); + + // Register the broker-2 heartbeat namespace bundle. + String heartbeatNamespaceBroker2V1 = NamespaceName + .get(String.format(HEARTBEAT_NAMESPACE_FMT, conf.getClusterName(), lookupServiceAddress2)).toString(); + String heartbeatNamespaceBroker2V2 = NamespaceName + .get(String.format(HEARTBEAT_NAMESPACE_FMT_V2, lookupServiceAddress2)).toString(); + String heartbeatNamespaceBroker2V1Bundle = heartbeatNamespaceBroker2V1 + "/0x00000000_0xfffffff0"; + String heartbeatNamespaceBroker2V2Bundle = heartbeatNamespaceBroker2V2 + "/0x00000000_0xfffffff0"; + channel1.publishAssignEventAsync(heartbeatNamespaceBroker2V1Bundle, lookupServiceAddress2); + channel1.publishAssignEventAsync(heartbeatNamespaceBroker2V2Bundle, lookupServiceAddress2); + waitUntilNewOwner(channel1, heartbeatNamespaceBroker1V1Bundle, broker); + waitUntilNewOwner(channel1, heartbeatNamespaceBroker1V2Bundle, broker); + waitUntilNewOwner(channel2, heartbeatNamespaceBroker1V1Bundle, broker); + waitUntilNewOwner(channel2, heartbeatNamespaceBroker1V2Bundle, broker); + waitUntilNewOwner(channel1, heartbeatNamespaceBroker2V1Bundle, lookupServiceAddress2); + waitUntilNewOwner(channel1, heartbeatNamespaceBroker2V2Bundle, lookupServiceAddress2); + waitUntilNewOwner(channel2, heartbeatNamespaceBroker2V1Bundle, lookupServiceAddress2); + waitUntilNewOwner(channel2, heartbeatNamespaceBroker2V2Bundle, lookupServiceAddress2); + + // Verify to transfer the ownership to the other broker. channel1.publishUnloadEventAsync(new Unload(broker, bundle1, Optional.of(lookupServiceAddress2))); waitUntilNewOwner(channel1, bundle1, lookupServiceAddress2); waitUntilNewOwner(channel2, bundle1, lookupServiceAddress2); @@ -765,12 +798,24 @@ public void handleBrokerDeletionEventTest() System.currentTimeMillis() - (MAX_CLEAN_UP_DELAY_TIME_IN_SECS * 1000 + 1000), true); leaderChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); followerChannel.handleBrokerRegistrationEvent(broker, NotificationType.Deleted); + leaderChannel.handleBrokerRegistrationEvent(lookupServiceAddress2, NotificationType.Deleted); + followerChannel.handleBrokerRegistrationEvent(lookupServiceAddress2, NotificationType.Deleted); waitUntilNewOwner(channel1, bundle1, lookupServiceAddress2); waitUntilNewOwner(channel2, bundle1, lookupServiceAddress2); waitUntilNewOwner(channel1, bundle2, lookupServiceAddress2); waitUntilNewOwner(channel2, bundle2, lookupServiceAddress2); + waitUntilNewOwner(channel1, heartbeatNamespaceBroker1V1Bundle, null); + waitUntilNewOwner(channel1, heartbeatNamespaceBroker1V2Bundle, null); + waitUntilNewOwner(channel2, heartbeatNamespaceBroker1V1Bundle, null); + waitUntilNewOwner(channel2, heartbeatNamespaceBroker1V2Bundle, null); + + waitUntilNewOwner(channel1, heartbeatNamespaceBroker2V1Bundle, null); + waitUntilNewOwner(channel1, heartbeatNamespaceBroker2V2Bundle, null); + waitUntilNewOwner(channel2, heartbeatNamespaceBroker2V1Bundle, null); + waitUntilNewOwner(channel2, heartbeatNamespaceBroker2V2Bundle, null); + verify(leaderCleanupJobs, times(1)).computeIfAbsent(eq(broker), any()); verify(followerCleanupJobs, times(0)).computeIfAbsent(eq(broker), any()); @@ -780,11 +825,11 @@ public void handleBrokerDeletionEventTest() }); validateMonitorCounters(leaderChannel, - 1, + 2, 0, - 1, + 7, 0, - 1, + 2, 0, 0); @@ -811,11 +856,11 @@ public void handleBrokerDeletionEventTest() }); validateMonitorCounters(leaderChannel, - 1, + 2, 0, - 1, + 7, 0, - 2, + 3, 0, 0); @@ -832,11 +877,11 @@ public void handleBrokerDeletionEventTest() }); validateMonitorCounters(leaderChannel, - 1, + 2, 0, - 1, + 7, 0, - 2, + 3, 0, 1); @@ -854,11 +899,11 @@ public void handleBrokerDeletionEventTest() }); validateMonitorCounters(leaderChannel, - 1, + 2, 0, - 1, + 7, 0, - 3, + 4, 0, 1); @@ -876,11 +921,11 @@ public void handleBrokerDeletionEventTest() }); validateMonitorCounters(leaderChannel, - 2, - 0, 3, 0, - 3, + 9, + 0, + 4, 0, 1); @@ -905,11 +950,11 @@ public void handleBrokerDeletionEventTest() }); validateMonitorCounters(leaderChannel, - 2, - 0, 3, 0, - 3, + 9, + 0, + 4, 1, 1); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java index ac5d92c880227..03bb53eb9da24 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java @@ -683,7 +683,7 @@ public void testSplitBundleWithHighestThroughput() throws Exception { @Test public void testHeartbeatNamespaceMatch() throws Exception { - NamespaceName namespaceName = NamespaceService.getHeartbeatNamespace(pulsar.getAdvertisedAddress(), conf); + NamespaceName namespaceName = NamespaceService.getHeartbeatNamespace(pulsar.getLookupServiceAddress(), conf); NamespaceBundle namespaceBundle = pulsar.getNamespaceService().getNamespaceBundleFactory().getFullBundle(namespaceName); assertTrue(NamespaceService.isSystemServiceNamespace( NamespaceBundle.getBundleNamespace(namespaceBundle.toString()))); 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 c61da7fc03bdb..0977fb70dce85 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 @@ -1594,8 +1594,10 @@ public void testIsSystemTopic() { assertTrue(brokerService.isSystemTopic(TRANSACTION_COORDINATOR_ASSIGN)); assertTrue(brokerService.isSystemTopic(TRANSACTION_COORDINATOR_LOG)); - NamespaceName heartbeatNamespaceV1 = NamespaceService.getHeartbeatNamespace(pulsar.getAdvertisedAddress(), pulsar.getConfig()); - NamespaceName heartbeatNamespaceV2 = NamespaceService.getHeartbeatNamespaceV2(pulsar.getAdvertisedAddress(), pulsar.getConfig()); + NamespaceName heartbeatNamespaceV1 = NamespaceService + .getHeartbeatNamespace(pulsar.getLookupServiceAddress(), pulsar.getConfig()); + NamespaceName heartbeatNamespaceV2 = NamespaceService + .getHeartbeatNamespaceV2(pulsar.getLookupServiceAddress(), pulsar.getConfig()); assertTrue(brokerService.isSystemTopic("persistent://" + heartbeatNamespaceV1.toString() + "/healthcheck")); assertTrue(brokerService.isSystemTopic(heartbeatNamespaceV2.toString() + "/healthcheck")); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/InactiveTopicDeleteTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/InactiveTopicDeleteTest.java index 0ce9a33283bfa..84c4670f2bb3b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/InactiveTopicDeleteTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/InactiveTopicDeleteTest.java @@ -598,10 +598,12 @@ public void testHealthTopicInactiveNotClean() throws Exception { conf.setBrokerDeleteInactiveTopicsFrequencySeconds(1); super.baseSetup(); // init topic - NamespaceName heartbeatNamespaceV1 = NamespaceService.getHeartbeatNamespace(pulsar.getAdvertisedAddress(), pulsar.getConfig()); + NamespaceName heartbeatNamespaceV1 = NamespaceService + .getHeartbeatNamespace(pulsar.getLookupServiceAddress(), pulsar.getConfig()); final String healthCheckTopicV1 = "persistent://" + heartbeatNamespaceV1 + "/healthcheck"; - NamespaceName heartbeatNamespaceV2 = NamespaceService.getHeartbeatNamespaceV2(pulsar.getAdvertisedAddress(), pulsar.getConfig()); + NamespaceName heartbeatNamespaceV2 = NamespaceService + .getHeartbeatNamespaceV2(pulsar.getLookupServiceAddress(), pulsar.getConfig()); final String healthCheckTopicV2 = "persistent://" + heartbeatNamespaceV2 + "/healthcheck"; admin.brokers().healthcheck(TopicVersion.V1); 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 008c2143a3566..6f56852cae343 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 @@ -160,7 +160,7 @@ public void testProduceAndConsumeUnderSystemNamespace() throws Exception { @Test public void testHealthCheckTopicNotOffload() throws Exception { - NamespaceName namespaceName = NamespaceService.getHeartbeatNamespaceV2(pulsar.getAdvertisedAddress(), + NamespaceName namespaceName = NamespaceService.getHeartbeatNamespaceV2(pulsar.getLookupServiceAddress(), pulsar.getConfig()); TopicName topicName = TopicName.get("persistent", namespaceName, BrokersBase.HEALTH_CHECK_TOPIC_SUFFIX); PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService() @@ -180,7 +180,7 @@ public void testHealthCheckTopicNotOffload() throws Exception { @Test public void testSystemNamespaceNotCreateChangeEventsTopic() throws Exception { admin.brokers().healthcheck(TopicVersion.V2); - NamespaceName namespaceName = NamespaceService.getHeartbeatNamespaceV2(pulsar.getAdvertisedAddress(), + NamespaceName namespaceName = NamespaceService.getHeartbeatNamespaceV2(pulsar.getLookupServiceAddress(), pulsar.getConfig()); TopicName topicName = TopicName.get("persistent", namespaceName, SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME); Optional optionalTopic = pulsar.getBrokerService() @@ -191,7 +191,7 @@ public void testSystemNamespaceNotCreateChangeEventsTopic() throws Exception { @Test public void testHeartbeatTopicNotAllowedToSendEvent() throws Exception { admin.brokers().healthcheck(TopicVersion.V2); - NamespaceName namespaceName = NamespaceService.getHeartbeatNamespaceV2(pulsar.getAdvertisedAddress(), + NamespaceName namespaceName = NamespaceService.getHeartbeatNamespaceV2(pulsar.getLookupServiceAddress(), pulsar.getConfig()); TopicName topicName = TopicName.get("persistent", namespaceName, SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME); for (int partition = 0; partition < PARTITIONS; partition ++) { 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 e13dbb734bedb..5e2f06fb4666f 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 @@ -1773,9 +1773,9 @@ public void testReadUnCompacted(boolean batchEnabled) throws PulsarClientExcepti @SneakyThrows @Test public void testHealthCheckTopicNotCompacted() { - NamespaceName heartbeatNamespaceV1 = NamespaceService.getHeartbeatNamespace(pulsar.getAdvertisedAddress(), pulsar.getConfiguration()); + NamespaceName heartbeatNamespaceV1 = NamespaceService.getHeartbeatNamespace(pulsar.getLookupServiceAddress(), pulsar.getConfiguration()); String topicV1 = "persistent://" + heartbeatNamespaceV1.toString() + "/healthcheck"; - NamespaceName heartbeatNamespaceV2 = NamespaceService.getHeartbeatNamespaceV2(pulsar.getAdvertisedAddress(), pulsar.getConfiguration()); + NamespaceName heartbeatNamespaceV2 = NamespaceService.getHeartbeatNamespaceV2(pulsar.getLookupServiceAddress(), pulsar.getConfiguration()); String topicV2 = heartbeatNamespaceV2.toString() + "/healthcheck"; Producer producer1 = pulsarClient.newProducer().topic(topicV1).create(); Producer producer2 = pulsarClient.newProducer().topic(topicV2).create(); From 89b567571453df21335eafbde21b4f5743757322 Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Mon, 4 Sep 2023 14:13:06 +0800 Subject: [PATCH 036/699] [fix][broker] Avoid splitting one batch message into two entries in StrategicTwoPhaseCompactor (#21091) --- .../impl/RawBatchMessageContainerImpl.java | 41 +++++++-- .../StrategicTwoPhaseCompactor.java | 90 ++++++++----------- .../RawBatchMessageContainerImplTest.java | 53 +++++------ .../pulsar/compaction/CompactionTest.java | 11 +-- .../StrategicCompactionRetentionTest.java | 2 +- .../compaction/StrategicCompactionTest.java | 81 +++++++++++++---- .../compaction/StrategicCompactorTest.java | 4 +- 7 files changed, 170 insertions(+), 112 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 7e1c2cd5e3fe3..ba8d3db7178d9 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 @@ -23,6 +23,7 @@ import java.util.Set; import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.MessageCrypto; +import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.crypto.MessageCryptoBc; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; @@ -44,17 +45,17 @@ * [(k1, v1), (k2, v1), (k3, v1), (k1, v2), (k2, v2), (k3, v2), (k1, v3), (k2, v3), (k3, v3)] */ public class RawBatchMessageContainerImpl extends BatchMessageContainerImpl { - MessageCrypto msgCrypto; - Set encryptionKeys; - CryptoKeyReader cryptoKeyReader; + private MessageCrypto msgCrypto; + private Set encryptionKeys; + private CryptoKeyReader cryptoKeyReader; + private MessageIdAdv lastAddedMessageId; - public RawBatchMessageContainerImpl(int maxNumMessagesInBatch, int maxBytesInBatch) { + public RawBatchMessageContainerImpl() { super(); this.compressionType = CompressionType.NONE; this.compressor = new CompressionCodecNone(); - this.maxNumMessagesInBatch = maxNumMessagesInBatch; - this.maxBytesInBatch = maxBytesInBatch; } + private ByteBuf encrypt(ByteBuf compressedPayload) { if (msgCrypto == null) { return compressedPayload; @@ -90,6 +91,28 @@ public void setCryptoKeyReader(CryptoKeyReader cryptoKeyReader) { this.cryptoKeyReader = cryptoKeyReader; } + @Override + public boolean add(MessageImpl msg, SendCallback callback) { + this.lastAddedMessageId = (MessageIdAdv) msg.getMessageId(); + return super.add(msg, callback); + } + + @Override + protected boolean isBatchFull() { + return false; + } + + @Override + public boolean haveEnoughSpace(MessageImpl msg) { + if (lastAddedMessageId == null) { + return true; + } + // Keep same batch compact to same batch. + MessageIdAdv msgId = (MessageIdAdv) msg.getMessageId(); + return msgId.getLedgerId() == lastAddedMessageId.getLedgerId() + && msgId.getEntryId() == lastAddedMessageId.getEntryId(); + } + /** * Serializes the batched messages and return the ByteBuf. * It sets the CompressionType and Encryption Keys from the batched messages. @@ -168,4 +191,10 @@ public ByteBuf toByteBuf() { clear(); return buf; } + + @Override + public void clear() { + this.lastAddedMessageId = null; + super.clear(); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/StrategicTwoPhaseCompactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/StrategicTwoPhaseCompactor.java index a6b0942742763..fefa2ee959cc5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/StrategicTwoPhaseCompactor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/StrategicTwoPhaseCompactor.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.compaction; -import com.google.common.annotations.VisibleForTesting; import io.netty.buffer.ByteBuf; import java.time.Duration; import java.util.Iterator; @@ -63,39 +62,19 @@ public class StrategicTwoPhaseCompactor extends TwoPhaseCompactor { private static final Logger log = LoggerFactory.getLogger(StrategicTwoPhaseCompactor.class); private static final int MAX_OUTSTANDING = 500; - private static final int MAX_NUM_MESSAGES_IN_BATCH = 1000; - private static final int MAX_BYTES_IN_BATCH = 128 * 1024; private static final int MAX_READER_RECONNECT_WAITING_TIME_IN_MILLIS = 20 * 1000; private final Duration phaseOneLoopReadTimeout; private final RawBatchMessageContainerImpl batchMessageContainer; - @VisibleForTesting public StrategicTwoPhaseCompactor(ServiceConfiguration conf, PulsarClient pulsar, BookKeeper bk, - ScheduledExecutorService scheduler, - int maxNumMessagesInBatch) { - this(conf, pulsar, bk, scheduler, maxNumMessagesInBatch, MAX_BYTES_IN_BATCH); - } - - private StrategicTwoPhaseCompactor(ServiceConfiguration conf, - PulsarClient pulsar, - BookKeeper bk, - ScheduledExecutorService scheduler, - int maxNumMessagesInBatch, - int maxBytesInBatch) { + ScheduledExecutorService scheduler) { super(conf, pulsar, bk, scheduler); - batchMessageContainer = new RawBatchMessageContainerImpl(maxNumMessagesInBatch, maxBytesInBatch); + batchMessageContainer = new RawBatchMessageContainerImpl(); phaseOneLoopReadTimeout = Duration.ofSeconds(conf.getBrokerServiceCompactionPhaseOneLoopTimeInSeconds()); } - public StrategicTwoPhaseCompactor(ServiceConfiguration conf, - PulsarClient pulsar, - BookKeeper bk, - ScheduledExecutorService scheduler) { - this(conf, pulsar, bk, scheduler, MAX_NUM_MESSAGES_IN_BATCH, MAX_BYTES_IN_BATCH); - } - public CompletableFuture compact(String topic) { throw new UnsupportedOperationException(); } @@ -418,7 +397,6 @@ private void phaseTwoLoop(String topic, Iterator> reader, .whenComplete((res, exception2) -> { if (exception2 != null) { promise.completeExceptionally(exception2); - return; } }); phaseTwoLoop(topic, reader, lh, outstanding, promise); @@ -443,35 +421,45 @@ private void phaseTwoLoop(String topic, Iterator> reader, CompletableFuture addToCompactedLedger( LedgerHandle lh, Message m, String topic, Semaphore outstanding) { + if (m == null) { + return flushBatchMessage(lh, topic, outstanding); + } + if (batchMessageContainer.haveEnoughSpace((MessageImpl) m)) { + batchMessageContainer.add((MessageImpl) m, null); + return CompletableFuture.completedFuture(false); + } + CompletableFuture f = flushBatchMessage(lh, topic, outstanding); + batchMessageContainer.add((MessageImpl) m, null); + return f; + } + + private CompletableFuture flushBatchMessage(LedgerHandle lh, String topic, + Semaphore outstanding) { + if (batchMessageContainer.getNumMessagesInBatch() <= 0) { + return CompletableFuture.completedFuture(false); + } CompletableFuture bkf = new CompletableFuture<>(); - if (m == null || batchMessageContainer.add((MessageImpl) m, null)) { - if (batchMessageContainer.getNumMessagesInBatch() > 0) { - try { - ByteBuf serialized = batchMessageContainer.toByteBuf(); - outstanding.acquire(); - mxBean.addCompactionWriteOp(topic, serialized.readableBytes()); - long start = System.nanoTime(); - lh.asyncAddEntry(serialized, - (rc, ledger, eid, ctx) -> { - outstanding.release(); - mxBean.addCompactionLatencyOp(topic, System.nanoTime() - start, TimeUnit.NANOSECONDS); - if (rc != BKException.Code.OK) { - bkf.completeExceptionally(BKException.create(rc)); - } else { - bkf.complete(true); - } - }, null); + try { + ByteBuf serialized = batchMessageContainer.toByteBuf(); + outstanding.acquire(); + mxBean.addCompactionWriteOp(topic, serialized.readableBytes()); + long start = System.nanoTime(); + lh.asyncAddEntry(serialized, + (rc, ledger, eid, ctx) -> { + outstanding.release(); + mxBean.addCompactionLatencyOp(topic, System.nanoTime() - start, TimeUnit.NANOSECONDS); + if (rc != BKException.Code.OK) { + bkf.completeExceptionally(BKException.create(rc)); + } else { + bkf.complete(true); + } + }, null); - } catch (Throwable t) { - log.error("Failed to add entry", t); - batchMessageContainer.discard((Exception) t); - return FutureUtil.failedFuture(t); - } - } else { - bkf.complete(false); - } - } else { - bkf.complete(false); + } catch (Throwable t) { + log.error("Failed to add entry", t); + batchMessageContainer.discard((Exception) t); + bkf.completeExceptionally(t); + return bkf; } return bkf; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawBatchMessageContainerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawBatchMessageContainerImplTest.java index 9b8b1e5efb99c..d79a31c07f218 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawBatchMessageContainerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/RawBatchMessageContainerImplTest.java @@ -47,7 +47,6 @@ import org.apache.pulsar.compaction.CompactionTest; import org.testng.Assert; import org.testng.annotations.BeforeMethod; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; public class RawBatchMessageContainerImplTest { @@ -56,8 +55,6 @@ public class RawBatchMessageContainerImplTest { CryptoKeyReader cryptoKeyReader; Map encryptKeys; - int maxBytesInBatch = 5 * 1024 * 1024; - public void setEncryptionAndCompression(boolean encrypt, boolean compress) { if (compress) { compressionType = ZSTD; @@ -107,22 +104,22 @@ public MessageImpl createMessage(String topic, String value, int entryId) { public void setup() throws Exception { setEncryptionAndCompression(false, true); } - @DataProvider(name = "testBatchLimitByMessageCount") - public static Object[][] testBatchLimitByMessageCount() { - return new Object[][] {{true}, {false}}; - } - - @Test(timeOut = 20000, dataProvider = "testBatchLimitByMessageCount") - public void testToByteBufWithBatchLimit(boolean testBatchLimitByMessageCount) throws IOException { - RawBatchMessageContainerImpl container = testBatchLimitByMessageCount ? - new RawBatchMessageContainerImpl(2, Integer.MAX_VALUE) : - new RawBatchMessageContainerImpl(Integer.MAX_VALUE, 5); + @Test(timeOut = 20000) + public void testToByteBufWithBatchLimit()throws IOException { + RawBatchMessageContainerImpl container = new RawBatchMessageContainerImpl(); String topic = "my-topic"; - var full1 = container.add(createMessage(topic, "hi-1", 0), null); - var full2 = container.add(createMessage(topic, "hi-2", 1), null); + MessageImpl message1 = createMessage(topic, "hi-1", 0); + boolean hasEnoughSpase1 = container.haveEnoughSpace(message1); + var full1 = container.add(message1, null); assertFalse(full1); - assertTrue(full2); + assertTrue(hasEnoughSpase1); + MessageImpl message2 = createMessage(topic, "hi-2", 1); + boolean hasEnoughSpase2 = container.haveEnoughSpace(message2); + assertFalse(hasEnoughSpase2); + var full2 = container.add(message2, null); + assertFalse(full2); + ByteBuf buf = container.toByteBuf(); @@ -167,7 +164,7 @@ public void testToByteBufWithBatchLimit(boolean testBatchLimitByMessageCount) th public void testToByteBufWithCompressionAndEncryption() throws IOException { setEncryptionAndCompression(true, true); - RawBatchMessageContainerImpl container = new RawBatchMessageContainerImpl(2, maxBytesInBatch); + RawBatchMessageContainerImpl container = new RawBatchMessageContainerImpl(); container.setCryptoKeyReader(cryptoKeyReader); String topic = "my-topic"; container.add(createMessage(topic, "hi-1", 0), null); @@ -217,7 +214,7 @@ public void testToByteBufWithCompressionAndEncryption() throws IOException { @Test public void testToByteBufWithSingleMessage() throws IOException { - RawBatchMessageContainerImpl container = new RawBatchMessageContainerImpl(2, maxBytesInBatch); + RawBatchMessageContainerImpl container = new RawBatchMessageContainerImpl(); String topic = "my-topic"; container.add(createMessage(topic, "hi-1", 0), null); ByteBuf buf = container.toByteBuf(); @@ -250,25 +247,31 @@ public void testToByteBufWithSingleMessage() throws IOException { } @Test - public void testMaxNumMessagesInBatch() { - RawBatchMessageContainerImpl container = new RawBatchMessageContainerImpl(1, maxBytesInBatch); + public void testAddDifferentBatchMessage() { + RawBatchMessageContainerImpl container = new RawBatchMessageContainerImpl(); String topic = "my-topic"; boolean isFull = container.add(createMessage(topic, "hi", 0), null); - Assert.assertTrue(isFull); - Assert.assertTrue(container.isBatchFull()); + Assert.assertFalse(isFull); + Assert.assertFalse(container.isBatchFull()); + MessageImpl message = createMessage(topic, "hi-1", 0); + Assert.assertTrue(container.haveEnoughSpace(message)); + isFull = container.add(message, null); + Assert.assertFalse(isFull); + message = createMessage(topic, "hi-2", 1); + Assert.assertFalse(container.haveEnoughSpace(message)); } @Test(expectedExceptions = UnsupportedOperationException.class) public void testCreateOpSendMsg() { - RawBatchMessageContainerImpl container = new RawBatchMessageContainerImpl(1, maxBytesInBatch); + RawBatchMessageContainerImpl container = new RawBatchMessageContainerImpl(); container.createOpSendMsg(); } @Test public void testToByteBufWithEncryptionWithoutCryptoKeyReader() { setEncryptionAndCompression(true, false); - RawBatchMessageContainerImpl container = new RawBatchMessageContainerImpl(1, maxBytesInBatch); + RawBatchMessageContainerImpl container = new RawBatchMessageContainerImpl(); String topic = "my-topic"; container.add(createMessage(topic, "hi-1", 0), null); Assert.assertEquals(container.getNumMessagesInBatch(), 1); @@ -286,7 +289,7 @@ public void testToByteBufWithEncryptionWithoutCryptoKeyReader() { @Test public void testToByteBufWithEncryptionWithInvalidEncryptKeys() { setEncryptionAndCompression(true, false); - RawBatchMessageContainerImpl container = new RawBatchMessageContainerImpl(1, maxBytesInBatch); + RawBatchMessageContainerImpl container = new RawBatchMessageContainerImpl(); container.setCryptoKeyReader(cryptoKeyReader); encryptKeys = new HashMap<>(); encryptKeys.put(null, null); 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 5e2f06fb4666f..4c6db644f1e01 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 @@ -81,7 +81,6 @@ import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.ConsumerImpl; -import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; @@ -542,14 +541,8 @@ public void testBatchMessageIdsDontChange() throws Exception { Assert.assertEquals(message2.getKey(), "key2"); Assert.assertEquals(new String(message2.getData()), "my-message-3"); if (getCompactor() instanceof StrategicTwoPhaseCompactor) { - MessageIdImpl id = (MessageIdImpl) messages.get(0).getMessageId(); - MessageIdImpl id1 = new MessageIdImpl( - id.getLedgerId(), id.getEntryId(), id.getPartitionIndex()); - Assert.assertEquals(message1.getMessageId(), id1); - id = (MessageIdImpl) messages.get(2).getMessageId(); - MessageIdImpl id2 = new MessageIdImpl( - id.getLedgerId(), id.getEntryId(), id.getPartitionIndex()); - Assert.assertEquals(message2.getMessageId(), id2); + Assert.assertEquals(message1.getMessageId(), messages.get(0).getMessageId()); + Assert.assertEquals(message2.getMessageId(), messages.get(1).getMessageId()); } else { Assert.assertEquals(message1.getMessageId(), messages.get(0).getMessageId()); Assert.assertEquals(message2.getMessageId(), messages.get(2).getMessageId()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionRetentionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionRetentionTest.java index 1cac04c2fa956..e556ec8e0b200 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionRetentionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionRetentionTest.java @@ -34,7 +34,7 @@ public class StrategicCompactionRetentionTest extends CompactionRetentionTest { @Override public void setup() throws Exception { super.setup(); - compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler, 1); + compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler); strategy = new TopicCompactionStrategyTest.DummyTopicCompactionStrategy(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionTest.java index 799c2703e1ed4..54563431052eb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactionTest.java @@ -18,22 +18,33 @@ */ package org.apache.pulsar.compaction; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.MSG_COMPRESSION_TYPE; +import static org.testng.Assert.assertEquals; + import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Random; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.api.Consumer; 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.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerBuilder; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.TableView; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.topics.TopicCompactionStrategy; +import org.apache.pulsar.common.util.FutureUtil; import org.testng.Assert; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -47,7 +58,7 @@ public class StrategicCompactionTest extends CompactionTest { @Override public void setup() throws Exception { super.setup(); - compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler, 1); + compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler); strategy = new TopicCompactionStrategyTest.DummyTopicCompactionStrategy(); } @@ -148,24 +159,58 @@ public void testNumericOrderCompaction() throws Exception { Assert.assertEquals(tableView.entrySet(), expectedCopy.entrySet()); } - @Override - public void testCompactCompressedBatching() throws Exception { - compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler, 10); - super.testCompactCompressedBatching(); - compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler, 1); - } + @Test(timeOut = 20000) + public void testSameBatchCompactToSameBatch() throws Exception { + final String topic = + "persistent://my-property/use/my-ns/testSameBatchCompactToSameBatch" + UUID.randomUUID(); - @Override - public void testCompactEncryptedAndCompressedBatching() throws Exception { - compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler, 10); - super.testCompactEncryptedAndCompressedBatching(); - compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler, 1); - } + // Use odd number to make sure the last message is flush by `reader.hasNext() == false`. + final int messages = 11; + + // 1.create producer and publish message to the topic. + ProducerBuilder builder = pulsarClient.newProducer(Schema.INT32) + .compressionType(MSG_COMPRESSION_TYPE).topic(topic); + builder.batchingMaxMessages(2) + .batchingMaxPublishDelay(10, TimeUnit.MILLISECONDS); + + Producer producer = builder.create(); + + List> futures = new ArrayList<>(messages); + for (int i = 0; i < messages; i++) { + futures.add(producer.newMessage().key(String.valueOf(i)) + .value(i) + .sendAsync()); + } + FutureUtil.waitForAll(futures).get(); + + // 2.compact the topic. + StrategicTwoPhaseCompactor compactor + = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler); + compactor.compact(topic, strategy).get(); + + // consumer with readCompacted enabled only get compacted entries + try (Consumer consumer = pulsarClient + .newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName("sub1") + .readCompacted(true) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest).subscribe()) { + int received = 0; + while (true) { + Message m = consumer.receive(2, TimeUnit.SECONDS); + if (m == null) { + break; + } + MessageIdAdv messageId = (MessageIdAdv) m.getMessageId(); + if (received < messages - 1) { + assertEquals(messageId.getBatchSize(), 2); + } else { + assertEquals(messageId.getBatchSize(), 0); + } + received++; + } + assertEquals(received, messages); + } - @Override - public void testCompactEncryptedBatching() throws Exception { - compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler, 10); - super.testCompactEncryptedBatching(); - compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler, 1); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactorTest.java index 91dd8a2bd358b..bc65791b323cd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/StrategicCompactorTest.java @@ -33,7 +33,7 @@ public class StrategicCompactorTest extends CompactorTest { @Override public void setup() throws Exception { super.setup(); - compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler, 1); + compactor = new StrategicTwoPhaseCompactor(conf, pulsarClient, bk, compactionScheduler); strategy = new TopicCompactionStrategyTest.DummyTopicCompactionStrategy(); } @@ -46,4 +46,4 @@ protected long compact(String topic) throws ExecutionException, InterruptedExcep protected Compactor getCompactor() { return compactor; } -} \ No newline at end of file +} From 67f34a8e1c6bd23848c5795716725643d8b98e09 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=90=A7=E6=98=93=E5=AE=A2?= Date: Thu, 31 Aug 2023 16:18:17 +0800 Subject: [PATCH 037/699] [fix][client] Fix logging problem in pulsar client (#21094) --- .../java/org/apache/pulsar/client/impl/ConsumerImpl.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 ca3d2727ef0c0..06a0c1dddddc3 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 @@ -421,7 +421,7 @@ public CompletableFuture unsubscribeAsync() { unsubscribeFuture.completeExceptionally( PulsarClientException.wrap(e.getCause(), String.format("Failed to unsubscribe the subscription %s of topic %s", - topicName.toString(), subscription))); + subscription, topicName.toString()))); return null; }); } else { @@ -2493,9 +2493,9 @@ private void internalGetLastMessageIdAsync(final Backoff backoff, return; } + log.warn("[{}] [{}] Could not get connection while getLastMessageId -- Will try again in {} ms", + topic, getHandlerName(), nextDelay); ((ScheduledExecutorService) client.getScheduledExecutorProvider().getExecutor()).schedule(() -> { - log.warn("[{}] [{}] Could not get connection while getLastMessageId -- Will try again in {} ms", - topic, getHandlerName(), nextDelay); remainingTime.addAndGet(-nextDelay); internalGetLastMessageIdAsync(backoff, remainingTime, future); }, nextDelay, TimeUnit.MILLISECONDS); From 8e51da1830f048cc49522579fbebc88b865f3591 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Fri, 1 Sep 2023 18:02:06 +0800 Subject: [PATCH 038/699] [fix][broker] Fix unsubscribe non-durable subscription error (#21099) --- .../service/persistent/PersistentTopic.java | 11 ++++---- .../broker/service/BrokerServiceTest.java | 26 +++++++++++++++++++ 2 files changed, 31 insertions(+), 6 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 38fe0639cfa1f..727c896006600 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 @@ -1165,15 +1165,14 @@ public void deleteLedgerFailed(ManagedLedgerException exception, Object ctx) { private void asyncDeleteCursorWithClearDelayedMessage(String subscriptionName, CompletableFuture unsubscribeFuture) { - if (!isDelayedDeliveryEnabled() - || !(brokerService.getDelayedDeliveryTrackerFactory() instanceof BucketDelayedDeliveryTrackerFactory)) { - asyncDeleteCursor(subscriptionName, unsubscribeFuture); - return; - } - PersistentSubscription persistentSubscription = subscriptions.get(subscriptionName); if (persistentSubscription == null) { log.warn("[{}][{}] Can't find subscription, skip clear delayed message", topic, subscriptionName); + unsubscribeFuture.complete(null); + return; + } + if (!isDelayedDeliveryEnabled() + || !(brokerService.getDelayedDeliveryTrackerFactory() instanceof BucketDelayedDeliveryTrackerFactory)) { asyncDeleteCursor(subscriptionName, unsubscribeFuture); return; } 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 0977fb70dce85..6c0bc6efec758 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 @@ -93,6 +93,7 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; 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.ClientCnx; import org.apache.pulsar.client.impl.ConnectionPool; @@ -1760,4 +1761,29 @@ public void testDuplicateAcknowledgement() throws Exception { assertEquals(admin.topics().getStats(topicName).getSubscriptions() .get("sub-1").getUnackedMessages(), 0); } + + @Test + public void testUnsubscribeNonDurableSub() throws Exception { + final String ns = "prop/ns-test"; + final String topic = ns + "/testUnsubscribeNonDurableSub"; + + admin.namespaces().createNamespace(ns, 2); + admin.topics().createPartitionedTopic(String.format("persistent://%s", topic), 1); + + pulsarClient.newProducer(Schema.STRING).topic(topic).create().close(); + @Cleanup + Consumer consumer = pulsarClient + .newConsumer(Schema.STRING) + .topic(topic) + .subscriptionMode(SubscriptionMode.NonDurable) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscriptionName("sub1") + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + try { + consumer.unsubscribe(); + } catch (Exception ex) { + fail("Unsubscribe failed"); + } + } } From f779a19883eca45a74c984975aa35fca851a6302 Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Mon, 4 Sep 2023 08:50:49 +0800 Subject: [PATCH 039/699] [fix][client] Avoid ack hole for chunk message (#21101) ## Motivation Handle ack hole case: For example: ```markdown Chunk-1 sequence ID: 0, chunk ID: 0, msgID: 1:1 Chunk-2 sequence ID: 0, chunk ID: 1, msgID: 1:2 Chunk-3 sequence ID: 0, chunk ID: 0, msgID: 1:3 Chunk-4 sequence ID: 0, chunk ID: 1, msgID: 1:4 Chunk-5 sequence ID: 0, chunk ID: 2, msgID: 1:5 ``` Consumer ack chunk message via ChunkMessageIdImpl that consists of all the chunks in this chunk message(Chunk-3, Chunk-4, Chunk-5). The Chunk-1 and Chunk-2 are not included in the ChunkMessageIdImpl, so we should process it here. ## Modification Ack chunk-1 and chunk-2. --- .../client/impl/MessageChunkingTest.java | 33 +++++++++++++ .../pulsar/client/impl/ConsumerImpl.java | 46 ++++++++++++++++--- 2 files changed, 72 insertions(+), 7 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 dffa003524864..f266afd8a2ee1 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 @@ -356,6 +356,38 @@ public void testMaxPendingChunkMessages() throws Exception { assertNull(consumer.receive(5, TimeUnit.SECONDS)); } + @Test + public void testResendChunkMessagesWithoutAckHole() throws Exception { + log.info("-- Starting {} test --", methodName); + final String topicName = "persistent://my-property/my-ns/testResendChunkMessagesWithoutAckHole"; + final String subName = "my-subscriber-name"; + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topicName) + .subscriptionName(subName) + .maxPendingChunkedMessage(10) + .autoAckOldestChunkedMessageOnQueueFull(true) + .subscribe(); + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topicName) + .chunkMaxMessageSize(100) + .enableChunking(true) + .enableBatching(false) + .create(); + + sendSingleChunk(producer, "0", 0, 2); + + sendSingleChunk(producer, "0", 0, 2); // Resending the first chunk + sendSingleChunk(producer, "0", 1, 2); + + Message receivedMsg = consumer.receive(5, TimeUnit.SECONDS); + assertEquals(receivedMsg.getValue(), "chunk-0-0|chunk-0-1|"); + consumer.acknowledge(receivedMsg); + assertEquals(admin.topics().getStats(topicName).getSubscriptions().get(subName) + .getNonContiguousDeletedMessagesRanges(), 0); + } + @Test public void testResendChunkMessages() throws Exception { log.info("-- Starting {} test --", methodName); @@ -395,6 +427,7 @@ public void testResendChunkMessages() throws Exception { receivedMsg = consumer.receive(5, TimeUnit.SECONDS); assertEquals(receivedMsg.getValue(), "chunk-1-0|chunk-1-1|chunk-1-2|"); consumer.acknowledge(receivedMsg); + Assert.assertEquals(((ConsumerImpl) consumer).getAvailablePermits(), 8); } /** 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 06a0c1dddddc3..a275554d917f0 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 @@ -1421,7 +1421,9 @@ void messageReceived(CommandMessage cmdMessage, ByteBuf headersAndPayload, Clien private ByteBuf processMessageChunk(ByteBuf compressedPayload, MessageMetadata msgMetadata, MessageIdImpl msgId, MessageIdData messageId, ClientCnx cnx) { - + if (msgMetadata.getChunkId() != (msgMetadata.getNumChunksFromMsg() - 1)) { + increaseAvailablePermits(cnx); + } // Lazy task scheduling to expire incomplete chunk message if (expireTimeOfIncompleteChunkedMessageMillis > 0 && expireChunkMessageTaskScheduled.compareAndSet(false, true)) { @@ -1437,6 +1439,37 @@ private ByteBuf processMessageChunk(ByteBuf compressedPayload, MessageMetadata m if (msgMetadata.getChunkId() == 0) { if (chunkedMsgCtx != null) { + // Handle ack hole case when receive duplicated chunks. + // There are two situation that receives chunks with the same sequence ID and chunk ID. + // Situation 1 - Message redeliver: + // For example: + // Chunk-1 sequence ID: 0, chunk ID: 0, msgID: 1:1 + // Chunk-2 sequence ID: 0, chunk ID: 1, msgID: 1:2 + // Chunk-3 sequence ID: 0, chunk ID: 0, msgID: 1:1 + // Chunk-4 sequence ID: 0, chunk ID: 1, msgID: 1:2 + // Chunk-5 sequence ID: 0, chunk ID: 2, msgID: 1:3 + // In this case, chunk-3 and chunk-4 have the same msgID with chunk-1 and chunk-2. + // This may be caused by message redeliver, we can't ack any chunk in this case here. + // Situation 2 - Corrupted chunk message + // For example: + // Chunk-1 sequence ID: 0, chunk ID: 0, msgID: 1:1 + // Chunk-2 sequence ID: 0, chunk ID: 1, msgID: 1:2 + // Chunk-3 sequence ID: 0, chunk ID: 0, msgID: 1:3 + // Chunk-4 sequence ID: 0, chunk ID: 1, msgID: 1:4 + // Chunk-5 sequence ID: 0, chunk ID: 2, msgID: 1:5 + // In this case, all the chunks with different msgIDs and are persistent in the topic. + // But Chunk-1 and Chunk-2 belong to a corrupted chunk message that must be skipped since + // they will not be delivered to end users. So we should ack them here to avoid ack hole. + boolean isCorruptedChunkMessageDetected = Arrays.stream(chunkedMsgCtx.chunkedMessageIds) + .noneMatch(messageId1 -> messageId1 != null && messageId1.ledgerId == messageId.getLedgerId() + && messageId1.entryId == messageId.getEntryId()); + if (isCorruptedChunkMessageDetected) { + Arrays.stream(chunkedMsgCtx.chunkedMessageIds).forEach(messageId1 -> { + if (messageId1 != null) { + doAcknowledge(messageId1, AckType.Individual, Collections.emptyMap(), null); + } + }); + } // The first chunk of a new chunked-message received before receiving other chunks of previous // chunked-message // so, remove previous chunked-message from map and release buffer @@ -1476,11 +1509,12 @@ private ByteBuf processMessageChunk(ByteBuf compressedPayload, MessageMetadata m msgMetadata.getProducerName(), msgId, chunkedMsgCtx.lastChunkedMessageId, msgMetadata.getChunkId(), msgMetadata.getSequenceId()); compressedPayload.release(); - increaseAvailablePermits(cnx); - boolean repeatedlyReceived = Arrays.stream(chunkedMsgCtx.chunkedMessageIds) - .anyMatch(messageId1 -> messageId1 != null && messageId1.ledgerId == messageId.getLedgerId() + // Just like the above logic of receiving the first chunk again. We only ack this chunk in the message + // duplication case. + boolean isDuplicatedChunk = Arrays.stream(chunkedMsgCtx.chunkedMessageIds) + .noneMatch(messageId1 -> messageId1 != null && messageId1.ledgerId == messageId.getLedgerId() && messageId1.entryId == messageId.getEntryId()); - if (!repeatedlyReceived) { + if (isDuplicatedChunk) { doAcknowledge(msgId, AckType.Individual, Collections.emptyMap(), null); } return null; @@ -1497,7 +1531,6 @@ private ByteBuf processMessageChunk(ByteBuf compressedPayload, MessageMetadata m } chunkedMessagesMap.remove(msgMetadata.getUuid()); compressedPayload.release(); - increaseAvailablePermits(cnx); if (expireTimeOfIncompleteChunkedMessageMillis > 0 && System.currentTimeMillis() > (msgMetadata.getPublishTime() + expireTimeOfIncompleteChunkedMessageMillis)) { @@ -1516,7 +1549,6 @@ private ByteBuf processMessageChunk(ByteBuf compressedPayload, MessageMetadata m // if final chunk is not received yet then release payload and return if (msgMetadata.getChunkId() != (msgMetadata.getNumChunksFromMsg() - 1)) { compressedPayload.release(); - increaseAvailablePermits(cnx); return null; } From db6c7e1b5595dc3c1ca1bde197ba0e8776dd8bab Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Mon, 21 Aug 2023 09:42:05 +0800 Subject: [PATCH 040/699] [improve][meta] Improve fault tolerance of blocking calls by supporting timeout (#21028) --- .../bookkeeper/AbstractMetadataDriver.java | 2 + ...LegacyHierarchicalLedgerRangeIterator.java | 18 ++- .../LongHierarchicalLedgerRangeIterator.java | 7 +- .../bookkeeper/PulsarLayoutManager.java | 18 ++- .../PulsarLedgerManagerFactory.java | 38 +++++- .../PulsarLedgerUnderreplicationManager.java | 122 +++++++++++------- .../bookkeeper/PulsarRegistrationManager.java | 112 ++++++++++------ 7 files changed, 209 insertions(+), 108 deletions(-) diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/AbstractMetadataDriver.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/AbstractMetadataDriver.java index cc5f759c73fe5..435f94b05dc2b 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/AbstractMetadataDriver.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/AbstractMetadataDriver.java @@ -21,6 +21,7 @@ import java.io.Closeable; import java.io.IOException; import java.net.URI; +import java.util.concurrent.TimeUnit; import lombok.SneakyThrows; import org.apache.bookkeeper.conf.AbstractConfiguration; import org.apache.bookkeeper.discover.RegistrationClient; @@ -40,6 +41,7 @@ public abstract class AbstractMetadataDriver implements Closeable { public static final String METADATA_STORE_SCHEME = "metadata-store"; public static final String METADATA_STORE_INSTANCE = "metadata-store-instance"; + public static final long BLOCKING_CALL_TIMEOUT = TimeUnit.SECONDS.toMillis(30); protected MetadataStoreExtended store; private boolean storeInstanceIsOwned; diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/LegacyHierarchicalLedgerRangeIterator.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/LegacyHierarchicalLedgerRangeIterator.java index 15b1d561f901c..37e6dc836f254 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/LegacyHierarchicalLedgerRangeIterator.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/LegacyHierarchicalLedgerRangeIterator.java @@ -18,17 +18,21 @@ */ package org.apache.pulsar.metadata.bookkeeper; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.pulsar.metadata.bookkeeper.AbstractMetadataDriver.BLOCKING_CALL_TIMEOUT; import java.io.IOException; import java.util.Iterator; import java.util.List; import java.util.NavigableSet; import java.util.NoSuchElementException; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.util.StringUtils; import org.apache.pulsar.metadata.api.MetadataStore; + /** * Hierarchical Ledger Manager which manages ledger meta in zookeeper using 2-level hierarchical znodes. * @@ -67,7 +71,7 @@ public LegacyHierarchicalLedgerRangeIterator(MetadataStore store, String ledgers * @return false if have visited all level1 nodes * @throws InterruptedException/KeeperException if error occurs reading zookeeper children */ - private boolean nextL1Node() throws ExecutionException, InterruptedException { + private boolean nextL1Node() throws ExecutionException, InterruptedException, TimeoutException { l2NodesIter = null; while (l2NodesIter == null) { if (l1NodesIter.hasNext()) { @@ -79,7 +83,8 @@ private boolean nextL1Node() throws ExecutionException, InterruptedException { if (!isLedgerParentNode(curL1Nodes)) { continue; } - List l2Nodes = store.getChildren(ledgersRoot + "/" + curL1Nodes).get(); + List l2Nodes = store.getChildren(ledgersRoot + "/" + curL1Nodes) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); l2NodesIter = l2Nodes.iterator(); if (!l2NodesIter.hasNext()) { l2NodesIter = null; @@ -94,7 +99,8 @@ private synchronized void preload() throws IOException { boolean hasMoreElements = false; try { if (l1NodesIter == null) { - List l1Nodes = store.getChildren(ledgersRoot).get(); + List l1Nodes = store.getChildren(ledgersRoot) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); l1NodesIter = l1Nodes.iterator(); hasMoreElements = nextL1Node(); } else if (l2NodesIter == null || !l2NodesIter.hasNext()) { @@ -102,7 +108,7 @@ private synchronized void preload() throws IOException { } else { hasMoreElements = true; } - } catch (ExecutionException ke) { + } catch (ExecutionException | TimeoutException ke) { throw new IOException("Error preloading next range", ke); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); @@ -156,8 +162,8 @@ LedgerManager.LedgerRange getLedgerRangeByLevel(final String level1, final Strin String nodePath = nodeBuilder.toString(); List ledgerNodes = null; try { - ledgerNodes = store.getChildren(nodePath).get(); - } catch (ExecutionException e) { + ledgerNodes = store.getChildren(nodePath).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); + } catch (ExecutionException | TimeoutException e) { throw new IOException("Error when get child nodes from zk", e); } catch (InterruptedException e) { Thread.currentThread().interrupt(); diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/LongHierarchicalLedgerRangeIterator.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/LongHierarchicalLedgerRangeIterator.java index 9a36ac53b8991..3b32916e6e7a9 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/LongHierarchicalLedgerRangeIterator.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/LongHierarchicalLedgerRangeIterator.java @@ -24,6 +24,8 @@ import java.util.NoSuchElementException; import java.util.Set; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.util.StringUtils; @@ -57,8 +59,9 @@ class LongHierarchicalLedgerRangeIterator implements LedgerManager.LedgerRangeIt */ List getChildrenAt(String path) throws IOException { try { - return store.getChildren(path).get(); - } catch (ExecutionException e) { + return store.getChildren(path) + .get(AbstractMetadataDriver.BLOCKING_CALL_TIMEOUT, TimeUnit.MILLISECONDS); + } catch (ExecutionException | TimeoutException e) { if (log.isDebugEnabled()) { log.debug("Failed to get children at {}", path); } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLayoutManager.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLayoutManager.java index a4336b876398a..4444295b6169e 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLayoutManager.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLayoutManager.java @@ -18,9 +18,12 @@ */ package org.apache.pulsar.metadata.bookkeeper; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.pulsar.metadata.bookkeeper.AbstractMetadataDriver.BLOCKING_CALL_TIMEOUT; import java.io.IOException; import java.util.Optional; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; import lombok.AccessLevel; import lombok.Getter; import org.apache.bookkeeper.bookie.BookieException; @@ -30,6 +33,7 @@ import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; + class PulsarLayoutManager implements LayoutManager { @Getter(AccessLevel.PACKAGE) @@ -49,14 +53,14 @@ class PulsarLayoutManager implements LayoutManager { @Override public LedgerLayout readLedgerLayout() throws IOException { try { - byte[] layoutData = store.get(layoutPath).get() + byte[] layoutData = store.get(layoutPath).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS) .orElseThrow(() -> new BookieException.MetadataStoreException("Layout node not found")) .getValue(); return LedgerLayout.parseLayout(layoutData); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new IOException(e); - } catch (BookieException | ExecutionException e) { + } catch (BookieException | ExecutionException | TimeoutException e) { throw new IOException(e); } } @@ -66,10 +70,13 @@ public void storeLedgerLayout(LedgerLayout ledgerLayout) throws IOException { try { byte[] layoutData = ledgerLayout.serialize(); - store.put(layoutPath, layoutData, Optional.of(-1L)).get(); + store.put(layoutPath, layoutData, Optional.of(-1L)) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new IOException(e); + } catch (TimeoutException e) { + throw new IOException(e); } catch (ExecutionException e) { if (e.getCause() instanceof MetadataStoreException.BadVersionException) { throw new LedgerLayoutExistsException(e); @@ -82,11 +89,12 @@ public void storeLedgerLayout(LedgerLayout ledgerLayout) throws IOException { @Override public void deleteLedgerLayout() throws IOException { try { - store.delete(layoutPath, Optional.empty()).get(); + store.delete(layoutPath, Optional.empty()) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new IOException(e); - } catch (ExecutionException e) { + } catch (ExecutionException | TimeoutException e) { throw new IOException(e); } } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerManagerFactory.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerManagerFactory.java index 1b229757c9c30..bfcbf0b22d924 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerManagerFactory.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerManagerFactory.java @@ -19,8 +19,12 @@ package org.apache.pulsar.metadata.bookkeeper; import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.pulsar.metadata.bookkeeper.AbstractMetadataDriver.BLOCKING_CALL_TIMEOUT; import java.io.IOException; import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.conf.AbstractConfiguration; @@ -110,7 +114,13 @@ public boolean validateAndNukeExistingCluster(AbstractConfiguration conf, * before proceeding with nuking existing cluster, make sure there * are no unexpected nodes under ledgersRootPath */ - List ledgersRootPathChildrenList = store.getChildren(ledgerRootPath).join(); + final List ledgersRootPathChildrenList; + try { + ledgersRootPathChildrenList = store.getChildren(ledgerRootPath) + .get(BLOCKING_CALL_TIMEOUT, TimeUnit.MILLISECONDS); + } catch (ExecutionException | TimeoutException e) { + throw new IOException(e); + } for (String ledgersRootPathChildren : ledgersRootPathChildrenList) { if ((!AbstractZkLedgerManager.isSpecialZnode(ledgersRootPathChildren)) && (!ledgerManager.isLedgerParentNode(ledgersRootPathChildren))) { @@ -124,18 +134,34 @@ public boolean validateAndNukeExistingCluster(AbstractConfiguration conf, format(conf, layoutManager); // now delete all the special nodes recursively - for (String ledgersRootPathChildren : store.getChildren(ledgerRootPath).join()) { - if (AbstractZkLedgerManager.isSpecialZnode(ledgersRootPathChildren)) { - store.deleteRecursive(ledgerRootPath + "/" + ledgersRootPathChildren).join(); + final List ledgersRootPathChildren; + try { + ledgersRootPathChildren = store.getChildren(ledgerRootPath) + .get(BLOCKING_CALL_TIMEOUT, TimeUnit.MILLISECONDS); + } catch (ExecutionException | TimeoutException e) { + throw new IOException(e); + } + for (String ledgersRootPathChild :ledgersRootPathChildren) { + if (AbstractZkLedgerManager.isSpecialZnode(ledgersRootPathChild)) { + try { + store.deleteRecursive(ledgerRootPath + "/" + ledgersRootPathChild) + .get(BLOCKING_CALL_TIMEOUT, TimeUnit.MILLISECONDS); + } catch (ExecutionException | TimeoutException e) { + throw new IOException(e); + } } else { log.error("Found unexpected node : {} under ledgersRootPath : {} so exiting nuke operation", - ledgersRootPathChildren, ledgerRootPath); + ledgersRootPathChild, ledgerRootPath); return false; } } // finally deleting the ledgers rootpath - store.deleteRecursive(ledgerRootPath).join(); + try { + store.deleteRecursive(ledgerRootPath).get(BLOCKING_CALL_TIMEOUT, TimeUnit.MILLISECONDS); + } catch (ExecutionException | TimeoutException e) { + throw new IOException(e); + } log.info("Successfully nuked existing cluster"); return true; diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerUnderreplicationManager.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerUnderreplicationManager.java index 79fdc44cb2b06..1124090a98d8b 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerUnderreplicationManager.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerUnderreplicationManager.java @@ -19,12 +19,14 @@ package org.apache.pulsar.metadata.bookkeeper; import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.bookkeeper.proto.DataFormats.CheckAllLedgersFormat; import static org.apache.bookkeeper.proto.DataFormats.LedgerRereplicationLayoutFormat; import static org.apache.bookkeeper.proto.DataFormats.LockDataFormat; import static org.apache.bookkeeper.proto.DataFormats.PlacementPolicyCheckFormat; import static org.apache.bookkeeper.proto.DataFormats.ReplicasCheckFormat; import static org.apache.bookkeeper.proto.DataFormats.UnderreplicatedLedgerFormat; +import static org.apache.pulsar.metadata.bookkeeper.AbstractMetadataDriver.BLOCKING_CALL_TIMEOUT; import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.TextFormat; import java.net.UnknownHostException; @@ -41,6 +43,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; import java.util.function.Predicate; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -249,7 +252,7 @@ public UnderreplicatedLedger getLedgerUnreplicationInfo(long ledgerId) try { String path = getUrLedgerPath(ledgerId); - Optional optRes = store.get(path).get(); + Optional optRes = store.get(path).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); if (!optRes.isPresent()) { if (log.isDebugEnabled()) { log.debug("Ledger: {} is not marked underreplicated", ledgerId); @@ -270,7 +273,7 @@ public UnderreplicatedLedger getLedgerUnreplicationInfo(long ledgerId) underreplicatedLedger.setCtime(ctime); underreplicatedLedger.setReplicaList(replicaList); return underreplicatedLedger; - } catch (ExecutionException ee) { + } catch (ExecutionException | TimeoutException ee) { throw new ReplicationException.UnavailableException("Error contacting with metadata store", ee); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); @@ -374,14 +377,16 @@ private void handleLedgerUnderreplicatedAlreadyMarked(final String path, public void acquireUnderreplicatedLedger(long ledgerId) throws ReplicationException { try { internalAcquireUnderreplicatedLedger(ledgerId); - } catch (ExecutionException | InterruptedException e) { + } catch (ExecutionException | TimeoutException | InterruptedException e) { throw new ReplicationException.UnavailableException("Failed to acuire under-replicated ledger", e); } } - private void internalAcquireUnderreplicatedLedger(long ledgerId) throws ExecutionException, InterruptedException { + private void internalAcquireUnderreplicatedLedger(long ledgerId) throws ExecutionException, + InterruptedException, TimeoutException { String lockPath = getUrLedgerLockPath(urLockPath, ledgerId); - store.put(lockPath, LOCK_DATA, Optional.of(-1L), EnumSet.of(CreateOption.Ephemeral)).get(); + store.put(lockPath, LOCK_DATA, Optional.of(-1L), EnumSet.of(CreateOption.Ephemeral)) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); } @Override @@ -392,7 +397,8 @@ public void markLedgerReplicated(long ledgerId) throws ReplicationException.Unav try { Lock l = heldLocks.get(ledgerId); if (l != null) { - store.delete(getUrLedgerPath(ledgerId), Optional.of(l.getLedgerNodeVersion())).get(); + store.delete(getUrLedgerPath(ledgerId), Optional.of(l.getLedgerNodeVersion())) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); } } catch (ExecutionException ee) { if (ee.getCause() instanceof MetadataStoreException.NotFoundException) { @@ -405,6 +411,8 @@ public void markLedgerReplicated(long ledgerId) throws ReplicationException.Unav log.error("Error deleting underreplicated ledger node", ee); throw new ReplicationException.UnavailableException("Error contacting metadata store", ee); } + } catch (TimeoutException ex) { + throw new ReplicationException.UnavailableException("Error contacting metadata store", ex); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); throw new ReplicationException.UnavailableException("Interrupted while contacting metadata store", ie); @@ -445,7 +453,7 @@ public boolean hasNext() { while (queue.size() > 0 && curBatch.size() == 0) { String parent = queue.remove(); try { - for (String c : store.getChildren(parent).get()) { + for (String c : store.getChildren(parent).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS)) { String child = parent + "/" + c; if (c.startsWith("urL")) { long ledgerId = getLedgerId(child); @@ -479,21 +487,23 @@ public UnderreplicatedLedger next() { } private long getLedgerToRereplicateFromHierarchy(String parent, long depth) - throws ExecutionException, InterruptedException { + throws ExecutionException, InterruptedException, TimeoutException { if (depth == 4) { - List children = new ArrayList<>(store.getChildren(parent).get()); + List children = new ArrayList<>(store.getChildren(parent) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS)); Collections.shuffle(children); while (!children.isEmpty()) { String tryChild = children.get(0); try { - List locks = store.getChildren(urLockPath).get(); + List locks = store.getChildren(urLockPath).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); if (locks.contains(tryChild)) { children.remove(tryChild); continue; } - Optional optRes = store.get(parent + "/" + tryChild).get(); + Optional optRes = store.get(parent + "/" + tryChild) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); if (!optRes.isPresent()) { if (log.isDebugEnabled()) { log.debug("{}/{} doesn't exist", parent, tryChild); @@ -522,7 +532,7 @@ private long getLedgerToRereplicateFromHierarchy(String parent, long depth) return -1; } - List children = new ArrayList<>(store.getChildren(parent).join()); + List children = new ArrayList<>(store.getChildren(parent).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS)); Collections.shuffle(children); while (children.size() > 0) { @@ -545,7 +555,7 @@ public long pollLedgerToRereplicate() throws ReplicationException.UnavailableExc } try { return getLedgerToRereplicateFromHierarchy(urLedgerPath, 0); - } catch (ExecutionException ee) { + } catch (ExecutionException | TimeoutException ee) { throw new ReplicationException.UnavailableException("Error contacting metadata store", ee); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); @@ -571,7 +581,7 @@ public long getLedgerToRereplicate() throws ReplicationException.UnavailableExce // nothing found, wait for a watcher to trigger this.wait(1000); } - } catch (ExecutionException ee) { + } catch (ExecutionException | TimeoutException ee) { throw new ReplicationException.UnavailableException("Error contacting metadata store", ee); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); @@ -597,7 +607,8 @@ public void releaseUnderreplicatedLedger(long ledgerId) throws ReplicationExcept try { Lock l = heldLocks.get(ledgerId); if (l != null) { - store.delete(l.getLockPath(), Optional.empty()).get(); + store.delete(l.getLockPath(), Optional.empty()) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); } } catch (ExecutionException ee) { if (ee.getCause() instanceof MetadataStoreException.NotFoundException) { @@ -606,6 +617,8 @@ public void releaseUnderreplicatedLedger(long ledgerId) throws ReplicationExcept log.error("Error deleting underreplicated ledger lock", ee); throw new ReplicationException.UnavailableException("Error contacting metadata store", ee); } + } catch (TimeoutException ex) { + throw new ReplicationException.UnavailableException("Error contacting metadata store", ex); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); throw new ReplicationException.UnavailableException("Interrupted while connecting metadata store", ie); @@ -620,7 +633,8 @@ public void close() throws ReplicationException.UnavailableException { } try { for (Map.Entry e : heldLocks.entrySet()) { - store.delete(e.getValue().getLockPath(), Optional.empty()).get(); + store.delete(e.getValue().getLockPath(), Optional.empty()) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); } } catch (ExecutionException ee) { if (ee.getCause() instanceof MetadataStoreException.NotFoundException) { @@ -629,6 +643,8 @@ public void close() throws ReplicationException.UnavailableException { log.error("Error deleting underreplicated ledger lock", ee); throw new ReplicationException.UnavailableException("Error contacting metadata store", ee); } + } catch (TimeoutException ex) { + throw new ReplicationException.UnavailableException("Error contacting metadata store", ex); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); throw new ReplicationException.UnavailableException("Interrupted while connecting metadata store", ie); @@ -643,9 +659,10 @@ public void disableLedgerReplication() } try { String path = basePath + '/' + BookKeeperConstants.DISABLE_NODE; - store.put(path, "".getBytes(UTF_8), Optional.of(-1L)).get(); + store.put(path, "".getBytes(UTF_8), Optional.of(-1L)) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); log.info("Auto ledger re-replication is disabled!"); - } catch (ExecutionException ee) { + } catch (ExecutionException | TimeoutException ee) { log.error("Exception while stopping auto ledger re-replication", ee); throw new ReplicationException.UnavailableException( "Exception while stopping auto ledger re-replication", ee); @@ -663,9 +680,10 @@ public void enableLedgerReplication() log.debug("enableLedegerReplication()"); } try { - store.delete(basePath + '/' + BookKeeperConstants.DISABLE_NODE, Optional.empty()).get(); + store.delete(basePath + '/' + BookKeeperConstants.DISABLE_NODE, Optional.empty()) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); log.info("Resuming automatic ledger re-replication"); - } catch (ExecutionException ee) { + } catch (ExecutionException | TimeoutException ee) { log.error("Exception while resuming ledger replication", ee); throw new ReplicationException.UnavailableException( "Exception while resuming auto ledger re-replication", ee); @@ -683,8 +701,9 @@ public boolean isLedgerReplicationEnabled() log.debug("isLedgerReplicationEnabled()"); } try { - return !store.exists(basePath + '/' + BookKeeperConstants.DISABLE_NODE).get(); - } catch (ExecutionException ee) { + return !store.exists(basePath + '/' + BookKeeperConstants.DISABLE_NODE) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); + } catch (ExecutionException | TimeoutException ee) { log.error("Error while checking the state of " + "ledger re-replication", ee); throw new ReplicationException.UnavailableException( @@ -708,13 +727,14 @@ public void notifyLedgerReplicationEnabled(final BookkeeperInternalCallbacks.Gen } try { - if (!store.exists(basePath + '/' + BookKeeperConstants.DISABLE_NODE).get()) { + if (!store.exists(basePath + '/' + BookKeeperConstants.DISABLE_NODE) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS)) { log.info("LedgerReplication is enabled externally through metadata store, " + "since DISABLE_NODE node is deleted"); cb.operationComplete(0, null); return; } - } catch (ExecutionException ee) { + } catch (ExecutionException | TimeoutException ee) { log.error("Error while checking the state of " + "ledger re-replication", ee); throw new ReplicationException.UnavailableException( @@ -732,7 +752,7 @@ public void notifyLedgerReplicationEnabled(final BookkeeperInternalCallbacks.Gen @Override public boolean isLedgerBeingReplicated(long ledgerId) throws ReplicationException { try { - return store.exists(getUrLedgerLockPath(urLockPath, ledgerId)).get(); + return store.exists(getUrLedgerLockPath(urLockPath, ledgerId)).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); } catch (Exception e) { throw new ReplicationException.UnavailableException("Failed to check if ledger is beinge replicated", e); } @@ -744,7 +764,7 @@ public boolean initializeLostBookieRecoveryDelay(int lostBookieRecoveryDelay) th log.debug("initializeLostBookieRecoveryDelay()"); try { store.put(lostBookieRecoveryDelayPath, Integer.toString(lostBookieRecoveryDelay).getBytes(UTF_8), - Optional.of(-1L)).get(); + Optional.of(-1L)).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); } catch (ExecutionException ee) { if (ee.getCause() instanceof MetadataStoreException.BadVersionException) { log.info("lostBookieRecoveryDelay node is already present, so using " @@ -754,6 +774,9 @@ public boolean initializeLostBookieRecoveryDelay(int lostBookieRecoveryDelay) th log.error("Error while initializing LostBookieRecoveryDelay", ee); throw new ReplicationException.UnavailableException("Error contacting zookeeper", ee); } + } catch (TimeoutException ex) { + log.error("Error while initializing LostBookieRecoveryDelay", ex); + throw new ReplicationException.UnavailableException("Error contacting zookeeper", ex); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); throw new ReplicationException.UnavailableException("Interrupted while contacting zookeeper", ie); @@ -767,9 +790,9 @@ public void setLostBookieRecoveryDelay(int lostBookieRecoveryDelay) throws log.debug("setLostBookieRecoveryDelay()"); try { store.put(lostBookieRecoveryDelayPath, Integer.toString(lostBookieRecoveryDelay).getBytes(UTF_8), - Optional.empty()).get(); + Optional.empty()).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); - } catch (ExecutionException ee) { + } catch (ExecutionException | TimeoutException ee) { log.error("Error while setting LostBookieRecoveryDelay ", ee); throw new ReplicationException.UnavailableException("Error contacting zookeeper", ee); } catch (InterruptedException ie) { @@ -782,9 +805,10 @@ public void setLostBookieRecoveryDelay(int lostBookieRecoveryDelay) throws public int getLostBookieRecoveryDelay() throws ReplicationException.UnavailableException { log.debug("getLostBookieRecoveryDelay()"); try { - byte[] data = store.get(lostBookieRecoveryDelayPath).get().get().getValue(); + byte[] data = store.get(lostBookieRecoveryDelayPath).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS) + .get().getValue(); return Integer.parseInt(new String(data, UTF_8)); - } catch (ExecutionException ee) { + } catch (ExecutionException | TimeoutException ee) { log.error("Error while getting LostBookieRecoveryDelay ", ee); throw new ReplicationException.UnavailableException("Error contacting zookeeper", ee); } catch (InterruptedException ie) { @@ -801,12 +825,12 @@ public void notifyLostBookieRecoveryDelayChanged(BookkeeperInternalCallbacks.Gen lostBookieRecoveryDelayListener = cb; } try { - if (!store.exists(lostBookieRecoveryDelayPath).get()) { + if (!store.exists(lostBookieRecoveryDelayPath).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS)) { cb.operationComplete(0, null); return; } - } catch (ExecutionException ee) { + } catch (ExecutionException | TimeoutException ee) { log.error("Error while checking the state of lostBookieRecoveryDelay", ee); throw new ReplicationException.UnavailableException("Error contacting zookeeper", ee); } catch (InterruptedException ie) { @@ -820,7 +844,8 @@ public String getReplicationWorkerIdRereplicatingLedger(long ledgerId) throws ReplicationException.UnavailableException { try { - Optional optRes = store.get(getUrLedgerLockPath(urLockPath, ledgerId)).get(); + Optional optRes = store.get(getUrLedgerLockPath(urLockPath, ledgerId)) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); if (!optRes.isPresent()) { // this is ok. return null; @@ -831,7 +856,7 @@ public String getReplicationWorkerIdRereplicatingLedger(long ledgerId) TextFormat.merge(new String(lockData, UTF_8), lockDataBuilder); LockDataFormat lock = lockDataBuilder.build(); return lock.getBookieId(); - } catch (ExecutionException e) { + } catch (ExecutionException | TimeoutException e) { log.error("Error while getting ReplicationWorkerId rereplicating Ledger", e); throw new ReplicationException.UnavailableException( "Error while getting ReplicationWorkerId rereplicating Ledger", e); @@ -855,8 +880,9 @@ public void setCheckAllLedgersCTime(long checkAllLedgersCTime) throws Replicatio builder.setCheckAllLedgersCTime(checkAllLedgersCTime); byte[] checkAllLedgersFormatByteArray = builder.build().toByteArray(); - store.put(checkAllLedgersCtimePath, checkAllLedgersFormatByteArray, Optional.empty()).get(); - } catch (ExecutionException ee) { + store.put(checkAllLedgersCtimePath, checkAllLedgersFormatByteArray, Optional.empty()) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); + } catch (ExecutionException | TimeoutException ee) { throw new ReplicationException.UnavailableException("Error contacting zookeeper", ee); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); @@ -870,7 +896,7 @@ public long getCheckAllLedgersCTime() throws ReplicationException.UnavailableExc log.debug("setCheckAllLedgersCTime"); } try { - Optional optRes = store.get(checkAllLedgersCtimePath).get(); + Optional optRes = store.get(checkAllLedgersCtimePath).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); if (!optRes.isPresent()) { log.warn("checkAllLedgersCtimeZnode is not yet available"); return -1; @@ -879,7 +905,7 @@ public long getCheckAllLedgersCTime() throws ReplicationException.UnavailableExc CheckAllLedgersFormat checkAllLedgersFormat = CheckAllLedgersFormat.parseFrom(data); return checkAllLedgersFormat.hasCheckAllLedgersCTime() ? checkAllLedgersFormat.getCheckAllLedgersCTime() : -1; - } catch (ExecutionException ee) { + } catch (ExecutionException | TimeoutException ee) { throw new ReplicationException.UnavailableException("Error contacting zookeeper", ee); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); @@ -899,8 +925,9 @@ public void setPlacementPolicyCheckCTime(long placementPolicyCheckCTime) throws PlacementPolicyCheckFormat.Builder builder = PlacementPolicyCheckFormat.newBuilder(); builder.setPlacementPolicyCheckCTime(placementPolicyCheckCTime); byte[] placementPolicyCheckFormatByteArray = builder.build().toByteArray(); - store.put(placementPolicyCheckCtimePath, placementPolicyCheckFormatByteArray, Optional.empty()).get(); - } catch (ExecutionException ke) { + store.put(placementPolicyCheckCtimePath, placementPolicyCheckFormatByteArray, Optional.empty()) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); + } catch (ExecutionException | TimeoutException ke) { throw new ReplicationException.UnavailableException("Error contacting zookeeper", ke); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); @@ -914,7 +941,8 @@ public long getPlacementPolicyCheckCTime() throws ReplicationException.Unavailab log.debug("getPlacementPolicyCheckCTime"); } try { - Optional optRes = store.get(placementPolicyCheckCtimePath).get(); + Optional optRes = store.get(placementPolicyCheckCtimePath) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); if (!optRes.isPresent()) { log.warn("placementPolicyCheckCtimeZnode is not yet available"); return -1; @@ -923,7 +951,7 @@ public long getPlacementPolicyCheckCTime() throws ReplicationException.Unavailab PlacementPolicyCheckFormat placementPolicyCheckFormat = PlacementPolicyCheckFormat.parseFrom(data); return placementPolicyCheckFormat.hasPlacementPolicyCheckCTime() ? placementPolicyCheckFormat.getPlacementPolicyCheckCTime() : -1; - } catch (ExecutionException ee) { + } catch (ExecutionException | TimeoutException ee) { throw new ReplicationException.UnavailableException("Error contacting zookeeper", ee); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); @@ -939,11 +967,12 @@ public void setReplicasCheckCTime(long replicasCheckCTime) throws ReplicationExc ReplicasCheckFormat.Builder builder = ReplicasCheckFormat.newBuilder(); builder.setReplicasCheckCTime(replicasCheckCTime); byte[] replicasCheckFormatByteArray = builder.build().toByteArray(); - store.put(replicasCheckCtimePath, replicasCheckFormatByteArray, Optional.empty()).get(); + store.put(replicasCheckCtimePath, replicasCheckFormatByteArray, Optional.empty()) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); if (log.isDebugEnabled()) { log.debug("setReplicasCheckCTime completed successfully"); } - } catch (ExecutionException ke) { + } catch (ExecutionException | TimeoutException ke) { throw new ReplicationException.UnavailableException("Error contacting zookeeper", ke); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); @@ -954,7 +983,8 @@ public void setReplicasCheckCTime(long replicasCheckCTime) throws ReplicationExc @Override public long getReplicasCheckCTime() throws ReplicationException.UnavailableException { try { - Optional optRes = store.get(replicasCheckCtimePath).get(); + Optional optRes = store.get(replicasCheckCtimePath) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); if (!optRes.isPresent()) { log.warn("placementPolicyCheckCtimeZnode is not yet available"); return -1; @@ -965,7 +995,7 @@ public long getReplicasCheckCTime() throws ReplicationException.UnavailableExcep log.debug("getReplicasCheckCTime completed successfully"); } return replicasCheckFormat.hasReplicasCheckCTime() ? replicasCheckFormat.getReplicasCheckCTime() : -1; - } catch (ExecutionException ee) { + } catch (ExecutionException | TimeoutException ee) { throw new ReplicationException.UnavailableException("Error contacting zookeeper", ee); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarRegistrationManager.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarRegistrationManager.java index 25c3f10aa18fa..c6aba6b7d93d0 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarRegistrationManager.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarRegistrationManager.java @@ -19,10 +19,12 @@ package org.apache.pulsar.metadata.bookkeeper; import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE; import static org.apache.bookkeeper.util.BookKeeperConstants.COOKIE_NODE; import static org.apache.bookkeeper.util.BookKeeperConstants.INSTANCEID; import static org.apache.bookkeeper.util.BookKeeperConstants.READONLY; +import static org.apache.pulsar.metadata.bookkeeper.AbstractMetadataDriver.BLOCKING_CALL_TIMEOUT; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collection; @@ -32,8 +34,8 @@ import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; import lombok.Cleanup; -import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.bookie.BookieException; import org.apache.bookkeeper.conf.AbstractConfiguration; @@ -85,12 +87,11 @@ public class PulsarRegistrationManager implements RegistrationManager { } @Override - @SneakyThrows public void close() { for (ResourceLock rwBookie : bookieRegistration.values()) { try { - rwBookie.release().get(); - } catch (ExecutionException ignore) { + rwBookie.release().get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); + } catch (ExecutionException | TimeoutException ignore) { log.error("Cannot release correctly {}", rwBookie, ignore.getCause()); } catch (InterruptedException ignore) { log.error("Cannot release correctly {}", rwBookie, ignore); @@ -100,26 +101,30 @@ public void close() { for (ResourceLock roBookie : bookieRegistrationReadOnly.values()) { try { - roBookie.release().get(); - } catch (ExecutionException ignore) { + roBookie.release().get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); + } catch (ExecutionException | TimeoutException ignore) { log.error("Cannot release correctly {}", roBookie, ignore.getCause()); } catch (InterruptedException ignore) { log.error("Cannot release correctly {}", roBookie, ignore); Thread.currentThread().interrupt(); } } - coordinationService.close(); + try { + coordinationService.close(); + } catch (Exception e) { + throw new RuntimeException(e); + } } @Override public String getClusterInstanceId() throws BookieException { try { return store.get(ledgersRootPath + "/" + INSTANCEID) - .get() + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS) .map(res -> new String(res.getValue(), UTF_8)) .orElseThrow( () -> new BookieException.MetadataStoreException("BookKeeper cluster not initialized")); - } catch (ExecutionException | InterruptedException e) { + } catch (ExecutionException | InterruptedException | TimeoutException e) { throw new BookieException.MetadataStoreException("Failed to get cluster instance id", e); } } @@ -136,22 +141,24 @@ public void registerBookie(BookieId bookieId, boolean readOnly, BookieServiceInf ResourceLock rwRegistration = bookieRegistration.remove(bookieId); if (rwRegistration != null) { log.info("Bookie {} was already registered as writable, unregistering", bookieId); - rwRegistration.release().get(); + rwRegistration.release().get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); } bookieRegistrationReadOnly.put(bookieId, - lockManager.acquireLock(regPathReadOnly, bookieServiceInfo).get()); + lockManager.acquireLock(regPathReadOnly, bookieServiceInfo) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS)); } else { ResourceLock roRegistration = bookieRegistrationReadOnly.remove(bookieId); if (roRegistration != null) { log.info("Bookie {} was already registered as read-only, unregistering", bookieId); - roRegistration.release().get(); + roRegistration.release().get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); } bookieRegistration.put(bookieId, - lockManager.acquireLock(regPath, bookieServiceInfo).get()); + lockManager.acquireLock(regPath, bookieServiceInfo) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS)); } - } catch (ExecutionException ee) { + } catch (ExecutionException | TimeoutException ee) { log.error("Exception registering ephemeral node for Bookie!", ee); // Throw an IOException back up. This will cause the Bookie // constructor to error out. Alternatively, we could do a System @@ -173,18 +180,18 @@ public void unregisterBookie(BookieId bookieId, boolean readOnly) throws BookieE if (readOnly) { ResourceLock roRegistration = bookieRegistrationReadOnly.get(bookieId); if (roRegistration != null) { - roRegistration.release().get(); + roRegistration.release().get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); } } else { ResourceLock rwRegistration = bookieRegistration.get(bookieId); if (rwRegistration != null) { - rwRegistration.release().get(); + rwRegistration.release().get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); } } } catch (InterruptedException ie) { Thread.currentThread().interrupt(); throw new BookieException.MetadataStoreException(ie); - } catch (ExecutionException e) { + } catch (ExecutionException | TimeoutException e) { throw new BookieException.MetadataStoreException(e); } } @@ -195,8 +202,9 @@ public boolean isBookieRegistered(BookieId bookieId) throws BookieException { String readonlyRegPath = bookieReadonlyRegistrationPath + "/" + bookieId; try { - return (store.exists(regPath).get() || store.exists(readonlyRegPath).get()); - } catch (ExecutionException e) { + return (store.exists(regPath).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS) + || store.exists(readonlyRegPath).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS)); + } catch (ExecutionException | TimeoutException e) { log.error("Exception while checking registration ephemeral nodes for BookieId: {}", bookieId, e); throw new BookieException.MetadataStoreException(e); } catch (InterruptedException e) { @@ -222,7 +230,8 @@ public void writeCookie(BookieId bookieId, Versioned cookieData) throws version = ((LongVersion) cookieData.getVersion()).getLongVersion(); } - store.put(path, cookieData.getValue(), Optional.of(version)).get(); + store.put(path, cookieData.getValue(), Optional.of(version)) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); } catch (InterruptedException ie) { Thread.currentThread().interrupt(); throw new BookieException.MetadataStoreException("Interrupted writing cookie for bookie " + bookieId, ie); @@ -232,6 +241,8 @@ public void writeCookie(BookieId bookieId, Versioned cookieData) throws } else { throw new BookieException.MetadataStoreException("Failed to write cookie for bookie " + bookieId); } + } catch (TimeoutException ex) { + throw new BookieException.MetadataStoreException("Failed to write cookie for bookie " + bookieId, ex); } } @@ -239,7 +250,7 @@ public void writeCookie(BookieId bookieId, Versioned cookieData) throws public Versioned readCookie(BookieId bookieId) throws BookieException { String path = this.cookiePath + "/" + bookieId; try { - Optional res = store.get(path).get(); + Optional res = store.get(path).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); if (!res.isPresent()) { throw new BookieException.CookieNotFoundException(bookieId.toString()); } @@ -250,7 +261,7 @@ public Versioned readCookie(BookieId bookieId) throws BookieException { } catch (InterruptedException ie) { Thread.currentThread().interrupt(); throw new BookieException.MetadataStoreException(ie); - } catch (ExecutionException e) { + } catch (ExecutionException | TimeoutException e) { throw new BookieException.MetadataStoreException(e); } } @@ -259,7 +270,8 @@ public Versioned readCookie(BookieId bookieId) throws BookieException { public void removeCookie(BookieId bookieId, Version version) throws BookieException { String path = this.cookiePath + "/" + bookieId; try { - store.delete(path, Optional.of(((LongVersion) version).getLongVersion())).get(); + store.delete(path, Optional.of(((LongVersion) version).getLongVersion())) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new BookieException.MetadataStoreException("Interrupted deleting cookie for bookie " + bookieId, e); @@ -269,6 +281,8 @@ public void removeCookie(BookieId bookieId, Version version) throws BookieExcept } else { throw new BookieException.MetadataStoreException("Failed to delete cookie for bookie " + bookieId); } + } catch (TimeoutException ex) { + throw new BookieException.MetadataStoreException("Failed to delete cookie for bookie " + bookieId); } log.info("Removed cookie from {} for bookie {}.", cookiePath, bookieId); @@ -276,20 +290,23 @@ public void removeCookie(BookieId bookieId, Version version) throws BookieExcept @Override public boolean prepareFormat() throws Exception { - boolean ledgerRootExists = store.exists(ledgersRootPath).get(); - boolean availableNodeExists = store.exists(bookieRegistrationPath).get(); + boolean ledgerRootExists = store.exists(ledgersRootPath).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); + boolean availableNodeExists = store.exists(bookieRegistrationPath).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); // Create ledgers root node if not exists if (!ledgerRootExists) { - store.put(ledgersRootPath, new byte[0], Optional.empty()).get(); + store.put(ledgersRootPath, new byte[0], Optional.empty()) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); } // create available bookies node if not exists if (!availableNodeExists) { - store.put(bookieRegistrationPath, new byte[0], Optional.empty()).get(); + store.put(bookieRegistrationPath, new byte[0], Optional.empty()) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); } // create readonly bookies node if not exists - if (!store.exists(bookieReadonlyRegistrationPath).get()) { - store.put(bookieReadonlyRegistrationPath, new byte[0], Optional.empty()).get(); + if (!store.exists(bookieReadonlyRegistrationPath).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS)) { + store.put(bookieReadonlyRegistrationPath, new byte[0], Optional.empty()) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); } return ledgerRootExists; @@ -301,16 +318,18 @@ public boolean initNewCluster() throws Exception { log.info("Initializing metadata for new cluster, ledger root path: {}", ledgersRootPath); - if (store.exists(instanceIdPath).get()) { + if (store.exists(instanceIdPath).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS)) { log.error("Ledger root path: {} already exists", ledgersRootPath); return false; } - store.put(ledgersRootPath, new byte[0], Optional.empty()).get(); + store.put(ledgersRootPath, new byte[0], Optional.empty()) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); // create INSTANCEID String instanceId = UUID.randomUUID().toString(); - store.put(instanceIdPath, instanceId.getBytes(UTF_8), Optional.of(-1L)).join(); + store.put(instanceIdPath, instanceId.getBytes(UTF_8), Optional.of(-1L)) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); log.info("Successfully initiated cluster. ledger root path: {} instanceId: {}", ledgersRootPath, instanceId); @@ -321,23 +340,28 @@ public boolean initNewCluster() throws Exception { public boolean format() throws Exception { // Clear underreplicated ledgers store.deleteRecursive(PulsarLedgerUnderreplicationManager.getBasePath(ledgersRootPath) - + BookKeeperConstants.DEFAULT_ZK_LEDGERS_ROOT_PATH).get(); + + BookKeeperConstants.DEFAULT_ZK_LEDGERS_ROOT_PATH) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); // Clear underreplicatedledger locks - store.deleteRecursive(PulsarLedgerUnderreplicationManager.getUrLockPath(ledgersRootPath)).get(); + store.deleteRecursive(PulsarLedgerUnderreplicationManager.getUrLockPath(ledgersRootPath)) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); // Clear the cookies - store.deleteRecursive(cookiePath).get(); + store.deleteRecursive(cookiePath).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); // Clear the INSTANCEID - if (store.exists(ledgersRootPath + "/" + BookKeeperConstants.INSTANCEID).get()) { - store.delete(ledgersRootPath + "/" + BookKeeperConstants.INSTANCEID, Optional.empty()).get(); + if (store.exists(ledgersRootPath + "/" + BookKeeperConstants.INSTANCEID) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS)) { + store.delete(ledgersRootPath + "/" + BookKeeperConstants.INSTANCEID, Optional.empty()) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); } // create INSTANCEID String instanceId = UUID.randomUUID().toString(); store.put(ledgersRootPath + "/" + BookKeeperConstants.INSTANCEID, - instanceId.getBytes(StandardCharsets.UTF_8), Optional.of(-1L)).get(); + instanceId.getBytes(StandardCharsets.UTF_8), Optional.of(-1L)) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); log.info("Successfully formatted BookKeeper metadata"); return true; @@ -347,7 +371,7 @@ public boolean format() throws Exception { public boolean nukeExistingCluster() throws Exception { log.info("Nuking metadata of existing cluster, ledger root path: {}", ledgersRootPath); - if (!store.exists(ledgersRootPath + "/" + INSTANCEID).join()) { + if (!store.exists(ledgersRootPath + "/" + INSTANCEID).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS)) { log.info("There is no existing cluster with ledgersRootPath: {}, so exiting nuke operation", ledgersRootPath); return true; @@ -356,17 +380,19 @@ public boolean nukeExistingCluster() throws Exception { @Cleanup RegistrationClient registrationClient = new PulsarRegistrationClient(store, ledgersRootPath); - Collection rwBookies = registrationClient.getWritableBookies().join().getValue(); + Collection rwBookies = registrationClient.getWritableBookies() + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS).getValue(); if (rwBookies != null && !rwBookies.isEmpty()) { log.error("Bookies are still up and connected to this cluster, " - + "stop all bookies before nuking the cluster"); + + "stop all bookies before nuking the cluster"); return false; } - Collection roBookies = registrationClient.getReadOnlyBookies().join().getValue(); + Collection roBookies = registrationClient.getReadOnlyBookies() + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS).getValue(); if (roBookies != null && !roBookies.isEmpty()) { log.error("Readonly Bookies are still up and connected to this cluster, " - + "stop all bookies before nuking the cluster"); + + "stop all bookies before nuking the cluster"); return false; } From e7149404aa5b2963c50c0c1b5514b8a79a0c132c Mon Sep 17 00:00:00 2001 From: Yan Zhao Date: Thu, 31 Aug 2023 15:12:03 +0800 Subject: [PATCH 041/699] [fix][auto-recovery] Improve to the ReplicaitonWorker performance by deleting invalid underreplication nodes (#21059) --- .../PulsarLedgerUnderreplicationManager.java | 30 +++++++++ .../LedgerUnderreplicationManagerTest.java | 66 +++++++++++++++++++ 2 files changed, 96 insertions(+) diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerUnderreplicationManager.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerUnderreplicationManager.java index 1124090a98d8b..aaa247bfb32fc 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerUnderreplicationManager.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerUnderreplicationManager.java @@ -27,10 +27,12 @@ import static org.apache.bookkeeper.proto.DataFormats.ReplicasCheckFormat; import static org.apache.bookkeeper.proto.DataFormats.UnderreplicatedLedgerFormat; import static org.apache.pulsar.metadata.bookkeeper.AbstractMetadataDriver.BLOCKING_CALL_TIMEOUT; +import com.google.common.base.Joiner; import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.TextFormat; import java.net.UnknownHostException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.EnumSet; @@ -64,6 +66,8 @@ import org.apache.pulsar.metadata.api.NotificationType; import org.apache.pulsar.metadata.api.extended.CreateOption; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; +import org.apache.pulsar.metadata.impl.ZKMetadataStore; +import org.apache.zookeeper.KeeperException; @Slf4j public class PulsarLedgerUnderreplicationManager implements LedgerUnderreplicationManager { @@ -399,6 +403,32 @@ public void markLedgerReplicated(long ledgerId) throws ReplicationException.Unav if (l != null) { store.delete(getUrLedgerPath(ledgerId), Optional.of(l.getLedgerNodeVersion())) .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); + if (store instanceof ZKMetadataStore) { + try { + // clean up the hierarchy + String[] parts = getUrLedgerPath(ledgerId).split("/"); + for (int i = 1; i <= 4; i++) { + String[] p = Arrays.copyOf(parts, parts.length - i); + String path = Joiner.on("/").join(p); + Optional getResult = store.get(path).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); + if (getResult.isPresent()) { + store.delete(path, Optional.of(getResult.get().getStat().getVersion())) + .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); + } + } + } catch (ExecutionException ee) { + // This can happen when cleaning up the hierarchy. + // It's safe to ignore, it simply means another + // ledger in the same hierarchy has been marked as + // underreplicated. + if (ee.getCause() instanceof MetadataStoreException && ee.getCause().getCause() + instanceof KeeperException.NotEmptyException) { + //do nothing. + } else { + log.warn("Error deleting underrepcalited ledger parent node", ee); + } + } + } } } catch (ExecutionException ee) { if (ee.getCause() instanceof MetadataStoreException.NotFoundException) { 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 0df325b3c57a0..649dc1663c68f 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 @@ -23,12 +23,14 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; import com.google.protobuf.TextFormat; +import java.lang.reflect.Field; import java.nio.charset.Charset; import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -54,6 +56,7 @@ import org.apache.bookkeeper.util.BookKeeperConstants; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.metadata.BaseMetadataStoreTest; +import org.apache.pulsar.metadata.api.GetResult; import org.apache.pulsar.metadata.api.MetadataStoreConfig; import org.apache.pulsar.metadata.api.NotificationType; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; @@ -296,6 +299,69 @@ 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())); + + String missingReplica = "localhost:3181"; + + @Cleanup + LedgerUnderreplicationManager m1 = lmf.newLedgerUnderreplicationManager(); + + Long ledgerA = 0xfeadeefdacL; + m1.markLedgerUnderreplicated(ledgerA, missingReplica); + + Field storeField = m1.getClass().getDeclaredField("store"); + storeField.setAccessible(true); + MetadataStoreExtended metadataStore = (MetadataStoreExtended) storeField.get(m1); + + String fiveLevelPath = PulsarLedgerUnderreplicationManager.getUrLedgerPath(urLedgerPath, ledgerA); + Optional getResult = metadataStore.get(fiveLevelPath).get(1, TimeUnit.SECONDS); + assertTrue(getResult.isPresent()); + + String fourLevelPath = fiveLevelPath.substring(0, fiveLevelPath.lastIndexOf("/")); + getResult = metadataStore.get(fourLevelPath).get(1, TimeUnit.SECONDS); + assertTrue(getResult.isPresent()); + + String threeLevelPath = fourLevelPath.substring(0, fourLevelPath.lastIndexOf("/")); + getResult = metadataStore.get(threeLevelPath).get(1, TimeUnit.SECONDS); + assertTrue(getResult.isPresent()); + + String twoLevelPath = fourLevelPath.substring(0, threeLevelPath.lastIndexOf("/")); + getResult = metadataStore.get(twoLevelPath).get(1, TimeUnit.SECONDS); + assertTrue(getResult.isPresent()); + + String oneLevelPath = fourLevelPath.substring(0, twoLevelPath.lastIndexOf("/")); + getResult = metadataStore.get(oneLevelPath).get(1, TimeUnit.SECONDS); + assertTrue(getResult.isPresent()); + + getResult = metadataStore.get(urLedgerPath).get(1, TimeUnit.SECONDS); + assertTrue(getResult.isPresent()); + + long ledgerToRereplicate = m1.getLedgerToRereplicate(); + assertEquals(ledgerToRereplicate, ledgerA); + m1.markLedgerReplicated(ledgerA); + + getResult = metadataStore.get(fiveLevelPath).get(1, TimeUnit.SECONDS); + assertFalse(getResult.isPresent()); + + getResult = metadataStore.get(fourLevelPath).get(1, TimeUnit.SECONDS); + assertFalse(getResult.isPresent()); + + getResult = metadataStore.get(threeLevelPath).get(1, TimeUnit.SECONDS); + assertFalse(getResult.isPresent()); + + getResult = metadataStore.get(twoLevelPath).get(1, TimeUnit.SECONDS); + assertFalse(getResult.isPresent()); + + getResult = metadataStore.get(oneLevelPath).get(1, TimeUnit.SECONDS); + assertFalse(getResult.isPresent()); + + getResult = metadataStore.get(urLedgerPath).get(1, TimeUnit.SECONDS); + assertTrue(getResult.isPresent()); + } + /** * Test releasing of a ledger * A ledger is released when a client decides it does not want From f5e6feb9f8c1f9fc5ba68d53c0428a34fe445c34 Mon Sep 17 00:00:00 2001 From: houxiaoyu Date: Sat, 12 Aug 2023 14:18:32 +0800 Subject: [PATCH 042/699] [improve][bk] Improve getIsolationGroup by avoid creating arrayList (#20952) ### Motivation Minor improve `getIsolationGroup` by avoid creating arrayList. And it' beneficial to GC ### Modifications `new HashSet<>(Arrays.asList(...))` -> `Sets.newHashSet(...)` --- .../IsolatedBookieEnsemblePlacementPolicy.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/IsolatedBookieEnsemblePlacementPolicy.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/IsolatedBookieEnsemblePlacementPolicy.java index 2594798485a20..164677ca9c663 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/IsolatedBookieEnsemblePlacementPolicy.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/IsolatedBookieEnsemblePlacementPolicy.java @@ -19,8 +19,8 @@ package org.apache.pulsar.bookie.rackawareness; import static org.apache.pulsar.bookie.rackawareness.BookieRackAffinityMapping.METADATA_STORE_INSTANCE; +import com.google.common.collect.Sets; import io.netty.util.HashedWheelTimer; -import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.List; @@ -166,12 +166,12 @@ private static Pair, Set> getIsolationGroup( String secondaryIsolationGroupString = ConfigurationStringUtil .castToString(properties.getOrDefault(SECONDARY_ISOLATION_BOOKIE_GROUPS, "")); if (!primaryIsolationGroupString.isEmpty()) { - pair.setLeft(new HashSet<>(Arrays.asList(primaryIsolationGroupString.split(",")))); + pair.setLeft(Sets.newHashSet(primaryIsolationGroupString.split(","))); } else { pair.setLeft(Collections.emptySet()); } if (!secondaryIsolationGroupString.isEmpty()) { - pair.setRight(new HashSet<>(Arrays.asList(secondaryIsolationGroupString.split(",")))); + pair.setRight(Sets.newHashSet(secondaryIsolationGroupString.split(","))); } else { pair.setRight(Collections.emptySet()); } From 153840c8e028577b02a4f952cd02e88c7ccb7c73 Mon Sep 17 00:00:00 2001 From: Yan Zhao Date: Tue, 5 Sep 2023 08:12:49 +0800 Subject: [PATCH 043/699] [fix] [broker] Fix isolated group not work problem. (#21096) ### Modifications When upgraded the pulsar version from 2.9.2 to 2.10.3, and the isolated group feature not work anymore. Finally, we found the problem. In IsolatedBookieEnsemblePlacementPolicy, when it gets the bookie rack from the metadata store cache, uses future.isDone() to avoid sync operation. If the future is incomplete, return empty blacklists. The cache may expire due to the caffeine cache `getExpireAfterWriteMillis` config, if the cache expires, the future may be incomplete. (#21095 will correct the behavior) In 2.9.2, it uses the sync to get data from the metadata store, we should also keep the behavior. --- ...IsolatedBookieEnsemblePlacementPolicy.java | 31 +++-- ...atedBookieEnsemblePlacementPolicyTest.java | 115 ++++++++++++++++++ 2 files changed, 134 insertions(+), 12 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/IsolatedBookieEnsemblePlacementPolicy.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/IsolatedBookieEnsemblePlacementPolicy.java index 164677ca9c663..02ddea9487469 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/IsolatedBookieEnsemblePlacementPolicy.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/IsolatedBookieEnsemblePlacementPolicy.java @@ -19,6 +19,7 @@ package org.apache.pulsar.bookie.rackawareness; import static org.apache.pulsar.bookie.rackawareness.BookieRackAffinityMapping.METADATA_STORE_INSTANCE; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Sets; import io.netty.util.HashedWheelTimer; import java.util.Collections; @@ -27,7 +28,6 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.concurrent.CompletableFuture; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy; @@ -61,6 +61,7 @@ public class IsolatedBookieEnsemblePlacementPolicy extends RackawareEnsemblePlac private static final String PULSAR_SYSTEM_TOPIC_ISOLATION_GROUP = "*"; + private volatile BookiesRackConfiguration cachedRackConfiguration = null; public IsolatedBookieEnsemblePlacementPolicy() { super(); @@ -86,7 +87,12 @@ public RackawareEnsemblePlacementPolicyImpl initialize(ClientConfiguration conf, } // Only add the bookieMappingCache if we have defined an isolation group bookieMappingCache = store.getMetadataCache(BookiesRackConfiguration.class); - bookieMappingCache.get(BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH).join(); + bookieMappingCache.get(BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH).thenAccept(opt -> opt.ifPresent( + bookiesRackConfiguration -> cachedRackConfiguration = bookiesRackConfiguration)) + .exceptionally(e -> { + log.warn("Failed to load bookies rack configuration while initialize the PlacementPolicy."); + return null; + }); } if (conf.getProperty(SECONDARY_ISOLATION_BOOKIE_GROUPS) != null) { String secondaryIsolationGroupsString = ConfigurationStringUtil @@ -179,25 +185,26 @@ private static Pair, Set> getIsolationGroup( return pair; } - private Set getExcludedBookiesWithIsolationGroups(int ensembleSize, + @VisibleForTesting + Set getExcludedBookiesWithIsolationGroups(int ensembleSize, Pair, Set> isolationGroups) { Set excludedBookies = new HashSet<>(); - if (isolationGroups != null && isolationGroups.getLeft().contains(PULSAR_SYSTEM_TOPIC_ISOLATION_GROUP)) { + if (isolationGroups != null && isolationGroups.getLeft().contains(PULSAR_SYSTEM_TOPIC_ISOLATION_GROUP)) { return excludedBookies; } try { if (bookieMappingCache != null) { - CompletableFuture> future = - bookieMappingCache.get(BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH); + bookieMappingCache.get(BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH) + .thenAccept(opt -> cachedRackConfiguration = opt.orElse(null)).exceptionally(e -> { + log.warn("Failed to update the newest bookies rack config."); + return null; + }); - Optional optRes = (future.isDone() && !future.isCompletedExceptionally()) - ? future.join() : Optional.empty(); - - if (optRes.isEmpty()) { + BookiesRackConfiguration allGroupsBookieMapping = cachedRackConfiguration; + if (allGroupsBookieMapping == null) { + log.debug("The bookies rack config is not available at now."); return excludedBookies; } - - BookiesRackConfiguration allGroupsBookieMapping = optRes.get(); Set allBookies = allGroupsBookieMapping.keySet(); int totalAvailableBookiesInPrimaryGroup = 0; Set primaryIsolationGroup = Collections.emptySet(); diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/bookie/rackawareness/IsolatedBookieEnsemblePlacementPolicyTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/bookie/rackawareness/IsolatedBookieEnsemblePlacementPolicyTest.java index f535ced08f731..beb00197e4e9a 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/bookie/rackawareness/IsolatedBookieEnsemblePlacementPolicyTest.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/bookie/rackawareness/IsolatedBookieEnsemblePlacementPolicyTest.java @@ -18,11 +18,14 @@ */ package org.apache.pulsar.bookie.rackawareness; +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; import static org.testng.Assert.fail; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Sets; import io.netty.util.HashedWheelTimer; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -34,18 +37,23 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.feature.SettableFeatureProvider; import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.common.policies.data.BookieInfo; +import org.apache.pulsar.common.policies.data.BookiesRackConfiguration; import org.apache.pulsar.common.policies.data.EnsemblePlacementPolicyConfig; import org.apache.pulsar.common.util.ObjectMapperFactory; 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.api.extended.MetadataStoreExtended; +import org.apache.pulsar.metadata.cache.impl.MetadataCacheImpl; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -114,6 +122,113 @@ public void testNonRegionBookie() throws Exception { assertFalse(ensemble.contains(new BookieSocketAddress(BOOKIE4).toBookieId())); } + @Test + public void testMetadataStoreCases() throws Exception { + Map mainBookieGroup = new HashMap<>(); + mainBookieGroup.put(BOOKIE1, BookieInfo.builder().rack("rack0").build()); + mainBookieGroup.put(BOOKIE2, BookieInfo.builder().rack("rack1").build()); + mainBookieGroup.put(BOOKIE3, BookieInfo.builder().rack("rack1").build()); + mainBookieGroup.put(BOOKIE4, BookieInfo.builder().rack("rack0").build()); + + Map secondaryBookieGroup = new HashMap<>(); + + store = mock(MetadataStoreExtended.class); + MetadataCacheImpl cache = mock(MetadataCacheImpl.class); + when(store.getMetadataCache(BookiesRackConfiguration.class)).thenReturn(cache); + CompletableFuture> initialFuture = new CompletableFuture<>(); + //The initialFuture only has group1. + BookiesRackConfiguration rackConfiguration1 = new BookiesRackConfiguration(); + rackConfiguration1.put("group1", mainBookieGroup); + rackConfiguration1.put("group2", secondaryBookieGroup); + initialFuture.complete(Optional.of(rackConfiguration1)); + + long waitTime = 2000; + CompletableFuture> waitingCompleteFuture = new CompletableFuture<>(); + new Thread(() -> { + try { + Thread.sleep(waitTime); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + //The waitingCompleteFuture has group1 and group2. + BookiesRackConfiguration rackConfiguration2 = new BookiesRackConfiguration(); + Map mainBookieGroup2 = new HashMap<>(); + mainBookieGroup2.put(BOOKIE1, BookieInfo.builder().rack("rack0").build()); + mainBookieGroup2.put(BOOKIE2, BookieInfo.builder().rack("rack1").build()); + mainBookieGroup2.put(BOOKIE4, BookieInfo.builder().rack("rack0").build()); + + Map secondaryBookieGroup2 = new HashMap<>(); + secondaryBookieGroup2.put(BOOKIE3, BookieInfo.builder().rack("rack0").build()); + rackConfiguration2.put("group1", mainBookieGroup2); + rackConfiguration2.put("group2", secondaryBookieGroup2); + waitingCompleteFuture.complete(Optional.of(rackConfiguration2)); + }).start(); + + long longWaitTime = 4000; + CompletableFuture> emptyFuture = new CompletableFuture<>(); + new Thread(() -> { + try { + Thread.sleep(longWaitTime); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + //The emptyFuture means that the zk node /bookies already be removed. + emptyFuture.complete(Optional.empty()); + }).start(); + + //Return different future means that cache expire. + when(cache.get(BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH)) + .thenReturn(initialFuture).thenReturn(initialFuture) + .thenReturn(waitingCompleteFuture).thenReturn(waitingCompleteFuture) + .thenReturn(emptyFuture).thenReturn(emptyFuture); + + IsolatedBookieEnsemblePlacementPolicy isolationPolicy = new IsolatedBookieEnsemblePlacementPolicy(); + ClientConfiguration bkClientConf = new ClientConfiguration(); + bkClientConf.setProperty(BookieRackAffinityMapping.METADATA_STORE_INSTANCE, store); + bkClientConf.setProperty(IsolatedBookieEnsemblePlacementPolicy.ISOLATION_BOOKIE_GROUPS, isolationGroups); + isolationPolicy.initialize(bkClientConf, Optional.empty(), timer, SettableFeatureProvider.DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); + isolationPolicy.onClusterChanged(writableBookies, readOnlyBookies); + + MutablePair, Set> groups = new MutablePair<>(); + groups.setLeft(Sets.newHashSet("group1")); + groups.setRight(new HashSet<>()); + + //initialFuture, the future is waiting done. + Set blacklist = + isolationPolicy.getExcludedBookiesWithIsolationGroups(2, groups); + assertTrue(blacklist.isEmpty()); + + //waitingCompleteFuture, the future is waiting done. + blacklist = + isolationPolicy.getExcludedBookiesWithIsolationGroups(2, groups); + assertTrue(blacklist.isEmpty()); + + Thread.sleep(waitTime); + + //waitingCompleteFuture, the future is already done. + blacklist = + isolationPolicy.getExcludedBookiesWithIsolationGroups(2, groups); + assertFalse(blacklist.isEmpty()); + assertEquals(blacklist.size(), 1); + BookieId excludeBookie = blacklist.iterator().next(); + assertEquals(excludeBookie.toString(), BOOKIE3); + + //emptyFuture, the future is waiting done. + blacklist = + isolationPolicy.getExcludedBookiesWithIsolationGroups(2, groups); + assertFalse(blacklist.isEmpty()); + assertEquals(blacklist.size(), 1); + excludeBookie = blacklist.iterator().next(); + assertEquals(excludeBookie.toString(), BOOKIE3); + + Thread.sleep(longWaitTime - waitTime); + + //emptyFuture, the future is already done. + blacklist = + isolationPolicy.getExcludedBookiesWithIsolationGroups(2, groups); + assertTrue(blacklist.isEmpty()); + } + @Test public void testBasic() throws Exception { Map> bookieMapping = new HashMap<>(); From ebb1c4a68f973edd4e267313b43c432f2434134c Mon Sep 17 00:00:00 2001 From: jiangpengcheng Date: Tue, 5 Sep 2023 10:35:49 +0800 Subject: [PATCH 044/699] [fix][fn] Fix the --batch-builder not working error for functions (#21023) --- .../pulsar/functions/utils/FunctionConfigUtils.java | 6 ++++++ .../functions/utils/FunctionConfigUtilsTest.java | 12 ++++++++++++ 2 files changed, 18 insertions(+) diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java index 769621da402c4..e4609672a3d0d 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java @@ -280,6 +280,12 @@ public static FunctionDetails convert(FunctionConfig functionConfig, ExtractedFu } sinkSpecBuilder.setProducerSpec(pbldr.build()); } + if (functionConfig.getBatchBuilder() != null) { + Function.ProducerSpec.Builder builder = sinkSpecBuilder.getProducerSpec() != null + ? sinkSpecBuilder.getProducerSpec().toBuilder() + : Function.ProducerSpec.newBuilder(); + sinkSpecBuilder.setProducerSpec(builder.setBatchBuilder(functionConfig.getBatchBuilder()).build()); + } functionDetailsBuilder.setSink(sinkSpecBuilder); if (functionConfig.getTenant() != null) { diff --git a/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java b/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java index ef4e72dc8d022..8f46199e8ffd5 100644 --- a/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java +++ b/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java @@ -158,6 +158,18 @@ public void testConvertWindow() { ); } + @Test + public void testConvertBatchBuilder() { + FunctionConfig functionConfig = createFunctionConfig(); + functionConfig.setBatchBuilder("KEY_BASED"); + + Function.FunctionDetails functionDetails = FunctionConfigUtils.convert(functionConfig, (ClassLoader) null); + assertEquals(functionDetails.getSink().getProducerSpec().getBatchBuilder(), "KEY_BASED"); + + FunctionConfig convertedConfig = FunctionConfigUtils.convertFromDetails(functionDetails); + assertEquals(convertedConfig.getProducerConfig().getBatchBuilder(), "KEY_BASED"); + } + @Test public void testMergeEqual() { FunctionConfig functionConfig = createFunctionConfig(); From 402e13199438980785c81d7e4ebb73a4f197e03c Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 5 Sep 2023 10:28:17 +0800 Subject: [PATCH 045/699] [fix][client] Fix repeat consume when using n-ack and batched messages (#21116) --- .../BatchMessageWithBatchIndexLevelTest.java | 85 +++++++++++++++++++ ...sistentAcknowledgmentsGroupingTracker.java | 13 ++- 2 files changed, 97 insertions(+), 1 deletion(-) 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 433f5e56d952d..d04647e21c1be 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 @@ -27,19 +27,26 @@ import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentTopic; 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.MessageIdAdv; import org.apache.pulsar.client.api.Producer; 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.ConsumerImpl; import org.apache.pulsar.common.util.FutureUtil; import org.awaitility.Awaitility; +import org.testng.Assert; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; +@Slf4j @Test(groups = "broker") public class BatchMessageWithBatchIndexLevelTest extends BatchMessageTest { @@ -280,4 +287,82 @@ public void testAckMessageWithNotOwnerConsumerUnAckMessageCount() throws Excepti Awaitility.await().until(() -> getPulsar().getBrokerService().getTopic(topicName, false) .get().get().getSubscription(subName).getConsumers().get(0).getUnackedMessages() == 0); } + + @Test + public void testNegativeAckAndLongAckDelayWillNotLeadRepeatConsume() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://prop/ns-abc/tp_"); + final String subscriptionName = "s1"; + final int redeliveryDelaySeconds = 2; + + // Create producer and consumer. + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topicName) + .enableBatching(true) + .batchingMaxMessages(1000) + .batchingMaxPublishDelay(1, TimeUnit.HOURS) + .create(); + ConsumerImpl consumer = (ConsumerImpl) pulsarClient.newConsumer(Schema.STRING) + .topic(topicName) + .subscriptionName(subscriptionName) + .subscriptionType(SubscriptionType.Shared) + .negativeAckRedeliveryDelay(redeliveryDelaySeconds, TimeUnit.SECONDS) + .enableBatchIndexAcknowledgment(true) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .acknowledgmentGroupTime(1, TimeUnit.HOURS) + .subscribe(); + + // Send 10 messages in batch. + ArrayList messagesSent = new ArrayList<>(); + List> sendTasks = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + String msg = Integer.valueOf(i).toString(); + sendTasks.add(producer.sendAsync(Integer.valueOf(i).toString())); + messagesSent.add(msg); + } + producer.flush(); + FutureUtil.waitForAll(sendTasks).join(); + + // Receive messages. + ArrayList messagesReceived = new ArrayList<>(); + // NegativeAck "batchMessageIdIndex1" once. + boolean index1HasBeenNegativeAcked = false; + while (true) { + Message message = consumer.receive(2, TimeUnit.SECONDS); + if (message == null) { + break; + } + if (index1HasBeenNegativeAcked) { + messagesReceived.add(message.getValue()); + consumer.acknowledge(message); + continue; + } + if (((MessageIdAdv) message.getMessageId()).getBatchIndex() == 1) { + consumer.negativeAcknowledge(message); + index1HasBeenNegativeAcked = true; + continue; + } + messagesReceived.add(message.getValue()); + consumer.acknowledge(message); + } + + // Receive negative acked messages. + // Wait the message negative acknowledgment finished. + int tripleRedeliveryDelaySeconds = redeliveryDelaySeconds * 3; + while (true) { + Message message = consumer.receive(tripleRedeliveryDelaySeconds, TimeUnit.SECONDS); + if (message == null) { + break; + } + messagesReceived.add(message.getValue()); + consumer.acknowledge(message); + } + + log.info("messagesSent: {}, messagesReceived: {}", messagesSent, messagesReceived); + Assert.assertEquals(messagesReceived.size(), messagesSent.size()); + + // cleanup. + producer.close(); + consumer.close(); + admin.topics().delete(topicName); + } } 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 9086ccc4ef0e0..0cf776aea5942 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 @@ -124,7 +124,18 @@ public boolean isDuplicate(MessageId messageId) { // Already included in a cumulative ack return true; } else { - return pendingIndividualAcks.contains(MessageIdAdvUtils.discardBatch(messageIdAdv)); + // If "batchIndexAckEnabled" is false, the batched messages acknowledgment will be traced by + // pendingIndividualAcks. So no matter what type the message ID is, check with "pendingIndividualAcks" + // first. + MessageIdAdv key = MessageIdAdvUtils.discardBatch(messageIdAdv); + if (pendingIndividualAcks.contains(key)) { + return true; + } + if (messageIdAdv.getBatchIndex() >= 0) { + ConcurrentBitSetRecyclable bitSet = pendingIndividualBatchIndexAcks.get(key); + return bitSet != null && !bitSet.get(messageIdAdv.getBatchIndex()); + } + return false; } } From 3897c4bf255ae9d57d19424220db0b3bf6265c25 Mon Sep 17 00:00:00 2001 From: thetumbled <52550727+thetumbled@users.noreply.github.com> Date: Thu, 31 Aug 2023 15:26:52 +0800 Subject: [PATCH 046/699] [fix] [broker] consider iowait as idle. (#19110) --- .../broker/loadbalance/LinuxInfoUtils.java | 5 +- .../impl/LinuxBrokerHostUsageImpl.java | 3 +- .../loadbalance/LinuxInfoUtilsTest.java | 50 +++++++++++++++++++ 3 files changed, 55 insertions(+), 3 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LinuxInfoUtilsTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LinuxInfoUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LinuxInfoUtils.java index 17aa7170fc63c..61f34ef4901ba 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LinuxInfoUtils.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LinuxInfoUtils.java @@ -161,7 +161,8 @@ public static long getCpuUsageForCGroup() { * *

* Line is split in "words", filtering the first. The sum of all numbers give the amount of cpu cycles used this - * far. Real CPU usage should equal the sum subtracting the idle cycles, this would include iowait, irq and steal. + * far. Real CPU usage should equal the sum substracting the idle cycles(that is idle+iowait), this would include + * cpu, user, nice, system, irq, softirq, steal, guest and guest_nice. */ public static ResourceUsage getCpuUsageForEntireHost() { try (Stream stream = Files.lines(Paths.get(PROC_STAT_PATH))) { @@ -175,7 +176,7 @@ public static ResourceUsage getCpuUsageForEntireHost() { .filter(s -> !s.contains("cpu")) .mapToLong(Long::parseLong) .sum(); - long idle = Long.parseLong(words[4]); + long idle = Long.parseLong(words[4]) + Long.parseLong(words[5]); return ResourceUsage.builder() .usage(total - idle) .idle(idle) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LinuxBrokerHostUsageImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LinuxBrokerHostUsageImpl.java index 2f7ca614943b1..6d0e6bb907346 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LinuxBrokerHostUsageImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/LinuxBrokerHostUsageImpl.java @@ -155,7 +155,8 @@ private double getTotalCpuUsageForCGroup(double elapsedTimeSeconds) { * * * Line is split in "words", filtering the first. The sum of all numbers give the amount of cpu cycles used this - * far. Real CPU usage should equal the sum subtracting the idle cycles, this would include iowait, irq and steal. + * far. Real CPU usage should equal the sum substracting the idle cycles(that is idle+iowait), this would include + * cpu, user, nice, system, irq, softirq, steal, guest and guest_nice. */ private double getTotalCpuUsageForEntireHost() { LinuxInfoUtils.ResourceUsage cpuUsageForEntireHost = getCpuUsageForEntireHost(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LinuxInfoUtilsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LinuxInfoUtilsTest.java new file mode 100644 index 0000000000000..ac21b30bdde51 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LinuxInfoUtilsTest.java @@ -0,0 +1,50 @@ +/* + * 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; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mockStatic; +import static org.testng.Assert.assertEquals; +import java.nio.file.Files; +import java.util.stream.Stream; +import lombok.extern.slf4j.Slf4j; +import org.mockito.MockedStatic; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class LinuxInfoUtilsTest { + + /** + * simulate reading first line of /proc/stat to get total cpu usage. + */ + @Test + public void testGetCpuUsageForEntireHost(){ + try (MockedStatic filesMockedStatic = mockStatic(Files.class)) { + filesMockedStatic.when(() -> Files.lines(any())).thenReturn( + Stream.generate(() -> "cpu 317808 128 58637 2503692 7634 0 13472 0 0 0")); + long idle = 2503692 + 7634, total = 2901371; + LinuxInfoUtils.ResourceUsage resourceUsage = LinuxInfoUtils.ResourceUsage.builder() + .usage(total - idle) + .idle(idle) + .total(total).build(); + assertEquals(LinuxInfoUtils.getCpuUsageForEntireHost(), resourceUsage); + } + } +} From 9f336506b1c0967323a897c91eca649ab0511479 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Fri, 1 Sep 2023 18:07:46 +0800 Subject: [PATCH 047/699] [fix][broker] Fix deleting topic not delete the related topic policy and schema. (#21093) Fixes #21075 ### Motivation When the topic is loaded, it will delete the topic-level policy if it is enabled. But if the topic is not loaded, it will directly delete through managed ledger factory. But then we will leave the topic policy there. When the topic is created next time, it will use the old topic policy ### Modifications When deleting the topic, delete the schema and topic policies even if the topic is not loaded. --- .../pulsar/broker/service/AbstractTopic.java | 17 +----- .../pulsar/broker/service/BrokerService.java | 55 ++++++++++++++----- .../service/BrokerBkEnsemblesTests.java | 32 +++++++++-- .../systopic/PartitionedSystemTopicTest.java | 25 +++++++++ 4 files changed, 94 insertions(+), 35 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 b15f8cbf0b848..cef2dd2080cf0 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 @@ -58,7 +58,6 @@ import org.apache.pulsar.broker.service.BrokerServiceException.TopicMigratedException; import org.apache.pulsar.broker.service.BrokerServiceException.TopicTerminatedException; import org.apache.pulsar.broker.service.plugin.EntryFilter; -import org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage; import org.apache.pulsar.broker.service.schema.SchemaRegistryService; import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; import org.apache.pulsar.broker.stats.prometheus.metrics.Summary; @@ -674,21 +673,7 @@ private boolean allowAutoUpdateSchema() { @Override public CompletableFuture deleteSchema() { - String id = getSchemaId(); - SchemaRegistryService schemaRegistryService = brokerService.pulsar().getSchemaRegistryService(); - return BookkeeperSchemaStorage.ignoreUnrecoverableBKException(schemaRegistryService.getSchema(id)) - .thenCompose(schema -> { - if (schema != null) { - // It's different from `SchemasResource.deleteSchema` - // because when we delete a topic, the schema - // history is meaningless. But when we delete a schema of a topic, a new schema could be - // registered in the future. - log.info("Delete schema storage of id: {}", id); - return schemaRegistryService.deleteSchemaStorage(id); - } else { - return CompletableFuture.completedFuture(null); - } - }); + return brokerService.deleteSchema(TopicName.get(getName())); } @Override 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 6c48e4d9ae89f..2cf141ed329ab 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 @@ -119,6 +119,8 @@ import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.persistent.SystemTopic; import org.apache.pulsar.broker.service.plugin.EntryFilterProvider; +import org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage; +import org.apache.pulsar.broker.service.schema.SchemaRegistryService; 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; @@ -159,6 +161,7 @@ import org.apache.pulsar.common.policies.data.TopicPolicies; import org.apache.pulsar.common.policies.data.TopicType; import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; +import org.apache.pulsar.common.protocol.schema.SchemaVersion; import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.util.FieldParser; import org.apache.pulsar.common.util.FutureUtil; @@ -1156,26 +1159,33 @@ private CompletableFuture deleteTopicInternal(String topic, boolean forceD CompletableFuture future = new CompletableFuture<>(); CompletableFuture deleteTopicAuthenticationFuture = new CompletableFuture<>(); deleteTopicAuthenticationWithRetry(topic, deleteTopicAuthenticationFuture, 5); - - deleteTopicAuthenticationFuture.whenComplete((v, ex) -> { + deleteTopicAuthenticationFuture + .thenCompose(__ -> deleteSchema(tn)) + .thenCompose(__ -> { + if (!SystemTopicNames.isTopicPoliciesSystemTopic(topic) + && getPulsar().getConfiguration().isSystemTopicEnabled()) { + return deleteTopicPolicies(tn); + } + return CompletableFuture.completedFuture(null); + }).whenComplete((v, ex) -> { if (ex != null) { future.completeExceptionally(ex); return; } CompletableFuture mlConfigFuture = getManagedLedgerConfig(topicName); managedLedgerFactory.asyncDelete(tn.getPersistenceNamingEncoding(), - mlConfigFuture, new DeleteLedgerCallback() { - @Override - public void deleteLedgerComplete(Object ctx) { - future.complete(null); - } + 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); + }); return future; } @@ -3440,6 +3450,25 @@ public CompletableFuture deleteTopicPolicies(TopicName topicName) { .deleteTopicPoliciesAsync(TopicName.get(topicName.getPartitionedTopicName())); } + CompletableFuture deleteSchema(TopicName topicName) { + String base = topicName.getPartitionedTopicName(); + String id = TopicName.get(base).getSchemaName(); + SchemaRegistryService schemaRegistryService = getPulsar().getSchemaRegistryService(); + return BookkeeperSchemaStorage.ignoreUnrecoverableBKException(schemaRegistryService.getSchema(id)) + .thenCompose(schema -> { + if (schema != null) { + // It's different from `SchemasResource.deleteSchema` + // because when we delete a topic, the schema + // history is meaningless. But when we delete a schema of a topic, a new schema could be + // registered in the future. + log.info("Delete schema storage of id: {}", id); + return getPulsar().getSchemaRegistryService().deleteSchemaStorage(id); + } else { + return CompletableFuture.completedFuture(null); + } + }); + } + private CompletableFuture checkMaxTopicsPerNamespace(TopicName topicName, int numPartitions) { return pulsar.getPulsarResources().getNamespaceResources() .getPoliciesAsync(topicName.getNamespaceObject()) 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 9f19bda3647f3..40649a4164047 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 @@ -21,8 +21,8 @@ import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.retryStrategically; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotEquals; +import static org.testng.Assert.assertThrows; import static org.testng.Assert.fail; - import java.lang.reflect.Field; import java.util.Map.Entry; import java.util.NavigableMap; @@ -31,10 +31,8 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; - import com.google.common.collect.Sets; import lombok.Cleanup; - import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; @@ -47,6 +45,7 @@ import org.apache.bookkeeper.util.StringUtils; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +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; @@ -497,10 +496,31 @@ public void testDeleteTopicWithMissingData() throws Exception { // Expected } - // Deletion must succeed - admin.topics().delete(topic); + assertThrows(PulsarAdminException.ServerSideErrorException.class, () -> admin.topics().delete(topic)); + } + + @Test + public void testDeleteTopicWithoutTopicLoaded() throws Exception { + String namespace = BrokerTestUtil.newUniqueName("prop/usc"); + admin.namespaces().createNamespace(namespace); + + String topic = BrokerTestUtil.newUniqueName(namespace + "/my-topic"); + + @Cleanup + PulsarClient client = PulsarClient.builder() + .serviceUrl(pulsar.getBrokerServiceUrl()) + .statsInterval(0, TimeUnit.SECONDS) + .build(); - // Topic will not be there after + @Cleanup + Producer producer = client.newProducer(Schema.STRING) + .topic(topic) + .create(); + + producer.close(); + admin.topics().unload(topic); + + admin.topics().delete(topic); assertEquals(pulsar.getBrokerService().getTopicIfExists(topic).join(), Optional.empty()); } 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 6f56852cae343..4af0bd9052391 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 @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker.systopic; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNull; import com.google.common.collect.Sets; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -37,6 +39,7 @@ import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.service.schema.SchemaRegistry; import org.apache.pulsar.client.admin.ListTopicsOptions; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -55,6 +58,7 @@ import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.TopicPolicies; import org.apache.pulsar.common.policies.data.TopicType; import org.apache.pulsar.common.util.FutureUtil; import org.awaitility.Awaitility; @@ -299,4 +303,25 @@ public void testSystemTopicNotCheckExceed() throws Exception { writer1.get().close(); writer2.get().close(); } + + @Test + public void testDeleteTopicSchemaAndPolicyWhenTopicIsNotLoaded() throws Exception { + final String ns = "prop/ns-test"; + admin.namespaces().createNamespace(ns, 2); + final String topicName = "persistent://prop/ns-test/testDeleteTopicSchemaAndPolicyWhenTopicIsNotLoaded"; + admin.topics().createNonPartitionedTopic(topicName); + pulsarClient.newProducer(Schema.STRING).topic(topicName).create().close(); + admin.topicPolicies().setMaxConsumers(topicName, 2); + Awaitility.await().untilAsserted(() -> assertEquals(admin.topicPolicies().getMaxConsumers(topicName), 2)); + CompletableFuture> topic = pulsar.getBrokerService().getTopic(topicName, false); + PersistentTopic persistentTopic = (PersistentTopic) topic.join().get(); + persistentTopic.close(); + admin.topics().delete(topicName); + TopicPolicies topicPolicies = pulsar.getTopicPoliciesService().getTopicPoliciesIfExists(TopicName.get(topicName)); + assertNull(topicPolicies); + String base = TopicName.get(topicName).getPartitionedTopicName(); + String id = TopicName.get(base).getSchemaName(); + CompletableFuture schema = pulsar.getSchemaRegistryService().getSchema(id); + assertNull(schema.join()); + } } From d896814ab08ddda76a97dc98b8f848cba84a9cf1 Mon Sep 17 00:00:00 2001 From: ken <1647023764@qq.com> Date: Tue, 5 Sep 2023 16:21:23 +0800 Subject: [PATCH 048/699] [fix][broker] revert remove duplicate topics name when deleteNamespace (#21087) --- .../org/apache/pulsar/broker/admin/impl/NamespacesBase.java | 6 ------ 1 file changed, 6 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 789a8e6dbdcb7..9c5813cbb1777 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 @@ -279,12 +279,6 @@ private void internalRetryableDeleteNamespaceAsync0(boolean force, int retryTime return old; }); } - allUserCreatedTopics.removeIf(t -> - allPartitionedTopics.contains(TopicName.get(t).getPartitionedTopicName())); - allSystemTopics.removeIf(t -> - allPartitionedTopics.contains(TopicName.get(t).getPartitionedTopicName())); - topicPolicy.removeIf(t -> - allPartitionedTopics.contains(TopicName.get(t).getPartitionedTopicName())); return markDeleteFuture.thenCompose(__ -> internalDeleteTopicsAsync(allUserCreatedTopics)) .thenCompose(ignore -> From 702ee731d71e0b887dd225726aea11794af2022b Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Mon, 11 Sep 2023 09:55:21 +0800 Subject: [PATCH 049/699] [improve][broker] Upgrade bookkeeper to 4.16.3 (#21146) --- .../server/src/assemble/LICENSE.bin.txt | 56 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 6 +- pom.xml | 2 +- pulsar-sql/presto-distribution/LICENSE | 30 +++++----- 4 files changed, 47 insertions(+), 47 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 23a0b67c4e8d5..d32c44a6651c5 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -346,34 +346,34 @@ The Apache Software License, Version 2.0 - net.java.dev.jna-jna-jpms-5.12.1.jar - net.java.dev.jna-jna-platform-jpms-5.12.1.jar * BookKeeper - - org.apache.bookkeeper-bookkeeper-common-4.16.2.jar - - org.apache.bookkeeper-bookkeeper-common-allocator-4.16.2.jar - - org.apache.bookkeeper-bookkeeper-proto-4.16.2.jar - - org.apache.bookkeeper-bookkeeper-server-4.16.2.jar - - org.apache.bookkeeper-bookkeeper-tools-framework-4.16.2.jar - - org.apache.bookkeeper-circe-checksum-4.16.2.jar - - org.apache.bookkeeper-cpu-affinity-4.16.2.jar - - org.apache.bookkeeper-statelib-4.16.2.jar - - org.apache.bookkeeper-stream-storage-api-4.16.2.jar - - org.apache.bookkeeper-stream-storage-common-4.16.2.jar - - org.apache.bookkeeper-stream-storage-java-client-4.16.2.jar - - org.apache.bookkeeper-stream-storage-java-client-base-4.16.2.jar - - org.apache.bookkeeper-stream-storage-proto-4.16.2.jar - - org.apache.bookkeeper-stream-storage-server-4.16.2.jar - - org.apache.bookkeeper-stream-storage-service-api-4.16.2.jar - - org.apache.bookkeeper-stream-storage-service-impl-4.16.2.jar - - org.apache.bookkeeper.http-http-server-4.16.2.jar - - org.apache.bookkeeper.http-vertx-http-server-4.16.2.jar - - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.16.2.jar - - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.16.2.jar - - org.apache.distributedlog-distributedlog-common-4.16.2.jar - - org.apache.distributedlog-distributedlog-core-4.16.2-tests.jar - - org.apache.distributedlog-distributedlog-core-4.16.2.jar - - org.apache.distributedlog-distributedlog-protocol-4.16.2.jar - - org.apache.bookkeeper.stats-codahale-metrics-provider-4.16.2.jar - - org.apache.bookkeeper-bookkeeper-slogger-api-4.16.2.jar - - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.16.2.jar - - org.apache.bookkeeper-native-io-4.16.2.jar + - org.apache.bookkeeper-bookkeeper-common-4.16.3.jar + - org.apache.bookkeeper-bookkeeper-common-allocator-4.16.3.jar + - org.apache.bookkeeper-bookkeeper-proto-4.16.3.jar + - org.apache.bookkeeper-bookkeeper-server-4.16.3.jar + - org.apache.bookkeeper-bookkeeper-tools-framework-4.16.3.jar + - org.apache.bookkeeper-circe-checksum-4.16.3.jar + - org.apache.bookkeeper-cpu-affinity-4.16.3.jar + - org.apache.bookkeeper-statelib-4.16.3.jar + - org.apache.bookkeeper-stream-storage-api-4.16.3.jar + - org.apache.bookkeeper-stream-storage-common-4.16.3.jar + - org.apache.bookkeeper-stream-storage-java-client-4.16.3.jar + - org.apache.bookkeeper-stream-storage-java-client-base-4.16.3.jar + - org.apache.bookkeeper-stream-storage-proto-4.16.3.jar + - org.apache.bookkeeper-stream-storage-server-4.16.3.jar + - org.apache.bookkeeper-stream-storage-service-api-4.16.3.jar + - org.apache.bookkeeper-stream-storage-service-impl-4.16.3.jar + - org.apache.bookkeeper.http-http-server-4.16.3.jar + - org.apache.bookkeeper.http-vertx-http-server-4.16.3.jar + - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.16.3.jar + - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.16.3.jar + - org.apache.distributedlog-distributedlog-common-4.16.3.jar + - org.apache.distributedlog-distributedlog-core-4.16.3-tests.jar + - org.apache.distributedlog-distributedlog-core-4.16.3.jar + - org.apache.distributedlog-distributedlog-protocol-4.16.3.jar + - org.apache.bookkeeper.stats-codahale-metrics-provider-4.16.3.jar + - org.apache.bookkeeper-bookkeeper-slogger-api-4.16.3.jar + - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.16.3.jar + - org.apache.bookkeeper-native-io-4.16.3.jar * Apache HTTP Client - org.apache.httpcomponents-httpclient-4.5.13.jar - org.apache.httpcomponents-httpcore-4.4.15.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index dc8babb32f329..f92d95928298a 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -386,9 +386,9 @@ The Apache Software License, Version 2.0 - log4j-web-2.18.0.jar * BookKeeper - - bookkeeper-common-allocator-4.16.2.jar - - cpu-affinity-4.16.2.jar - - circe-checksum-4.16.2.jar + - bookkeeper-common-allocator-4.16.3.jar + - cpu-affinity-4.16.3.jar + - circe-checksum-4.16.3.jar * AirCompressor - aircompressor-0.20.jar * AsyncHttpClient diff --git a/pom.xml b/pom.xml index b64fbe491d846..d67c48c6308c7 100644 --- a/pom.xml +++ b/pom.xml @@ -133,7 +133,7 @@ flexible messaging model and an intuitive client API. 1.21 - 4.16.2 + 4.16.3 3.8.1 1.5.0 1.10.0 diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 738ef83abff27..61332595c65b8 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -430,21 +430,21 @@ The Apache Software License, Version 2.0 - async-http-client-2.12.1.jar - async-http-client-netty-utils-2.12.1.jar * Apache Bookkeeper - - bookkeeper-common-4.16.2.jar - - bookkeeper-common-allocator-4.16.2.jar - - bookkeeper-proto-4.16.2.jar - - bookkeeper-server-4.16.2.jar - - bookkeeper-stats-api-4.16.2.jar - - bookkeeper-tools-framework-4.16.2.jar - - circe-checksum-4.16.2.jar - - codahale-metrics-provider-4.16.2.jar - - cpu-affinity-4.16.2.jar - - http-server-4.16.2.jar - - prometheus-metrics-provider-4.16.2.jar - - codahale-metrics-provider-4.16.2.jar - - bookkeeper-slogger-api-4.16.2.jar - - bookkeeper-slogger-slf4j-4.16.2.jar - - native-io-4.16.2.jar + - bookkeeper-common-4.16.3.jar + - bookkeeper-common-allocator-4.16.3.jar + - bookkeeper-proto-4.16.3.jar + - bookkeeper-server-4.16.3.jar + - bookkeeper-stats-api-4.16.3.jar + - bookkeeper-tools-framework-4.16.3.jar + - circe-checksum-4.16.3.jar + - codahale-metrics-provider-4.16.3.jar + - cpu-affinity-4.16.3.jar + - http-server-4.16.3.jar + - prometheus-metrics-provider-4.16.3.jar + - codahale-metrics-provider-4.16.3.jar + - bookkeeper-slogger-api-4.16.3.jar + - bookkeeper-slogger-slf4j-4.16.3.jar + - native-io-4.16.3.jar * Apache Commons - commons-cli-1.5.0.jar - commons-codec-1.15.jar From 52a7fb610776f63eb9ea25c1248730a182f44bc5 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 7 Sep 2023 03:15:22 +0800 Subject: [PATCH 050/699] [improve] [broker] Improve logs for troubleshooting (#21141) --- .../mledger/impl/ManagedCursorImpl.java | 26 +++++++++++-------- .../mledger/impl/NonDurableCursorImpl.java | 8 ++++-- .../mledger/impl/NonDurableCursorTest.java | 3 ++- .../service/persistent/PersistentTopic.java | 8 +++--- 4 files changed, 27 insertions(+), 18 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 a2420c1c29efb..ff8e0655d03be 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 @@ -509,7 +509,7 @@ public void operationComplete(ManagedCursorInfo info, Stat stat) { callback.operationComplete(); } else { // Need to proceed and read the last entry in the specified ledger to find out the last position - log.info("[{}] Consumer {} meta-data recover from ledger {}", ledger.getName(), name, + log.info("[{}] Cursor {} meta-data recover from ledger {}", ledger.getName(), name, info.getCursorsLedgerId()); recoverFromLedger(info, callback); } @@ -529,16 +529,16 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac long ledgerId = info.getCursorsLedgerId(); OpenCallback openCallback = (rc, lh, ctx) -> { if (log.isInfoEnabled()) { - log.info("[{}] Opened ledger {} for consumer {}. rc={}", ledger.getName(), ledgerId, name, rc); + log.info("[{}] Opened ledger {} for cursor {}. rc={}", ledger.getName(), ledgerId, name, rc); } if (isBkErrorNotRecoverable(rc)) { - log.error("[{}] Error opening metadata ledger {} for consumer {}: {}", ledger.getName(), ledgerId, name, + log.error("[{}] Error opening metadata ledger {} for cursor {}: {}", ledger.getName(), ledgerId, name, BKException.getMessage(rc)); // Rewind to oldest entry available initialize(getRollbackPosition(info), Collections.emptyMap(), Collections.emptyMap(), callback); return; } else if (rc != BKException.Code.OK) { - log.warn("[{}] Error opening metadata ledger {} for consumer {}: {}", ledger.getName(), ledgerId, name, + log.warn("[{}] Error opening metadata ledger {} for cursor {}: {}", ledger.getName(), ledgerId, name, BKException.getMessage(rc)); callback.operationFailed(new ManagedLedgerException(BKException.getMessage(rc))); return; @@ -548,7 +548,7 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac long lastEntryInLedger = lh.getLastAddConfirmed(); if (lastEntryInLedger < 0) { - log.warn("[{}] Error reading from metadata ledger {} for consumer {}: No entries in ledger", + log.warn("[{}] Error reading from metadata ledger {} for cursor {}: No entries in ledger", ledger.getName(), ledgerId, name); // Rewind to last cursor snapshot available initialize(getRollbackPosition(info), Collections.emptyMap(), cursorProperties, callback); @@ -560,13 +560,13 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac log.debug("[{}} readComplete rc={} entryId={}", ledger.getName(), rc1, lh1.getLastAddConfirmed()); } if (isBkErrorNotRecoverable(rc1)) { - log.error("[{}] Error reading from metadata ledger {} for consumer {}: {}", ledger.getName(), + log.error("[{}] Error reading from metadata ledger {} for cursor {}: {}", ledger.getName(), ledgerId, name, BKException.getMessage(rc1)); // Rewind to oldest entry available initialize(getRollbackPosition(info), Collections.emptyMap(), cursorProperties, callback); return; } else if (rc1 != BKException.Code.OK) { - log.warn("[{}] Error reading from metadata ledger {} for consumer {}: {}", ledger.getName(), + log.warn("[{}] Error reading from metadata ledger {} for cursor {}: {}", ledger.getName(), ledgerId, name, BKException.getMessage(rc1)); callback.operationFailed(createManagedLedgerException(rc1)); @@ -2453,8 +2453,12 @@ List filterReadEntries(List entries) { @Override public synchronized String toString() { - return MoreObjects.toStringHelper(this).add("ledger", ledger.getName()).add("name", name) - .add("ackPos", markDeletePosition).add("readPos", readPosition).toString(); + return MoreObjects.toStringHelper(this) + .add("ledger", ledger.getName()) + .add("name", name) + .add("ackPos", markDeletePosition) + .add("readPos", readPosition) + .toString(); } @Override @@ -3068,7 +3072,7 @@ void persistPositionToLedger(final LedgerHandle lh, MarkDeleteEntry mdEntry, fin if (shouldCloseLedger(lh1)) { if (log.isDebugEnabled()) { - log.debug("[{}] Need to create new metadata ledger for consumer {}", ledger.getName(), name); + log.debug("[{}] Need to create new metadata ledger for cursor {}", ledger.getName(), name); } startCreatingNewMetadataLedger(); } @@ -3153,7 +3157,7 @@ public void operationComplete(Void result, Stat stat) { @Override public void operationFailed(MetaStoreException e) { - log.warn("[{}] Failed to update consumer {}", ledger.getName(), name, e); + log.warn("[{}] Failed to update cursor metadata {}", ledger.getName(), name, e); // it means it failed to switch the newly created ledger so, it should be // deleted to prevent leak deleteLedgerAsync(lh).thenRun(() -> callback.operationFailed(e)); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java index 9d2829b1707f4..51e56158cad55 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java @@ -138,8 +138,12 @@ public void rewind() { @Override public synchronized String toString() { - return MoreObjects.toStringHelper(this).add("ledger", ledger.getName()).add("ackPos", markDeletePosition) - .add("readPos", readPosition).toString(); + return MoreObjects.toStringHelper(this) + .add("ledger", ledger.getName()) + .add("cursor", getName()) + .add("ackPos", markDeletePosition) + .add("readPos", readPosition) + .toString(); } private static final Logger log = LoggerFactory.getLogger(NonDurableCursorImpl.class); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java index 1ad3f5f8de631..1e1f7df0a46d5 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java @@ -80,7 +80,8 @@ void readFromEmptyLedger() throws Exception { entries.forEach(Entry::release); // Test string representation - assertEquals(c1.toString(), "NonDurableCursorImpl{ledger=my_test_ledger, ackPos=3:-1, readPos=3:1}"); + assertEquals(c1.toString(), "NonDurableCursorImpl{ledger=my_test_ledger, cursor=" + + c1.getName() + ", ackPos=3:-1, readPos=3:1}"); } @Test(timeOut = 20000) 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 727c896006600..1bfe55cf74c29 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 @@ -905,8 +905,8 @@ private CompletableFuture internalSubscribe(final TransportCnx cnx, St consumer.close(); } catch (BrokerServiceException e) { if (e instanceof ConsumerBusyException) { - log.warn("[{}][{}] Consumer {} {} already connected", - topic, subscriptionName, consumerId, consumerName); + log.warn("[{}][{}] Consumer {} {} already connected: {}", + topic, subscriptionName, consumerId, consumerName, e.getMessage()); } else if (e instanceof SubscriptionBusyException) { log.warn("[{}][{}] {}", topic, subscriptionName, e.getMessage()); } @@ -936,8 +936,8 @@ private CompletableFuture internalSubscribe(final TransportCnx cnx, St decrementUsageCount(); if (ex.getCause() instanceof ConsumerBusyException) { - log.warn("[{}][{}] Consumer {} {} already connected", topic, subscriptionName, consumerId, - consumerName); + log.warn("[{}][{}] Consumer {} {} already connected: {}", topic, subscriptionName, consumerId, + consumerName, ex.getCause().getMessage()); Consumer consumer = null; try { consumer = subscriptionFuture.isDone() ? getActiveConsumer(subscriptionFuture.get()) : null; From 87062a56c15aa935a03b41719021303cfc443d34 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Wed, 6 Sep 2023 20:27:23 -0500 Subject: [PATCH 051/699] [fix][broker] Fix web tls url null cause NPE (#21137) --- .../apache/pulsar/broker/rest/TopicsBase.java | 13 +- .../broker/admin/TopicsWithoutTlsTest.java | 128 ++++++++++++++++++ 2 files changed, 135 insertions(+), 6 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicsWithoutTlsTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/rest/TopicsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/rest/TopicsBase.java index 5ab81fcbbff27..20c35b4f7769c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/rest/TopicsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/rest/TopicsBase.java @@ -54,6 +54,7 @@ import org.apache.avro.io.DecoderFactory; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.admin.impl.PersistentTopicsBase; import org.apache.pulsar.broker.authentication.AuthenticationParameters; @@ -433,8 +434,10 @@ private CompletableFuture lookUpBrokerForTopic(TopicName partitionedTopicN } LookupResult result = optionalResult.get(); - if (result.getLookupData().getHttpUrl().equals(pulsar().getWebServiceAddress()) - || result.getLookupData().getHttpUrlTls().equals(pulsar().getWebServiceAddressTls())) { + String httpUrl = result.getLookupData().getHttpUrl(); + String httpUrlTls = result.getLookupData().getHttpUrlTls(); + if ((StringUtils.isNotBlank(httpUrl) && httpUrl.equals(pulsar().getWebServiceAddress())) + || (StringUtils.isNotBlank(httpUrlTls) && httpUrlTls.equals(pulsar().getWebServiceAddressTls()))) { // Current broker owns the topic, add to owning topic. if (log.isDebugEnabled()) { log.debug("Complete topic look up for rest produce message request for topic {}, " @@ -455,12 +458,10 @@ private CompletableFuture lookUpBrokerForTopic(TopicName partitionedTopicN } if (result.isRedirect()) { // Redirect lookup. - completeLookup(Pair.of(Arrays.asList(result.getLookupData().getHttpUrl(), - result.getLookupData().getHttpUrlTls()), false), redirectAddresses, future); + completeLookup(Pair.of(Arrays.asList(httpUrl, httpUrlTls), false), redirectAddresses, future); } else { // Found owner for topic. - completeLookup(Pair.of(Arrays.asList(result.getLookupData().getHttpUrl(), - result.getLookupData().getHttpUrlTls()), true), redirectAddresses, future); + completeLookup(Pair.of(Arrays.asList(httpUrl, httpUrlTls), true), redirectAddresses, future); } } }).exceptionally(exception -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicsWithoutTlsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicsWithoutTlsTest.java new file mode 100644 index 0000000000000..88bf2f8f42108 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicsWithoutTlsTest.java @@ -0,0 +1,128 @@ +/* + * 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 com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import lombok.Cleanup; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.broker.authentication.AuthenticationDataHttps; +import org.apache.pulsar.broker.rest.Topics; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.naming.TopicDomain; +import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.util.ObjectMapperFactory; +import org.apache.pulsar.websocket.data.ProducerMessage; +import org.apache.pulsar.websocket.data.ProducerMessages; +import org.mockito.ArgumentCaptor; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; +import javax.ws.rs.container.AsyncResponse; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.UriInfo; +import java.net.URI; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.timeout; +import static org.mockito.Mockito.verify; + +public class TopicsWithoutTlsTest extends MockedPulsarServiceBaseTest { + + private Topics topics; + private final String testLocalCluster = "test"; + private final String testTenant = "my-tenant"; + private final String testNamespace = "my-namespace"; + private final String testTopicName = "my-topic"; + + @Override + @BeforeMethod + protected void setup() throws Exception { + super.internalSetup(); + topics = spy(new Topics()); + topics.setPulsar(pulsar); + doReturn(TopicDomain.persistent.value()).when(topics).domain(); + doReturn("test-app").when(topics).clientAppId(); + doReturn(mock(AuthenticationDataHttps.class)).when(topics).clientAuthData(); + admin.clusters().createCluster(testLocalCluster, new ClusterDataImpl()); + admin.tenants().createTenant(testTenant, new TenantInfoImpl(Set.of("role1", "role2"), Set.of(testLocalCluster))); + admin.namespaces().createNamespace(testTenant + "/" + testNamespace, + Set.of(testLocalCluster)); + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + this.conf.setBrokerServicePortTls(Optional.empty()); + this.conf.setWebServicePortTls(Optional.empty()); + } + + @Override + @AfterMethod + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test + public void testLookUpWithRedirect() throws Exception { + String topicName = "persistent://" + testTenant + "/" + testNamespace + "/" + testTopicName; + URI requestPath = URI.create(pulsar.getWebServiceAddress() + "/topics/my-tenant/my-namespace/my-topic"); + //create topic on one broker + admin.topics().createNonPartitionedTopic(topicName); + conf.setBrokerServicePort(Optional.of(0)); + conf.setWebServicePort(Optional.of(0)); + @Cleanup + PulsarTestContext pulsarTestContext2 = createAdditionalPulsarTestContext(conf); + PulsarService pulsar2 = pulsarTestContext2.getPulsarService(); + doReturn(false).when(topics).isRequestHttps(); + UriInfo uriInfo = mock(UriInfo.class); + doReturn(requestPath).when(uriInfo).getRequestUri(); + FieldUtils.writeField(topics, "uri", uriInfo, true); + //do produce on another broker + topics.setPulsar(pulsar2); + AsyncResponse asyncResponse = mock(AsyncResponse.class); + ProducerMessages producerMessages = new ProducerMessages(); + producerMessages.setValueSchema(ObjectMapperFactory.getMapper().getObjectMapper(). + writeValueAsString(Schema.INT64.getSchemaInfo())); + String message = "[]"; + producerMessages.setMessages(createMessages(message)); + topics.produceOnPersistentTopic(asyncResponse, testTenant, testNamespace, testTopicName, false, producerMessages); + ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(Response.class); + verify(asyncResponse, timeout(5000).times(1)).resume(responseCaptor.capture()); + // Verify got redirect response + Assert.assertEquals(responseCaptor.getValue().getStatusInfo(), Response.Status.TEMPORARY_REDIRECT); + // Verify URI point to address of broker the topic was created on + Assert.assertEquals(responseCaptor.getValue().getLocation().toString(), requestPath.toString()); + } + + private static List createMessages(String message) throws JsonProcessingException { + return ObjectMapperFactory.getMapper().reader() + .forType(new TypeReference>() { + }).readValue(message); + } +} From ec0aa2ac260ae8d850ed125a65f20441e20913e5 Mon Sep 17 00:00:00 2001 From: erobot Date: Mon, 11 Sep 2023 09:56:34 +0800 Subject: [PATCH 052/699] [fix][broker] Fix unack count when mixing non batch index and batch index acks (#21126) --- .../pulsar/broker/service/Consumer.java | 9 ++--- .../BatchMessageWithBatchIndexLevelTest.java | 36 +++++++++++++++++++ 2 files changed, 38 insertions(+), 7 deletions(-) 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 d138eab37581c..ecf8682084803 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 @@ -487,7 +487,6 @@ public CompletableFuture messageAcked(CommandAck ack) { private CompletableFuture individualAckNormal(CommandAck ack, Map properties) { List positionsAcked = new ArrayList<>(); long totalAckCount = 0; - boolean individualAck = false; for (int i = 0; i < ack.getMessageIdsCount(); i++) { MessageIdData msgId = ack.getMessageIdAt(i); PositionImpl position; @@ -508,19 +507,15 @@ private CompletableFuture individualAckNormal(CommandAck ack, Map producer = pulsarClient.newProducer() + .topic(topicName) + .enableBatching(true) + .batchingMaxPublishDelay(100, TimeUnit.MILLISECONDS) + .create(); + @Cleanup + Consumer consumer = pulsarClient.newConsumer() + .topic(topicName) + .subscriptionName("sub") + .subscriptionType(SubscriptionType.Shared) + .acknowledgmentGroupTime(100, TimeUnit.MILLISECONDS) + .enableBatchIndexAcknowledgment(true) + .isAckReceiptEnabled(true) + .subscribe(); + + // send two batch messages: [(1), (2,3)] + producer.send("1".getBytes()); + producer.sendAsync("2".getBytes()); + producer.send("3".getBytes()); + + Message message1 = consumer.receive(); + Message message2 = consumer.receive(); + Message message3 = consumer.receive(); + consumer.acknowledgeAsync(message1); + consumer.acknowledge(message2); // send group ack: non-index ack for 1, index ack for 2 + consumer.acknowledge(message3); // index ack for 3 + + assertEquals(admin.topics().getStats(topicName).getSubscriptions() + .get("sub").getUnackedMessages(), 0); + } } From 1b9ff6009c494afc0d88b450b340b062b2e67c10 Mon Sep 17 00:00:00 2001 From: Apurva007 Date: Tue, 5 Sep 2023 07:46:30 -0700 Subject: [PATCH 053/699] [fix][proxy] Fix Proxy 502 gateway error when it is configured with Keystore TLS and admin API is called (#21077) --- .../proxy/server/AdminProxyHandler.java | 41 +++-- .../AdminProxyHandlerKeystoreTLSTest.java | 142 ++++++++++++++++++ 2 files changed, 171 insertions(+), 12 deletions(-) create mode 100644 pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java 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 d9dda9823ea89..c528ceb2cf5b7 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 @@ -41,8 +41,10 @@ import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationDataProvider; import org.apache.pulsar.client.api.AuthenticationFactory; +import org.apache.pulsar.client.api.KeyStoreParams; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.common.util.SecurityUtility; +import org.apache.pulsar.common.util.keystoretls.KeyStoreSSLContext; import org.apache.pulsar.policies.data.loadbalancer.ServiceLookupData; import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.client.HttpRequest; @@ -269,20 +271,35 @@ protected HttpClient newHttpClient() { SSLContext sslCtx; AuthenticationDataProvider authData = auth.getAuthData(); - if (authData.hasDataForTls()) { - sslCtx = SecurityUtility.createSslContext( + if (config.isBrokerClientTlsEnabledWithKeyStore()) { + KeyStoreParams params = authData.hasDataForTls() ? authData.getTlsKeyStoreParams() : null; + sslCtx = KeyStoreSSLContext.createClientSslContext( + config.getBrokerClientSslProvider(), + params != null ? params.getKeyStoreType() : null, + params != null ? params.getKeyStorePath() : null, + params != null ? params.getKeyStorePassword() : null, config.isTlsAllowInsecureConnection(), - trustCertificates, - authData.getTlsCertificates(), - authData.getTlsPrivateKey(), - config.getBrokerClientSslProvider() - ); + config.getBrokerClientTlsTrustStoreType(), + config.getBrokerClientTlsTrustStore(), + config.getBrokerClientTlsTrustStorePassword(), + config.getBrokerClientTlsCiphers(), + config.getBrokerClientTlsProtocols()); } else { - sslCtx = SecurityUtility.createSslContext( - config.isTlsAllowInsecureConnection(), - trustCertificates, - config.getBrokerClientSslProvider() - ); + if (authData.hasDataForTls()) { + sslCtx = SecurityUtility.createSslContext( + config.isTlsAllowInsecureConnection(), + trustCertificates, + authData.getTlsCertificates(), + authData.getTlsPrivateKey(), + config.getBrokerClientSslProvider() + ); + } else { + sslCtx = SecurityUtility.createSslContext( + config.isTlsAllowInsecureConnection(), + trustCertificates, + config.getBrokerClientSslProvider() + ); + } } SslContextFactory contextFactory = new SslContextFactory.Client(); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java new file mode 100644 index 0000000000000..d6796b7eaa6d2 --- /dev/null +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java @@ -0,0 +1,142 @@ +/* + * 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.proxy.server; + +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.broker.authentication.AuthenticationProviderTls; +import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.broker.resources.PulsarResources; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls; +import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.metadata.impl.ZKMetadataStore; +import org.apache.pulsar.policies.data.loadbalancer.LoadManagerReport; +import org.apache.pulsar.policies.data.loadbalancer.LoadReport; +import org.eclipse.jetty.servlet.ServletHolder; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; + +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; + +public class AdminProxyHandlerKeystoreTLSTest extends MockedPulsarServiceBaseTest { + + + private final ProxyConfiguration proxyConfig = new ProxyConfiguration(); + + private WebServer webServer; + + private BrokerDiscoveryProvider discoveryProvider; + + private PulsarResources resource; + + @BeforeMethod + @Override + protected void setup() throws Exception { + + conf.setAuthenticationEnabled(true); + conf.setAuthorizationEnabled(false); + conf.setWebServicePortTls(Optional.of(0)); + conf.setBrokerServicePortTls(Optional.of(0)); + conf.setTlsEnabledWithKeyStore(true); + conf.setTlsAllowInsecureConnection(false); + conf.setTlsKeyStoreType(KEYSTORE_TYPE); + conf.setTlsKeyStore(BROKER_KEYSTORE_FILE_PATH); + conf.setTlsKeyStorePassword(BROKER_KEYSTORE_PW); + conf.setTlsTrustStoreType(KEYSTORE_TYPE); + conf.setTlsTrustStore(CLIENT_TRUSTSTORE_FILE_PATH); + conf.setTlsTrustStorePassword(CLIENT_TRUSTSTORE_PW); + + super.internalSetup(); + + proxyConfig.setWebServicePort(Optional.of(0)); + proxyConfig.setBrokerProxyAllowedTargetPorts("*"); + proxyConfig.setServicePortTls(Optional.of(0)); + proxyConfig.setWebServicePortTls(Optional.of(0)); + proxyConfig.setTlsEnabledWithBroker(true); + proxyConfig.setTlsEnabledWithKeyStore(true); + + proxyConfig.setTlsKeyStoreType(KEYSTORE_TYPE); + proxyConfig.setTlsKeyStore(BROKER_KEYSTORE_FILE_PATH); + proxyConfig.setTlsKeyStorePassword(BROKER_KEYSTORE_PW); + proxyConfig.setTlsTrustStoreType(KEYSTORE_TYPE); + proxyConfig.setTlsTrustStore(CLIENT_TRUSTSTORE_FILE_PATH); + proxyConfig.setTlsTrustStorePassword(CLIENT_TRUSTSTORE_PW); + + proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); + proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyConfig.setBrokerClientTlsEnabledWithKeyStore(true); + proxyConfig.setBrokerClientTlsKeyStoreType(KEYSTORE_TYPE); + proxyConfig.setBrokerClientTlsKeyStore(BROKER_KEYSTORE_FILE_PATH); + proxyConfig.setBrokerClientTlsKeyStorePassword(BROKER_KEYSTORE_PW); + proxyConfig.setBrokerClientTlsTrustStoreType(KEYSTORE_TYPE); + proxyConfig.setBrokerClientTlsTrustStore(BROKER_TRUSTSTORE_FILE_PATH); + proxyConfig.setBrokerClientTlsTrustStorePassword(BROKER_TRUSTSTORE_PW); + Set providers = new HashSet<>(); + providers.add(AuthenticationProviderTls.class.getName()); + proxyConfig.setAuthenticationProviders(providers); + proxyConfig.setBrokerClientAuthenticationPlugin(AuthenticationKeyStoreTls.class.getName()); + proxyConfig.setBrokerClientAuthenticationParameters(String.format("keyStoreType:%s,keyStorePath:%s,keyStorePassword:%s", + KEYSTORE_TYPE, BROKER_KEYSTORE_FILE_PATH, BROKER_KEYSTORE_PW)); + + resource = new PulsarResources(new ZKMetadataStore(mockZooKeeper), + new ZKMetadataStore(mockZooKeeperGlobal)); + webServer = new WebServer(proxyConfig, new AuthenticationService( + PulsarConfigurationLoader.convertFrom(proxyConfig))); + discoveryProvider = spy(new BrokerDiscoveryProvider(proxyConfig, resource)); + LoadManagerReport report = new LoadReport(brokerUrl.toString(), brokerUrlTls.toString(), null, null); + doReturn(report).when(discoveryProvider).nextBroker(); + ServletHolder servletHolder = new ServletHolder(new AdminProxyHandler(proxyConfig, discoveryProvider)); + webServer.addServlet("/admin", servletHolder); + webServer.addServlet("/lookup", servletHolder); + webServer.start(); + } + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + webServer.stop(); + super.internalCleanup(); + } + + PulsarAdmin getAdminClient() throws Exception { + return PulsarAdmin.builder() + .serviceHttpUrl("https://localhost:" + webServer.getListenPortHTTPS().get()) + .useKeyStoreTls(true) + .allowTlsInsecureConnection(false) + .tlsTrustStorePath(BROKER_TRUSTSTORE_FILE_PATH) + .tlsTrustStorePassword(BROKER_TRUSTSTORE_PW) + .authentication(AuthenticationKeyStoreTls.class.getName(), + String.format("keyStoreType:%s,keyStorePath:%s,keyStorePassword:%s", + KEYSTORE_TYPE, BROKER_KEYSTORE_FILE_PATH, BROKER_KEYSTORE_PW)) + .build(); + } + + @Test + public void testAdmin() throws Exception { + getAdminClient().clusters().createCluster(configClusterName, ClusterData.builder().serviceUrl(brokerUrl.toString()).build()); + } + +} From 6a605c8a127462257eec7e8bf21ad7f0c7b90bd9 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 11 Sep 2023 21:06:05 +0800 Subject: [PATCH 054/699] [fix] [client] fix same producer/consumer use more than one connection per broker (#21144) Motivation: Pulsar has two mechanisms to guarantee that a producer connects to the broker multiple times the result is still correct. - In a connection, the second connection waits for the first connection to complete. - In a topic, the second connection will override the previous one. However, if a producer can use different connections to connect to the broker, these two mechanisms will not work. When the config `connectionsPerBroker` of `PulsarClient` is larger than `1`, a producer could use more than one connection, leading to the error above. You can reproduce this issue by the test `testSelectConnectionForSameProducer.` Modifications: Make the same producer/consumer usage the same connection --- .../impl/TransactionBufferHandlerImpl.java | 8 +++- .../service/AbstractReplicatorTest.java | 4 ++ .../broker/service/PersistentTopicTest.java | 3 ++ .../NonStartableTestPulsarService.java | 7 ++- .../buffer/TransactionBufferClientTest.java | 22 ++++++++-- .../TransactionBufferHandlerImplTest.java | 16 +++++-- .../client/impl/ConnectionPoolTest.java | 43 +++++++++++++++++-- .../pulsar/client/impl/PulsarTestClient.java | 2 +- .../pulsar/compaction/CompactorTest.java | 8 +++- .../pulsar/client/impl/ConnectionHandler.java | 6 ++- .../pulsar/client/impl/ConnectionPool.java | 16 +++++-- .../pulsar/client/impl/PulsarClientImpl.java | 19 ++++++-- .../AcknowledgementsGroupingTrackerTest.java | 2 + .../client/impl/AutoClusterFailoverTest.java | 9 ++++ .../impl/BatchMessageContainerImplTest.java | 4 ++ .../client/impl/ClientTestFixtures.java | 11 ++++- .../client/impl/ConsumerBuilderImplTest.java | 4 ++ .../impl/ControlledClusterFailoverTest.java | 3 ++ .../impl/PartitionedProducerImplTest.java | 2 + .../client/impl/ProducerBuilderImplTest.java | 2 + .../impl/ProducerStatsRecorderImplTest.java | 4 ++ .../client/impl/PulsarClientImplTest.java | 3 +- .../client/impl/TableViewBuilderImplTest.java | 2 + .../pulsar/client/impl/TableViewImplTest.java | 2 + .../client/impl/TopicListWatcherTest.java | 10 ++++- .../impl/UnAckedMessageTrackerTest.java | 4 ++ .../MultiVersionSchemaInfoProviderTest.java | 3 ++ .../functions/instance/ContextImplTest.java | 3 ++ .../pulsar/functions/sink/PulsarSinkTest.java | 4 ++ .../functions/source/PulsarSourceTest.java | 5 +++ .../functions/worker/LeaderServiceTest.java | 4 +- .../worker/MembershipManagerTest.java | 4 +- 32 files changed, 207 insertions(+), 32 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferHandlerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferHandlerImpl.java index 48dcf259edb1b..625d27329d329 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferHandlerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferHandlerImpl.java @@ -61,6 +61,8 @@ public class TransactionBufferHandlerImpl implements TransactionBufferHandler { private final PulsarService pulsarService; private final PulsarClientImpl pulsarClient; + private final int randomKeyForSelectConnection; + private static final AtomicIntegerFieldUpdater REQUEST_CREDITS_UPDATER = AtomicIntegerFieldUpdater.newUpdater(TransactionBufferHandlerImpl.class, "requestCredits"); private volatile int requestCredits; @@ -74,6 +76,7 @@ public TransactionBufferHandlerImpl(PulsarService pulsarService, HashedWheelTime this.operationTimeoutInMills = operationTimeoutInMills; this.timer = timer; this.requestCredits = Math.max(100, maxConcurrentRequests); + this.randomKeyForSelectConnection = pulsarClient.getCnxPool().genRandomKeyToSelectCon(); } @Override @@ -296,7 +299,7 @@ protected OpRequestSend newObject(Handle handle) { } public CompletableFuture getClientCnxWithLookup(String topic) { - return pulsarClient.getConnection(topic); + return pulsarClient.getConnection(topic, randomKeyForSelectConnection); } public CompletableFuture getClientCnx(String topic) { @@ -317,7 +320,8 @@ public CompletableFuture getClientCnx(String topic) { } InetSocketAddress brokerAddress = InetSocketAddress.createUnresolved(uri.getHost(), uri.getPort()); - return pulsarClient.getConnection(brokerAddress, brokerAddress); + return pulsarClient.getConnection(brokerAddress, brokerAddress, + randomKeyForSelectConnection); } else { // Bundle is unloading, lookup topic return getClientCnxWithLookup(topic); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java index 294a9b341ec69..f8034c37971cc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java @@ -40,6 +40,7 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; @@ -62,8 +63,11 @@ public void testRetryStartProducerStoppedByTopicRemove() throws Exception { final PulsarService pulsar = mock(PulsarService.class); final BrokerService broker = mock(BrokerService.class); final Topic localTopic = mock(Topic.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); final PulsarClientImpl localClient = mock(PulsarClientImpl.class); + when(localClient.getCnxPool()).thenReturn(connectionPool); final PulsarClientImpl remoteClient = mock(PulsarClientImpl.class); + when(remoteClient.getCnxPool()).thenReturn(connectionPool); final ProducerBuilder producerBuilder = mock(ProducerBuilder.class); final ConcurrentOpenHashMap>> topics = new ConcurrentOpenHashMap<>(); when(broker.executor()).thenReturn(eventLoopGroup); 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 078208f7e449c..71310fef8102a 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 @@ -105,6 +105,7 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.transaction.TxnID; +import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.ProducerBuilderImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; @@ -1713,6 +1714,8 @@ public void testAtomicReplicationRemoval() throws Exception { ManagedCursor cursor = mock(ManagedCursorImpl.class); doReturn(remoteCluster).when(cursor).getName(); PulsarClientImpl pulsarClientMock = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(pulsarClientMock.getCnxPool()).thenReturn(connectionPool); when(pulsarClientMock.newProducer(any())).thenAnswer( invocation -> { ProducerBuilderImpl producerBuilder = 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 2896f338e4a1f..2027bb33bf18e 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 @@ -20,6 +20,7 @@ import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgs; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import io.netty.channel.EventLoopGroup; import java.io.IOException; import java.util.Collections; @@ -42,6 +43,7 @@ import org.apache.pulsar.broker.transaction.buffer.TransactionBufferProvider; import org.apache.pulsar.broker.transaction.pendingack.TransactionPendingAckStoreProvider; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.common.naming.TopicName; @@ -77,7 +79,10 @@ public NonStartableTestPulsarService(SpyConfig spyConfig, ServiceConfiguration c throw new RuntimeException(e); } setSchemaRegistryService(spyWithClassAndConstructorArgs(DefaultSchemaRegistryService.class)); - setClient(mock(PulsarClientImpl.class)); + PulsarClientImpl mockClient = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(mockClient.getCnxPool()).thenReturn(connectionPool); + setClient(mockClient); this.namespaceService = mock(NamespaceService.class); try { startNamespaceService(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java index 2cfc9f46f0ebe..3873d9d37b20b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker.transaction.buffer; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyString; import com.google.common.collect.Multimap; import com.google.common.collect.Sets; @@ -51,6 +53,7 @@ import org.apache.pulsar.client.api.transaction.TransactionBufferClientException; import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.impl.ClientCnx; +import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.api.proto.TxnAction; import org.apache.pulsar.common.naming.NamespaceName; @@ -253,14 +256,21 @@ public void testTransactionBufferMetrics() throws Exception { assertEquals(pending.size(), 1); } + /** + * This is a flaky test. + */ @Test public void testTransactionBufferClientTimeout() throws Exception { PulsarService pulsarService = pulsarServiceList.get(0); - PulsarClient mockClient = mock(PulsarClientImpl.class); + PulsarClientImpl mockClient = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(mockClient.getCnxPool()).thenReturn(connectionPool); CompletableFuture completableFuture = new CompletableFuture<>(); ClientCnx clientCnx = mock(ClientCnx.class); completableFuture.complete(clientCnx); when(((PulsarClientImpl)mockClient).getConnection(anyString())).thenReturn(completableFuture); + when(((PulsarClientImpl)mockClient).getConnection(anyString(), anyInt())).thenReturn(completableFuture); + when(((PulsarClientImpl)mockClient).getConnection(any(), any(), anyInt())).thenReturn(completableFuture); ChannelHandlerContext cnx = mock(ChannelHandlerContext.class); when(clientCnx.ctx()).thenReturn(cnx); Channel channel = mock(Channel.class); @@ -287,7 +297,9 @@ public PulsarClient answer(InvocationOnMock invocation) throws Throwable { ConcurrentSkipListMap outstandingRequests = (ConcurrentSkipListMap) field.get(transactionBufferHandler); - assertEquals(outstandingRequests.size(), 1); + Awaitility.await().atMost(1, TimeUnit.SECONDS).untilAsserted(() -> { + assertEquals(outstandingRequests.size(), 1); + }); Awaitility.await().atLeast(2, TimeUnit.SECONDS).until(() -> { if (outstandingRequests.size() == 0) { @@ -307,11 +319,13 @@ public PulsarClient answer(InvocationOnMock invocation) throws Throwable { @Test public void testTransactionBufferChannelUnActive() throws PulsarServerException { PulsarService pulsarService = pulsarServiceList.get(0); - PulsarClient mockClient = mock(PulsarClientImpl.class); + PulsarClientImpl mockClient = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(mockClient.getCnxPool()).thenReturn(connectionPool); CompletableFuture completableFuture = new CompletableFuture<>(); ClientCnx clientCnx = mock(ClientCnx.class); completableFuture.complete(clientCnx); - when(((PulsarClientImpl)mockClient).getConnection(anyString())).thenReturn(completableFuture); + when(((PulsarClientImpl)mockClient).getConnection(anyString(), anyInt())).thenReturn(completableFuture); ChannelHandlerContext cnx = mock(ChannelHandlerContext.class); when(clientCnx.ctx()).thenReturn(cnx); Channel channel = mock(Channel.class); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferHandlerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferHandlerImplTest.java index d6ec092c4456f..278cdbac1f09d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferHandlerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferHandlerImplTest.java @@ -24,6 +24,7 @@ import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferHandlerImpl; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doReturn; @@ -31,8 +32,8 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; -import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.impl.ClientCnx; +import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.api.proto.TxnAction; import org.apache.pulsar.common.naming.NamespaceBundle; @@ -46,7 +47,9 @@ public class TransactionBufferHandlerImplTest { @Test public void testRequestCredits() throws PulsarServerException { - PulsarClient pulsarClient = mock(PulsarClientImpl.class); + PulsarClientImpl pulsarClient = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(pulsarClient.getCnxPool()).thenReturn(connectionPool); PulsarService pulsarService = mock(PulsarService.class); NamespaceService namespaceService = mock(NamespaceService.class); when(pulsarService.getNamespaceService()).thenReturn(namespaceService); @@ -54,7 +57,10 @@ public void testRequestCredits() throws PulsarServerException { when(namespaceService.getBundleAsync(any())).thenReturn(CompletableFuture.completedFuture(mock(NamespaceBundle.class))); Optional opData = Optional.empty(); when(namespaceService.getOwnerAsync(any())).thenReturn(CompletableFuture.completedFuture(opData)); - when(((PulsarClientImpl)pulsarClient).getConnection(anyString())).thenReturn(CompletableFuture.completedFuture(mock(ClientCnx.class))); + when(((PulsarClientImpl)pulsarClient).getConnection(anyString(), anyInt())) + .thenReturn(CompletableFuture.completedFuture(mock(ClientCnx.class))); + when(((PulsarClientImpl)pulsarClient).getConnection(anyString())) + .thenReturn(CompletableFuture.completedFuture(mock(ClientCnx.class))); TransactionBufferHandlerImpl handler = spy(new TransactionBufferHandlerImpl(pulsarService, null, 1000, 3000)); doNothing().when(handler).endTxn(any()); doReturn(CompletableFuture.completedFuture(mock(ClientCnx.class))).when(handler).getClientCnx(anyString()); @@ -75,7 +81,9 @@ public void testRequestCredits() throws PulsarServerException { @Test public void testMinRequestCredits() throws PulsarServerException { - PulsarClient pulsarClient = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + PulsarClientImpl pulsarClient = mock(PulsarClientImpl.class); + when(pulsarClient.getCnxPool()).thenReturn(connectionPool); PulsarService pulsarService = mock(PulsarService.class); when(pulsarService.getClient()).thenReturn(pulsarClient); TransactionBufferHandlerImpl handler = spy(new TransactionBufferHandlerImpl(pulsarService, null, 50, 3000)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java index fe0aa4dd4953b..79ffada4a90c8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ConnectionPoolTest.java @@ -31,9 +31,13 @@ import java.util.function.Supplier; import java.util.stream.IntStream; import io.netty.util.concurrent.Promise; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.common.api.proto.CommandCloseProducer; import org.apache.pulsar.common.util.netty.EventLoopUtil; +import org.awaitility.Awaitility; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -80,6 +84,36 @@ public void testSingleIpAddress() throws Exception { eventLoop.shutdownGracefully(); } + @Test + public void testSelectConnectionForSameProducer() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://sample/standalone/ns/tp_"); + admin.topics().createNonPartitionedTopic(topicName); + final CommandCloseProducer commandCloseProducer = new CommandCloseProducer(); + // 10 connection per broker. + final PulsarClient clientWith10ConPerBroker = PulsarClient.builder().connectionsPerBroker(10) + .serviceUrl(lookupUrl.toString()).build(); + ProducerImpl producer = (ProducerImpl) clientWith10ConPerBroker.newProducer().topic(topicName).create(); + commandCloseProducer.setProducerId(producer.producerId); + // An error will be reported when the Producer reconnects using a different connection. + // If no error is reported, the same connection was used when reconnecting. + for (int i = 0; i < 20; i++) { + // Trigger reconnect + ClientCnx cnx = producer.getClientCnx(); + if (cnx != null) { + cnx.handleCloseProducer(commandCloseProducer); + Awaitility.await().untilAsserted(() -> + Assert.assertEquals(producer.getState().toString(), HandlerState.State.Ready.toString(), + "The producer uses a different connection when reconnecting") + ); + } + } + + // cleanup. + producer.close(); + clientWith10ConPerBroker.close(); + admin.topics().delete(topicName); + } + @Test public void testDoubleIpAddress() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); @@ -205,20 +239,23 @@ protected void doResolveAll(SocketAddress socketAddress, Promise promise) throws ClientCnx cnx = pool.getConnection( InetSocketAddress.createUnresolved("proxy", 9999), - InetSocketAddress.createUnresolved("proxy", 9999)).get(); + InetSocketAddress.createUnresolved("proxy", 9999), + pool.genRandomKeyToSelectCon()).get(); Assert.assertEquals(cnx.remoteHostName, "proxy"); Assert.assertNull(cnx.proxyToTargetBrokerAddress); cnx = pool.getConnection( InetSocketAddress.createUnresolved("broker1", 9999), - InetSocketAddress.createUnresolved("proxy", 9999)).get(); + InetSocketAddress.createUnresolved("proxy", 9999), + pool.genRandomKeyToSelectCon()).get(); Assert.assertEquals(cnx.remoteHostName, "proxy"); Assert.assertEquals(cnx.proxyToTargetBrokerAddress, "broker1:9999"); cnx = pool.getConnection( InetSocketAddress.createUnresolved("broker2", 9999), - InetSocketAddress.createUnresolved("broker2", 9999)).get(); + InetSocketAddress.createUnresolved("broker2", 9999), + pool.genRandomKeyToSelectCon()).get(); Assert.assertEquals(cnx.remoteHostName, "broker2"); Assert.assertNull(cnx.proxyToTargetBrokerAddress); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java index a725562ac40aa..6555c152bacad 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PulsarTestClient.java @@ -122,7 +122,7 @@ public CompletableFuture getConnection(String topic) { result.completeExceptionally(new IOException("New connections are rejected.")); return result; } else { - return super.getConnection(topic); + return super.getConnection(topic, getCnxPool().genRandomKeyToSelectCon()); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java index d97d22ae7f107..4e442ac051326 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java @@ -19,6 +19,8 @@ package org.apache.pulsar.compaction; import static org.apache.pulsar.client.impl.RawReaderTest.extractKey; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -48,6 +50,7 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.RawMessage; +import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.RawMessageImpl; import org.apache.pulsar.common.policies.data.ClusterData; @@ -258,7 +261,10 @@ public void testCompactEmptyTopic() throws Exception { public void testPhaseOneLoopTimeConfiguration() { ServiceConfiguration configuration = new ServiceConfiguration(); configuration.setBrokerServiceCompactionPhaseOneLoopTimeInSeconds(60); - TwoPhaseCompactor compactor = new TwoPhaseCompactor(configuration, Mockito.mock(PulsarClientImpl.class), + PulsarClientImpl mockClient = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(mockClient.getCnxPool()).thenReturn(connectionPool); + TwoPhaseCompactor compactor = new TwoPhaseCompactor(configuration, mockClient, Mockito.mock(BookKeeper.class), compactionScheduler); Assert.assertEquals(compactor.getPhaseOneLoopReadTimeoutInSeconds(), 60); } 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 263507dac1dc6..fc7c89c3ce693 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 @@ -43,6 +43,7 @@ public class ConnectionHandler { private volatile long epoch = -1L; protected volatile long lastConnectionClosedTimestamp = 0L; private final AtomicBoolean duringConnect = new AtomicBoolean(false); + protected final int randomKeyForSelectConnection; interface Connection { @@ -58,6 +59,7 @@ default void connectionFailed(PulsarClientException e) { protected ConnectionHandler(HandlerState state, Backoff backoff, Connection connection) { this.state = state; + this.randomKeyForSelectConnection = state.client.getCnxPool().genRandomKeyToSelectCon(); this.connection = connection; this.backoff = backoff; CLIENT_CNX_UPDATER.set(this, null); @@ -88,11 +90,11 @@ protected void grabCnx() { if (state.redirectedClusterURI != null) { InetSocketAddress address = InetSocketAddress.createUnresolved(state.redirectedClusterURI.getHost(), state.redirectedClusterURI.getPort()); - cnxFuture = state.client.getConnection(address, address); + cnxFuture = state.client.getConnection(address, address, randomKeyForSelectConnection); } else if (state.topic == null) { cnxFuture = state.client.getConnectionToServiceUrl(); } else { - cnxFuture = state.client.getConnection(state.topic); // + cnxFuture = state.client.getConnection(state.topic, randomKeyForSelectConnection); } cnxFuture.thenCompose(cnx -> connection.connectionOpened(cnx)) .thenAccept(__ -> duringConnect.set(false)) 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 1420d81c688ee..ef3a9249c820a 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 @@ -165,8 +165,18 @@ private static AddressResolver createAddressResolver(ClientCo private static final Random random = new Random(); + public int genRandomKeyToSelectCon() { + if (maxConnectionsPerHosts == 0) { + return -1; + } + return signSafeMod(random.nextInt(), maxConnectionsPerHosts); + } + public CompletableFuture getConnection(final InetSocketAddress address) { - return getConnection(address, address); + if (maxConnectionsPerHosts == 0) { + return getConnection(address, address, -1); + } + return getConnection(address, address, signSafeMod(random.nextInt(), maxConnectionsPerHosts)); } void closeAllConnections() { @@ -204,14 +214,12 @@ void closeAllConnections() { * @return a future that will produce the ClientCnx object */ public CompletableFuture getConnection(InetSocketAddress logicalAddress, - InetSocketAddress physicalAddress) { + InetSocketAddress physicalAddress, final int randomKey) { if (maxConnectionsPerHosts == 0) { // Disable pooling return createConnection(logicalAddress, physicalAddress, -1); } - final int randomKey = signSafeMod(random.nextInt(), maxConnectionsPerHosts); - final ConcurrentMap> innerPool = pool.computeIfAbsent(logicalAddress, a -> new ConcurrentHashMap<>()); CompletableFuture completableFuture = innerPool 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 6c749a8cf4354..fdabb5fa8cfa5 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 @@ -944,10 +944,20 @@ public void updateTlsTrustStorePathAndPassword(String tlsTrustStorePath, String conf.setTlsTrustStorePassword(tlsTrustStorePassword); } + public CompletableFuture getConnection(final String topic, int randomKeyForSelectConnection) { + TopicName topicName = TopicName.get(topic); + return lookup.getBroker(topicName) + .thenCompose(pair -> getConnection(pair.getLeft(), pair.getRight(), randomKeyForSelectConnection)); + } + + /** + * Only for test. + */ + @VisibleForTesting public CompletableFuture getConnection(final String topic) { TopicName topicName = TopicName.get(topic); return lookup.getBroker(topicName) - .thenCompose(pair -> getConnection(pair.getLeft(), pair.getRight())); + .thenCompose(pair -> getConnection(pair.getLeft(), pair.getRight(), cnxPool.genRandomKeyToSelectCon())); } public CompletableFuture getConnectionToServiceUrl() { @@ -956,12 +966,13 @@ public CompletableFuture getConnectionToServiceUrl() { "Can't get client connection to HTTP service URL", null)); } InetSocketAddress address = lookup.resolveHost(); - return getConnection(address, address); + return getConnection(address, address, cnxPool.genRandomKeyToSelectCon()); } public CompletableFuture getConnection(final InetSocketAddress logicalAddress, - final InetSocketAddress physicalAddress) { - return cnxPool.getConnection(logicalAddress, physicalAddress); + final InetSocketAddress physicalAddress, + final int randomKeyForSelectConnection) { + return cnxPool.getConnection(logicalAddress, physicalAddress, randomKeyForSelectConnection); } /** visible for pulsar-functions. **/ 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 0418a54c772cc..1d1a6f85bfd41 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 @@ -65,6 +65,8 @@ public void setup() throws NoSuchFieldException, IllegalAccessException { ConcurrentOpenHashMap.newBuilder().build(); cnx = spy(new ClientCnxTest(new ClientConfigurationData(), eventLoopGroup)); PulsarClientImpl client = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(client.getCnxPool()).thenReturn(connectionPool); doReturn(client).when(consumer).getClient(); doReturn(cnx).when(consumer).getClientCnx(); doReturn(new ConsumerStatsRecorderImpl()).when(consumer).getStats(); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java index 36ffa30296bb0..63fbb239439bd 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.client.impl; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; @@ -121,6 +122,8 @@ public void testInitialize() { AutoClusterFailover autoClusterFailover = Mockito.spy((AutoClusterFailover) provider); PulsarClientImpl pulsarClient = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(pulsarClient.getCnxPool()).thenReturn(connectionPool); Mockito.doReturn(false).when(autoClusterFailover).probeAvailable(primary); Mockito.doReturn(true).when(autoClusterFailover).probeAvailable(secondary); Mockito.doReturn(configurationData).when(pulsarClient).getConfiguration(); @@ -163,6 +166,8 @@ public void testAutoClusterFailoverSwitchWithoutAuthentication() { AutoClusterFailover autoClusterFailover = Mockito.spy((AutoClusterFailover) provider); PulsarClientImpl pulsarClient = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(pulsarClient.getCnxPool()).thenReturn(connectionPool); Mockito.doReturn(false).when(autoClusterFailover).probeAvailable(primary); Mockito.doReturn(true).when(autoClusterFailover).probeAvailable(secondary); Mockito.doReturn(configurationData).when(pulsarClient).getConfiguration(); @@ -217,6 +222,8 @@ public void testAutoClusterFailoverSwitchWithAuthentication() throws IOException AutoClusterFailover autoClusterFailover = Mockito.spy((AutoClusterFailover) provider); PulsarClientImpl pulsarClient = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(pulsarClient.getCnxPool()).thenReturn(connectionPool); Mockito.doReturn(false).when(autoClusterFailover).probeAvailable(primary); Mockito.doReturn(true).when(autoClusterFailover).probeAvailable(secondary); Mockito.doReturn(configurationData).when(pulsarClient).getConfiguration(); @@ -270,6 +277,8 @@ public void testAutoClusterFailoverSwitchTlsTrustStore() throws IOException { AutoClusterFailover autoClusterFailover = Mockito.spy((AutoClusterFailover) provider); PulsarClientImpl pulsarClient = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(pulsarClient.getCnxPool()).thenReturn(connectionPool); Mockito.doReturn(false).when(autoClusterFailover).probeAvailable(primary); Mockito.doReturn(true).when(autoClusterFailover).probeAvailable(secondary); Mockito.doReturn(configurationData).when(pulsarClient).getConfiguration(); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BatchMessageContainerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BatchMessageContainerImplTest.java index 4b80e19c256d7..abb195c9830d0 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BatchMessageContainerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BatchMessageContainerImplTest.java @@ -105,6 +105,8 @@ public void recoveryAfterOom() { final ProducerConfigurationData producerConfigurationData = new ProducerConfigurationData(); producerConfigurationData.setCompressionType(CompressionType.NONE); PulsarClientImpl pulsarClient = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(pulsarClient.getCnxPool()).thenReturn(connectionPool); MemoryLimitController memoryLimitController = mock(MemoryLimitController.class); when(pulsarClient.getMemoryLimitController()).thenReturn(memoryLimitController); try { @@ -148,6 +150,8 @@ public void testMessagesSize() throws Exception { final ProducerConfigurationData producerConfigurationData = new ProducerConfigurationData(); producerConfigurationData.setCompressionType(CompressionType.NONE); PulsarClientImpl pulsarClient = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(pulsarClient.getCnxPool()).thenReturn(connectionPool); MemoryLimitController memoryLimitController = mock(MemoryLimitController.class); when(pulsarClient.getMemoryLimitController()).thenReturn(memoryLimitController); try { diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientTestFixtures.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientTestFixtures.java index 4db5dbe877685..ff7d7f12dd452 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientTestFixtures.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientTestFixtures.java @@ -19,7 +19,9 @@ package org.apache.pulsar.client.impl; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import io.netty.channel.ChannelHandlerContext; @@ -72,11 +74,16 @@ static PulsarClientImpl mockClientCnx(PulsarClientImpl clientMock) { .thenReturn(CompletableFuture.completedFuture(mock(ProducerResponse.class))); when(clientCnxMock.channel().remoteAddress()).thenReturn(mock(SocketAddress.class)); when(clientMock.getConnection(any())).thenReturn(CompletableFuture.completedFuture(clientCnxMock)); - when(clientMock.getConnection(any(), any())).thenReturn(CompletableFuture.completedFuture(clientCnxMock)); + when(clientMock.getConnection(anyString())).thenReturn(CompletableFuture.completedFuture(clientCnxMock)); + when(clientMock.getConnection(anyString(), anyInt())) + .thenReturn(CompletableFuture.completedFuture(clientCnxMock)); + when(clientMock.getConnection(any(), any(), anyInt())) + .thenReturn(CompletableFuture.completedFuture(clientCnxMock)); ConnectionPool connectionPoolMock = mock(ConnectionPool.class); when(clientMock.getCnxPool()).thenReturn(connectionPoolMock); when(connectionPoolMock.getConnection(any())).thenReturn(CompletableFuture.completedFuture(clientCnxMock)); - when(connectionPoolMock.getConnection(any(), any())).thenReturn(CompletableFuture.completedFuture(clientCnxMock)); + when(connectionPoolMock.getConnection(any(), any(), anyInt())) + .thenReturn(CompletableFuture.completedFuture(clientCnxMock)); return clientMock; } 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 8dbd23f9c29c9..3fe136630462f 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 @@ -76,6 +76,8 @@ public class ConsumerBuilderImplTest { @BeforeMethod(alwaysRun = true) public void setup() { PulsarClientImpl client = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(client.getCnxPool()).thenReturn(connectionPool); ConsumerConfigurationData consumerConfigurationData = mock(ConsumerConfigurationData.class); when(consumerConfigurationData.getTopicsPattern()).thenReturn(Pattern.compile("\\w+")); when(consumerConfigurationData.getSubscriptionName()).thenReturn("testSubscriptionName"); @@ -104,6 +106,8 @@ public void testConsumerBuilderImpl() throws PulsarClientException { @Test(expectedExceptions = IllegalArgumentException.class) public void testConsumerBuilderImplWhenSchemaIsNull() { PulsarClientImpl client = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(client.getCnxPool()).thenReturn(connectionPool); ConsumerConfigurationData consumerConfigurationData = mock(ConsumerConfigurationData.class); new ConsumerBuilderImpl(client, consumerConfigurationData, null); } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java index 570b139832806..227e0db10b724 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java @@ -31,6 +31,7 @@ import org.testng.annotations.Test; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; @Test(groups = "broker-impl") public class ControlledClusterFailoverTest { @@ -88,6 +89,8 @@ public void testControlledClusterFailoverSwitch() throws IOException { ControlledClusterFailover controlledClusterFailover = Mockito.spy((ControlledClusterFailover) provider); PulsarClientImpl pulsarClient = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(pulsarClient.getCnxPool()).thenReturn(connectionPool); controlledClusterFailover.initialize(pulsarClient); 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 223881d85a87b..2bd18f69386f1 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 @@ -70,6 +70,8 @@ public class PartitionedProducerImplTest { @BeforeMethod(alwaysRun = true) public void setup() { client = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(client.getCnxPool()).thenReturn(connectionPool); schema = mock(Schema.class); producerInterceptors = mock(ProducerInterceptors.class); producerCreatedFuture = new CompletableFuture<>(); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerBuilderImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerBuilderImplTest.java index bb3e3fc3accf6..b830d375303bb 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerBuilderImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerBuilderImplTest.java @@ -52,6 +52,8 @@ public class ProducerBuilderImplTest { public void setup() { Producer producer = mock(Producer.class); client = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(client.getCnxPool()).thenReturn(connectionPool); producerBuilderImpl = new ProducerBuilderImpl<>(client, Schema.BYTES); when(client.newProducer()).thenReturn(producerBuilderImpl); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerStatsRecorderImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerStatsRecorderImplTest.java index 32d0eff6e792e..27e2dcb37cee0 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerStatsRecorderImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ProducerStatsRecorderImplTest.java @@ -40,6 +40,8 @@ public void testIncrementNumAcksReceived() throws Exception { ClientConfigurationData conf = new ClientConfigurationData(); conf.setStatsIntervalSeconds(1); PulsarClientImpl client = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(client.getCnxPool()).thenReturn(connectionPool); when(client.getConfiguration()).thenReturn(conf); Timer timer = new HashedWheelTimer(); when(client.timer()).thenReturn(timer); @@ -60,6 +62,8 @@ public void testGetStatsAndCancelStatsTimeoutWithoutArriveUpdateInterval() { ClientConfigurationData conf = new ClientConfigurationData(); conf.setStatsIntervalSeconds(60); PulsarClientImpl client = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(client.getCnxPool()).thenReturn(connectionPool); when(client.getConfiguration()).thenReturn(conf); Timer timer = new HashedWheelTimer(); when(client.timer()).thenReturn(timer); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java index 54d13538d7867..e0b25db891247 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.client.impl; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.nullable; import static org.mockito.Mockito.mock; @@ -122,7 +123,7 @@ public void testConsumerIsClosed() throws Exception { when(cnx.ctx()).thenReturn(ctx); when(cnx.sendRequestWithId(any(ByteBuf.class), anyLong())) .thenReturn(CompletableFuture.completedFuture(mock(ProducerResponse.class))); - when(pool.getConnection(any(InetSocketAddress.class), any(InetSocketAddress.class))) + when(pool.getConnection(any(InetSocketAddress.class), any(InetSocketAddress.class), anyInt())) .thenReturn(CompletableFuture.completedFuture(cnx)); ClientConfigurationData conf = new ClientConfigurationData(); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TableViewBuilderImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TableViewBuilderImplTest.java index 9959a2038555c..eee8ba4e8f41a 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TableViewBuilderImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TableViewBuilderImplTest.java @@ -52,6 +52,8 @@ public void setup() { Reader reader = mock(Reader.class); when(reader.readNextAsync()).thenReturn(CompletableFuture.allOf()); client = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(client.getCnxPool()).thenReturn(connectionPool); when(client.newReader(any(Schema.class))) .thenReturn(new ReaderBuilderImpl(client, Schema.BYTES)); when(client.createReaderAsync(any(ReaderConfigurationData.class), any(Schema.class))) diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TableViewImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TableViewImplTest.java index 68c886bc7211a..6a866034ddbf8 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TableViewImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TableViewImplTest.java @@ -38,6 +38,8 @@ public class TableViewImplTest { @BeforeClass(alwaysRun = true) public void setup() { client = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(client.getCnxPool()).thenReturn(connectionPool); when(client.newReader(any(Schema.class))) .thenReturn(new ReaderBuilderImpl(client, Schema.BYTES)); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java index e462bb4d62cb5..1b39448fbe770 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java @@ -28,7 +28,9 @@ import org.apache.pulsar.common.api.proto.CommandWatchTopicUpdate; import org.apache.pulsar.common.naming.NamespaceName; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -52,6 +54,9 @@ public class TopicListWatcherTest { public void setup() { listener = mock(TopicsChangedListener.class); client = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(client.getCnxPool()).thenReturn(connectionPool); + when(connectionPool.genRandomKeyToSelectCon()).thenReturn(0); when(client.getConfiguration()).thenReturn(new ClientConfigurationData()); clientCnxFuture = new CompletableFuture<>(); when(client.getConnectionToServiceUrl()).thenReturn(clientCnxFuture); @@ -59,6 +64,9 @@ public void setup() { when(client.timer()).thenReturn(timer); String topic = "persistent://tenant/ns/topic\\d+"; when(client.getConnection(topic)).thenReturn(clientCnxFuture); + when(client.getConnection(topic, 0)).thenReturn(clientCnxFuture); + when(client.getConnection(any(), any(), anyInt())).thenReturn(clientCnxFuture); + when(connectionPool.getConnection(any(), any(), anyInt())).thenReturn(clientCnxFuture); watcherFuture = new CompletableFuture<>(); watcher = new TopicListWatcher(listener, client, Pattern.compile(topic), 7, @@ -67,7 +75,7 @@ public void setup() { @Test public void testWatcherGrabsConnection() { - verify(client).getConnection(any()); + verify(client).getConnection(anyString(), anyInt()); } @Test 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 f6c668703d9db..91ad321048226 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 @@ -45,6 +45,8 @@ public class UnAckedMessageTrackerTest { @Test public void testAddAndRemove() { PulsarClientImpl client = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(client.getCnxPool()).thenReturn(connectionPool); Timer timer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-timer", Thread.currentThread().isDaemon()), 1, TimeUnit.MILLISECONDS); when(client.timer()).thenReturn(timer); @@ -83,6 +85,8 @@ public void testAddAndRemove() { @Test public void testTrackChunkedMessageId() { PulsarClientImpl client = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(client.getCnxPool()).thenReturn(connectionPool); Timer timer = new HashedWheelTimer(new DefaultThreadFactory("pulsar-timer", Thread.currentThread().isDaemon()), 1, TimeUnit.MILLISECONDS); when(client.timer()).thenReturn(timer); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/generic/MultiVersionSchemaInfoProviderTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/generic/MultiVersionSchemaInfoProviderTest.java index 8959e67023463..bfd6af37e3ea6 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/generic/MultiVersionSchemaInfoProviderTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/generic/MultiVersionSchemaInfoProviderTest.java @@ -27,6 +27,7 @@ import java.util.concurrent.CompletableFuture; import org.apache.pulsar.client.api.schema.SchemaDefinition; +import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.LookupService; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.schema.AvroSchema; @@ -46,6 +47,8 @@ public class MultiVersionSchemaInfoProviderTest { @BeforeMethod public void setup() { PulsarClientImpl client = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(client.getCnxPool()).thenReturn(connectionPool); when(client.getLookup()).thenReturn(mock(LookupService.class)); schemaProvider = new MultiVersionSchemaInfoProvider( TopicName.get("persistent://public/default/my-topic"), client); diff --git a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java index e0ebb52da7490..90f7df37fa196 100644 --- a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java +++ b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java @@ -47,6 +47,7 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.TypedMessageBuilder; +import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.ConsumerImpl; import org.apache.pulsar.client.impl.MultiTopicsConsumerImpl; import org.apache.pulsar.client.impl.ProducerBase; @@ -99,6 +100,8 @@ public void setup() throws PulsarClientException { producer = mock(Producer.class); client = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(client.getCnxPool()).thenReturn(connectionPool); when(client.newProducer()).thenReturn(new ProducerBuilderImpl(client, Schema.BYTES)); when(client.createProducerAsync(any(ProducerConfigurationData.class), any(), any())) .thenReturn(CompletableFuture.completedFuture(producer)); diff --git a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/sink/PulsarSinkTest.java b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/sink/PulsarSinkTest.java index fdac39512cc24..799bad839a451 100644 --- a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/sink/PulsarSinkTest.java +++ b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/sink/PulsarSinkTest.java @@ -29,6 +29,7 @@ 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.assertFalse; import static org.testng.Assert.assertNull; @@ -56,6 +57,7 @@ import org.apache.pulsar.client.api.schema.GenericSchema; import org.apache.pulsar.client.api.schema.RecordSchemaBuilder; import org.apache.pulsar.client.api.schema.SchemaBuilder; +import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; import org.apache.pulsar.common.functions.FunctionConfig; @@ -95,6 +97,8 @@ public byte[] serialize(String input) { */ private static PulsarClientImpl getPulsarClient() throws PulsarClientException { PulsarClientImpl pulsarClient = mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(pulsarClient.getCnxPool()).thenReturn(connectionPool); ConsumerBuilder consumerBuilder = mock(ConsumerBuilder.class); doReturn(consumerBuilder).when(consumerBuilder).topics(anyList()); doReturn(consumerBuilder).when(consumerBuilder).subscriptionName(anyString()); diff --git a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/source/PulsarSourceTest.java b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/source/PulsarSourceTest.java index 91e4c06fe5b49..5d6e4a3dc75e7 100644 --- a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/source/PulsarSourceTest.java +++ b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/source/PulsarSourceTest.java @@ -21,6 +21,8 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import static org.testng.Assert.assertSame; import static org.testng.AssertJUnit.assertEquals; import static org.testng.AssertJUnit.assertTrue; @@ -44,6 +46,7 @@ import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.schema.GenericRecord; +import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.functions.ConsumerConfig; @@ -105,6 +108,8 @@ public static Object[] getPulsarSourceImpls() { */ private static PulsarClientImpl getPulsarClient() throws PulsarClientException { PulsarClientImpl pulsarClient = Mockito.mock(PulsarClientImpl.class); + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(pulsarClient.getCnxPool()).thenReturn(connectionPool); ConsumerBuilder goodConsumerBuilder = Mockito.mock(ConsumerBuilder.class); ConsumerBuilder badConsumerBuilder = Mockito.mock(ConsumerBuilder.class); Mockito.doReturn(goodConsumerBuilder).when(goodConsumerBuilder) diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/LeaderServiceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/LeaderServiceTest.java index 8da24fd1b7250..5c10a59bd1388 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/LeaderServiceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/LeaderServiceTest.java @@ -36,6 +36,7 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.ConsumerImpl; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; @@ -76,7 +77,8 @@ public LeaderServiceTest() { @BeforeMethod public void setup() throws PulsarClientException { mockClient = mock(PulsarClientImpl.class); - + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(mockClient.getCnxPool()).thenReturn(connectionPool); mockConsumer = mock(ConsumerImpl.class); ConsumerBuilder mockConsumerBuilder = mock(ConsumerBuilder.class); diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/MembershipManagerTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/MembershipManagerTest.java index e066bb24e6ef0..ac3176b3135e2 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/MembershipManagerTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/MembershipManagerTest.java @@ -41,6 +41,7 @@ import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.ReaderBuilder; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.ConsumerImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.functions.WorkerInfo; @@ -68,7 +69,8 @@ public MembershipManagerTest() { private static PulsarClient mockPulsarClient() throws PulsarClientException { PulsarClientImpl mockClient = mock(PulsarClientImpl.class); - + ConnectionPool connectionPool = mock(ConnectionPool.class); + when(mockClient.getCnxPool()).thenReturn(connectionPool); ConsumerImpl mockConsumer = mock(ConsumerImpl.class); ConsumerBuilder mockConsumerBuilder = mock(ConsumerBuilder.class); From 40740c4e61f2a304e7a900a764ddf1dfe36d9c88 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Thu, 14 Sep 2023 07:05:54 -0500 Subject: [PATCH 055/699] [fix][ci] Enable CI for branch-3.1 (#21184) --- .github/workflows/pulsar-ci.yaml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 64b85cb14c580..32884c67d5447 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -22,6 +22,7 @@ on: pull_request: branches: - master + - branch-3.1 schedule: - cron: '0 12 * * *' workflow_dispatch: From 5da5b1481825a0e93bb0e122c63b4511184bf075 Mon Sep 17 00:00:00 2001 From: AloysZhang Date: Fri, 15 Sep 2023 09:04:21 +0800 Subject: [PATCH 056/699] [fix][broker] Backport fix UniformLoadShedder selecet wrong overloadbroker and underloadbroker (#21180) --- .../loadbalance/impl/UniformLoadShedder.java | 163 +++++++++++------- .../impl/UniformLoadShedderTest.java | 78 +++++++++ 2 files changed, 178 insertions(+), 63 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 b92af5b7c69f3..d8dcfa007cfc5 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 @@ -25,7 +25,7 @@ import org.apache.commons.lang3.mutable.MutableDouble; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.commons.lang3.mutable.MutableObject; -import org.apache.commons.lang3.tuple.Triple; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.loadbalance.LoadData; import org.apache.pulsar.broker.loadbalance.LoadSheddingStrategy; @@ -36,7 +36,7 @@ /** * This strategy tends to distribute load uniformly across all brokers. This strategy checks load difference between - * broker with highest load and broker with lowest load. If the difference is higher than configured thresholds + * broker with the highest load and broker with the lowest load. If the difference is higher than configured thresholds * {@link ServiceConfiguration#getLoadBalancerMsgRateDifferenceShedderThreshold()} or * {@link ServiceConfiguration#getLoadBalancerMsgThroughputMultiplierDifferenceShedderThreshold()} then it finds out * bundles which can be unloaded to distribute traffic evenly across all brokers. @@ -63,25 +63,37 @@ public Multimap findBundlesForUnloading(final LoadData loadData, Map loadBundleData = loadData.getBundleDataForLoadShedding(); Map recentlyUnloadedBundles = loadData.getRecentlyUnloadedBundles(); - MutableObject overloadedBroker = new MutableObject<>(); - MutableObject underloadedBroker = new MutableObject<>(); + MutableObject msgRateOverloadedBroker = new MutableObject<>(); + MutableObject msgThroughputOverloadedBroker = new MutableObject<>(); + MutableObject msgRateUnderloadedBroker = new MutableObject<>(); + MutableObject msgThroughputUnderloadedBroker = new MutableObject<>(); MutableDouble maxMsgRate = new MutableDouble(-1); - MutableDouble maxThroughputRate = new MutableDouble(-1); + MutableDouble maxThroughput = new MutableDouble(-1); MutableDouble minMsgRate = new MutableDouble(Integer.MAX_VALUE); - MutableDouble minThroughputRate = new MutableDouble(Integer.MAX_VALUE); + MutableDouble minThroughput = new MutableDouble(Integer.MAX_VALUE); + brokersData.forEach((broker, data) -> { double msgRate = data.getLocalData().getMsgRateIn() + data.getLocalData().getMsgRateOut(); double throughputRate = data.getLocalData().getMsgThroughputIn() + data.getLocalData().getMsgThroughputOut(); - if (msgRate > maxMsgRate.getValue() || throughputRate > maxThroughputRate.getValue()) { - overloadedBroker.setValue(broker); + if (msgRate > maxMsgRate.getValue()) { + msgRateOverloadedBroker.setValue(broker); maxMsgRate.setValue(msgRate); - maxThroughputRate.setValue(throughputRate); } - if (msgRate < minMsgRate.getValue() || throughputRate < minThroughputRate.getValue()) { - underloadedBroker.setValue(broker); + + if (throughputRate > maxThroughput.getValue()) { + msgThroughputOverloadedBroker.setValue(broker); + maxThroughput.setValue(throughputRate); + } + + if (msgRate < minMsgRate.getValue()) { + msgRateUnderloadedBroker.setValue(broker); minMsgRate.setValue(msgRate); - minThroughputRate.setValue(throughputRate); + } + + if (throughputRate < minThroughput.getValue()) { + msgThroughputUnderloadedBroker.setValue(broker); + minThroughput.setValue(throughputRate); } }); @@ -91,12 +103,12 @@ public Multimap findBundlesForUnloading(final LoadData loadData, if (minMsgRate.getValue() <= EPS && minMsgRate.getValue() >= -EPS) { minMsgRate.setValue(1.0); } - if (minThroughputRate.getValue() <= EPS && minThroughputRate.getValue() >= -EPS) { - minThroughputRate.setValue(1.0); + if (minThroughput.getValue() <= EPS && minThroughput.getValue() >= -EPS) { + minThroughput.setValue(1.0); } double msgRateDifferencePercentage = ((maxMsgRate.getValue() - minMsgRate.getValue()) * 100) / (minMsgRate.getValue()); - double msgThroughputDifferenceRate = maxThroughputRate.getValue() / minThroughputRate.getValue(); + double msgThroughputDifferenceRate = maxThroughput.getValue() / minThroughput.getValue(); // if the threshold matches then find out how much load needs to be unloaded by considering number of msgRate // and throughput. @@ -105,66 +117,91 @@ public Multimap findBundlesForUnloading(final LoadData loadData, boolean isMsgThroughputThresholdExceeded = conf .getLoadBalancerMsgThroughputMultiplierDifferenceShedderThreshold() > 0 && msgThroughputDifferenceRate > conf - .getLoadBalancerMsgThroughputMultiplierDifferenceShedderThreshold(); + .getLoadBalancerMsgThroughputMultiplierDifferenceShedderThreshold(); if (isMsgRateThresholdExceeded || isMsgThroughputThresholdExceeded) { - if (log.isDebugEnabled()) { - log.debug( - "Found bundles for uniform load balancing. " - + "overloaded broker {} with (msgRate,throughput)= ({},{}) " - + "and underloaded broker {} with (msgRate,throughput)= ({},{})", - overloadedBroker.getValue(), maxMsgRate.getValue(), maxThroughputRate.getValue(), - underloadedBroker.getValue(), minMsgRate.getValue(), minThroughputRate.getValue()); - } MutableInt msgRateRequiredFromUnloadedBundles = new MutableInt( (int) ((maxMsgRate.getValue() - minMsgRate.getValue()) * conf.getMaxUnloadPercentage())); MutableInt msgThroughputRequiredFromUnloadedBundles = new MutableInt( - (int) ((maxThroughputRate.getValue() - minThroughputRate.getValue()) + (int) ((maxThroughput.getValue() - minThroughput.getValue()) * conf.getMaxUnloadPercentage())); - LocalBrokerData overloadedBrokerData = brokersData.get(overloadedBroker.getValue()).getLocalData(); - - if (overloadedBrokerData.getBundles().size() > 1 - && (msgRateRequiredFromUnloadedBundles.getValue() >= conf.getMinUnloadMessage() - || msgThroughputRequiredFromUnloadedBundles.getValue() >= conf.getMinUnloadMessageThroughput())) { - // Sort bundles by throughput, then pick the bundle which can help to reduce load uniformly with - // under-loaded broker - loadBundleData.entrySet().stream() - .filter(e -> overloadedBrokerData.getBundles().contains(e.getKey())) - .map((e) -> { - String bundle = e.getKey(); - BundleData bundleData = e.getValue(); - TimeAverageMessageData shortTermData = bundleData.getShortTermData(); - double throughput = isMsgRateThresholdExceeded - ? shortTermData.getMsgRateIn() + shortTermData.getMsgRateOut() - : shortTermData.getMsgThroughputIn() + shortTermData.getMsgThroughputOut(); - return Triple.of(bundle, bundleData, throughput); - }).filter(e -> !recentlyUnloadedBundles.containsKey(e.getLeft())) - .sorted((e1, e2) -> Double.compare(e2.getRight(), e1.getRight())).forEach((e) -> { - if (conf.getMaxUnloadBundleNumPerShedding() != -1 - && selectedBundlesCache.size() >= conf.getMaxUnloadBundleNumPerShedding()) { - return; - } - String bundle = e.getLeft(); - BundleData bundleData = e.getMiddle(); - TimeAverageMessageData shortTermData = bundleData.getShortTermData(); - double throughput = shortTermData.getMsgThroughputIn() - + shortTermData.getMsgThroughputOut(); - double bundleMsgRate = shortTermData.getMsgRateIn() + shortTermData.getMsgRateOut(); - if (isMsgRateThresholdExceeded) { + if (isMsgRateThresholdExceeded) { + if (log.isDebugEnabled()) { + log.debug("Found bundles for uniform load balancing. " + + "msgRate overloaded broker: {} with msgRate: {}, " + + "msgRate underloaded broker: {} with msgRate: {}", + msgRateOverloadedBroker.getValue(), maxMsgRate.getValue(), + msgRateUnderloadedBroker.getValue(), minMsgRate.getValue()); + } + LocalBrokerData overloadedBrokerData = + brokersData.get(msgRateOverloadedBroker.getValue()).getLocalData(); + if (overloadedBrokerData.getBundles().size() > 1 + && (msgRateRequiredFromUnloadedBundles.getValue() >= conf.getMinUnloadMessage())) { + // Sort bundles by throughput, then pick the bundle which can help to reduce load uniformly with + // under-loaded broker + loadBundleData.entrySet().stream() + .filter(e -> overloadedBrokerData.getBundles().contains(e.getKey())) + .map((e) -> { + String bundle = e.getKey(); + TimeAverageMessageData shortTermData = e.getValue().getShortTermData(); + double msgRate = shortTermData.getMsgRateIn() + shortTermData.getMsgRateOut(); + return Pair.of(bundle, msgRate); + }).filter(e -> !recentlyUnloadedBundles.containsKey(e.getLeft())) + .sorted((e1, e2) -> Double.compare(e2.getRight(), e1.getRight())).forEach((e) -> { + if (conf.getMaxUnloadBundleNumPerShedding() != -1 + && selectedBundlesCache.size() >= conf.getMaxUnloadBundleNumPerShedding()) { + return; + } + String bundle = e.getLeft(); + double bundleMsgRate = e.getRight(); if (bundleMsgRate <= (msgRateRequiredFromUnloadedBundles.getValue() + 1000/* delta */)) { log.info("Found bundle to unload with msgRate {}", bundleMsgRate); msgRateRequiredFromUnloadedBundles.add(-bundleMsgRate); - selectedBundlesCache.put(overloadedBroker.getValue(), bundle); + selectedBundlesCache.put(msgRateOverloadedBroker.getValue(), bundle); } - } else { - if (throughput <= (msgThroughputRequiredFromUnloadedBundles.getValue())) { - log.info("Found bundle to unload with throughput {}", throughput); - msgThroughputRequiredFromUnloadedBundles.add(-throughput); - selectedBundlesCache.put(overloadedBroker.getValue(), bundle); + }); + } + } else { + if (log.isDebugEnabled()) { + log.debug("Found bundles for uniform load balancing. " + + "msgThroughput overloaded broker: {} with msgThroughput {}, " + + "msgThroughput underloaded broker: {} with msgThroughput: {}", + msgThroughputOverloadedBroker.getValue(), maxThroughput.getValue(), + msgThroughputUnderloadedBroker.getValue(), minThroughput.getValue()); + } + LocalBrokerData overloadedBrokerData = + brokersData.get(msgThroughputOverloadedBroker.getValue()).getLocalData(); + if (overloadedBrokerData.getBundles().size() > 1 + && + msgThroughputRequiredFromUnloadedBundles.getValue() >= conf.getMinUnloadMessageThroughput()) { + // Sort bundles by throughput, then pick the bundle which can help to reduce load uniformly with + // under-loaded broker + loadBundleData.entrySet().stream() + .filter(e -> overloadedBrokerData.getBundles().contains(e.getKey())) + .map((e) -> { + String bundle = e.getKey(); + TimeAverageMessageData shortTermData = e.getValue().getShortTermData(); + double msgThroughput = shortTermData.getMsgThroughputIn() + + shortTermData.getMsgThroughputOut(); + return Pair.of(bundle, msgThroughput); + }).filter(e -> !recentlyUnloadedBundles.containsKey(e.getLeft())) + .sorted((e1, e2) -> Double.compare(e2.getRight(), e1.getRight())).forEach((e) -> { + if (conf.getMaxUnloadBundleNumPerShedding() != -1 + && selectedBundlesCache.size() >= conf.getMaxUnloadBundleNumPerShedding()) { + return; } - } - }); + String bundle = e.getLeft(); + double msgThroughput = e.getRight(); + if (msgThroughput <= (msgThroughputRequiredFromUnloadedBundles.getValue() + + 1000/* delta */)) { + 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 00182fffb8a31..4b4042cf31a72 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 @@ -26,6 +26,7 @@ import org.testng.annotations.Test; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; @Test(groups = "broker") public class UniformLoadShedderTest { @@ -119,4 +120,81 @@ public void testBrokerWithMultipleBundles() { assertFalse(bundlesToUnload.isEmpty()); } + @Test + public void testOverloadBrokerSelect() { + conf.setMaxUnloadBundleNumPerShedding(1); + conf.setMaxUnloadPercentage(0.5); + int numBrokers = 5; + int numBundles = 5; + LoadData loadData = new LoadData(); + + LocalBrokerData[] localBrokerDatas = new LocalBrokerData[]{ + new LocalBrokerData(), + new LocalBrokerData(), + new LocalBrokerData(), + new LocalBrokerData(), + new LocalBrokerData()}; + + String[] brokerNames = new String[]{"broker0", "broker1", "broker2", "broker3", "broker4"}; + + double[] brokerMsgRates = new double[]{ + 50000, // broker0 + 60000, // broker1 + 70000, // broker2 + 10000, // broker3 + 20000};// broker4 + + double[] brokerMsgThroughputs = new double[]{ + 50 * 1024 * 1024, // broker0 + 60 * 1024 * 1024, // broker1 + 70 * 1024 * 1024, // broker2 + 80 * 1024 * 1024, // broker3 + 10 * 1024 * 1024};// broker4 + + + for (int brokerId = 0; brokerId < numBrokers; brokerId++) { + double msgRate = brokerMsgRates[brokerId] / numBundles; + double throughput = brokerMsgThroughputs[brokerId] / numBundles; + for (int i = 0; i < numBundles; ++i) { + String bundleName = "broker-" + brokerId + "-bundle-" + i; + localBrokerDatas[brokerId].getBundles().add(bundleName); + localBrokerDatas[brokerId].setMsgRateIn(brokerMsgRates[brokerId]); + localBrokerDatas[brokerId].setMsgThroughputIn(brokerMsgThroughputs[brokerId]); + BundleData bundle = new BundleData(); + + TimeAverageMessageData timeAverageMessageData = new TimeAverageMessageData(); + timeAverageMessageData.setMsgRateIn(msgRate); + timeAverageMessageData.setMsgThroughputIn(throughput); + bundle.setShortTermData(timeAverageMessageData); + loadData.getBundleData().put(bundleName, bundle); + } + loadData.getBrokerData().put(brokerNames[brokerId], new BrokerData(localBrokerDatas[brokerId])); + } + + // disable throughput based load shedding, enable rate based load shedding only + conf.setLoadBalancerMsgRateDifferenceShedderThreshold(50); + conf.setLoadBalancerMsgThroughputMultiplierDifferenceShedderThreshold(0); + + Multimap bundlesToUnload = uniformLoadShedder.findBundlesForUnloading(loadData, conf); + assertEquals(bundlesToUnload.size(), 1); + assertTrue(bundlesToUnload.containsKey("broker2")); + + + // disable rate based load shedding, enable throughput based load shedding only + conf.setLoadBalancerMsgRateDifferenceShedderThreshold(0); + conf.setLoadBalancerMsgThroughputMultiplierDifferenceShedderThreshold(2); + + bundlesToUnload = uniformLoadShedder.findBundlesForUnloading(loadData, conf); + assertEquals(bundlesToUnload.size(), 1); + assertTrue(bundlesToUnload.containsKey("broker3")); + + // enable both rate and throughput based load shedding, but rate based load shedding has higher priority + conf.setLoadBalancerMsgRateDifferenceShedderThreshold(50); + conf.setLoadBalancerMsgThroughputMultiplierDifferenceShedderThreshold(2); + + bundlesToUnload = uniformLoadShedder.findBundlesForUnloading(loadData, conf); + assertEquals(bundlesToUnload.size(), 1); + assertTrue(bundlesToUnload.containsKey("broker2")); + } + } From d656cb54aba31d8f2270d23ead7c8707bf43611b Mon Sep 17 00:00:00 2001 From: thetumbled <52550727+thetumbled@users.noreply.github.com> Date: Fri, 15 Sep 2023 21:12:39 +0800 Subject: [PATCH 057/699] [fix][broker] fix bug caused by optimistic locking (#18390) --- .../collections/ConcurrentLongHashMap.java | 25 ++++--- .../ConcurrentLongLongPairHashMap.java | 39 ++++++----- .../collections/ConcurrentLongPairSet.java | 41 +++++++----- .../collections/ConcurrentOpenHashMap.java | 39 +++++++---- .../collections/ConcurrentOpenHashSet.java | 22 +++---- .../pulsar/common/util/FutureUtilTest.java | 1 - .../ConcurrentLongHashMapTest.java | 62 +++++++++++++++++- .../ConcurrentLongLongPairHashMapTest.java | 65 +++++++++++++++++++ .../ConcurrentLongPairSetTest.java | 64 ++++++++++++++++++ .../ConcurrentOpenHashMapTest.java | 64 ++++++++++++++++++ .../ConcurrentOpenHashSetTest.java | 64 ++++++++++++++++++ 11 files changed, 414 insertions(+), 72 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongHashMap.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongHashMap.java index 31b4cb7cbf152..f5b47e7f1ab7a 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongHashMap.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongHashMap.java @@ -306,16 +306,17 @@ private static final class Section extends StampedLock { } V get(long key, int keyHash) { - int bucket = keyHash; - long stamp = tryOptimisticRead(); boolean acquiredLock = false; + // add local variable here, so OutOfBound won't happen + long[] keys = this.keys; + V[] values = this.values; + // calculate table.length as capacity to avoid rehash changing capacity + int bucket = signSafeMod(keyHash, values.length); + try { while (true) { - int capacity = this.capacity; - bucket = signSafeMod(bucket, capacity); - // First try optimistic locking long storedKey = keys[bucket]; V storedValue = values[bucket]; @@ -333,16 +334,15 @@ V get(long key, int keyHash) { if (!acquiredLock) { stamp = readLock(); acquiredLock = true; + + // update local variable + keys = this.keys; + values = this.values; + bucket = signSafeMod(keyHash, values.length); storedKey = keys[bucket]; storedValue = values[bucket]; } - if (capacity != this.capacity) { - // There has been a rehashing. We need to restart the search - bucket = keyHash; - continue; - } - if (storedKey == key) { return storedValue != DeletedValue ? storedValue : null; } else if (storedValue == EmptyValue) { @@ -350,8 +350,7 @@ V get(long key, int keyHash) { return null; } } - - ++bucket; + bucket = (bucket + 1) & (values.length - 1); } } finally { if (acquiredLock) { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongLongPairHashMap.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongLongPairHashMap.java index c0ccad9b73d5b..c3babbb8d1103 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongLongPairHashMap.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongLongPairHashMap.java @@ -284,6 +284,9 @@ public Map asMap() { // 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 4 continuous array space. + private static final int ITEM_SIZE = 4; + // Keys and values are stored interleaved in the table array private volatile long[] table; @@ -306,7 +309,7 @@ private static final class Section extends StampedLock { float expandFactor, float shrinkFactor) { this.capacity = alignToPowerOfTwo(capacity); this.initCapacity = this.capacity; - this.table = new long[4 * this.capacity]; + this.table = new long[ITEM_SIZE * this.capacity]; this.size = 0; this.usedBuckets = 0; this.autoShrink = autoShrink; @@ -322,7 +325,10 @@ private static final class Section extends StampedLock { LongPair get(long key1, long key2, int keyHash) { long stamp = tryOptimisticRead(); boolean acquiredLock = false; - int bucket = signSafeMod(keyHash, capacity); + // add local variable here, so OutOfBound won't happen + long[] table = this.table; + // calculate table.length / 4 as capacity to avoid rehash changing capacity + int bucket = signSafeMod(keyHash, table.length / ITEM_SIZE); try { while (true) { @@ -345,8 +351,9 @@ LongPair get(long key1, long key2, int keyHash) { if (!acquiredLock) { stamp = readLock(); acquiredLock = true; - - bucket = signSafeMod(keyHash, capacity); + // update local variable + table = this.table; + bucket = signSafeMod(keyHash, table.length / ITEM_SIZE); storedKey1 = table[bucket]; storedKey2 = table[bucket + 1]; storedValue1 = table[bucket + 2]; @@ -361,7 +368,7 @@ LongPair get(long key1, long key2, int keyHash) { } } - bucket = (bucket + 4) & (table.length - 1); + bucket = (bucket + ITEM_SIZE) & (table.length - 1); } } finally { if (acquiredLock) { @@ -413,7 +420,7 @@ boolean put(long key1, long key2, long value1, long value2, int keyHash, boolean } } - bucket = (bucket + 4) & (table.length - 1); + bucket = (bucket + ITEM_SIZE) & (table.length - 1); } } finally { if (usedBuckets > resizeThresholdUp) { @@ -454,7 +461,7 @@ private boolean remove(long key1, long key2, long value1, long value2, int keyHa return false; } - bucket = (bucket + 4) & (table.length - 1); + bucket = (bucket + ITEM_SIZE) & (table.length - 1); } } finally { @@ -480,7 +487,7 @@ private boolean remove(long key1, long key2, long value1, long value2, int keyHa } private void cleanBucket(int bucket) { - int nextInArray = (bucket + 4) & (table.length - 1); + int nextInArray = (bucket + ITEM_SIZE) & (table.length - 1); if (table[nextInArray] == EmptyKey) { table[bucket] = EmptyKey; table[bucket + 1] = EmptyKey; @@ -489,7 +496,7 @@ private void cleanBucket(int bucket) { --usedBuckets; // Cleanup all the buckets that were in `DeletedKey` state, so that we can reduce unnecessary expansions - bucket = (bucket - 4) & (table.length - 1); + bucket = (bucket - ITEM_SIZE) & (table.length - 1); while (table[bucket] == DeletedKey) { table[bucket] = EmptyKey; table[bucket + 1] = EmptyKey; @@ -497,7 +504,7 @@ private void cleanBucket(int bucket) { table[bucket + 3] = ValueNotFound; --usedBuckets; - bucket = (bucket - 4) & (table.length - 1); + bucket = (bucket - ITEM_SIZE) & (table.length - 1); } } else { table[bucket] = DeletedKey; @@ -540,7 +547,7 @@ public void forEach(BiConsumerLongPair processor) { } // Go through all the buckets for this section - for (int bucket = 0; bucket < table.length; bucket += 4) { + for (int bucket = 0; bucket < table.length; bucket += ITEM_SIZE) { long storedKey1 = table[bucket]; long storedKey2 = table[bucket + 1]; long storedValue1 = table[bucket + 2]; @@ -569,11 +576,11 @@ public void forEach(BiConsumerLongPair processor) { } private void rehash(int newCapacity) { - long[] newTable = new long[4 * newCapacity]; + long[] newTable = new long[ITEM_SIZE * newCapacity]; Arrays.fill(newTable, EmptyKey); // Re-hash table - for (int i = 0; i < table.length; i += 4) { + for (int i = 0; i < table.length; i += ITEM_SIZE) { long storedKey1 = table[i]; long storedKey2 = table[i + 1]; long storedValue1 = table[i + 2]; @@ -593,7 +600,7 @@ private void rehash(int newCapacity) { } private void shrinkToInitCapacity() { - long[] newTable = new long[4 * initCapacity]; + long[] newTable = new long[ITEM_SIZE * initCapacity]; Arrays.fill(newTable, EmptyKey); table = newTable; @@ -622,7 +629,7 @@ private static void insertKeyValueNoLock(long[] table, int capacity, long key1, return; } - bucket = (bucket + 4) & (table.length - 1); + bucket = (bucket + ITEM_SIZE) & (table.length - 1); } } } @@ -641,6 +648,8 @@ static final long hash(long key1, long key2) { } static final int signSafeMod(long n, int max) { + // as the ITEM_SIZE of Section is 4, so the index is the multiple of 4 + // that is to left shift 2 bits return (int) (n & (max - 1)) << 2; } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongPairSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongPairSet.java index 2a1090503857c..f0f04c4edf904 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongPairSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentLongPairSet.java @@ -294,6 +294,9 @@ public Set items(int numberOfItems, LongPairFunction longPairConverter // 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 long[] table; @@ -315,7 +318,7 @@ private static final class Section extends StampedLock { float expandFactor, float shrinkFactor) { this.capacity = alignToPowerOfTwo(capacity); this.initCapacity = this.capacity; - this.table = new long[2 * this.capacity]; + this.table = new long[ITEM_SIZE * this.capacity]; this.size = 0; this.usedBuckets = 0; this.autoShrink = autoShrink; @@ -331,7 +334,11 @@ private static final class Section extends StampedLock { boolean contains(long item1, long item2, int hash) { long stamp = tryOptimisticRead(); boolean acquiredLock = false; - int bucket = signSafeMod(hash, capacity); + + // add local variable here, so OutOfBound won't happen + long[] table = this.table; + // calculate table.length / 2 as capacity to avoid rehash changing capacity + int bucket = signSafeMod(hash, table.length / ITEM_SIZE); try { while (true) { @@ -353,7 +360,9 @@ boolean contains(long item1, long item2, int hash) { stamp = readLock(); acquiredLock = true; - bucket = signSafeMod(hash, capacity); + // update local variable + table = this.table; + bucket = signSafeMod(hash, table.length / ITEM_SIZE); storedItem1 = table[bucket]; storedItem2 = table[bucket + 1]; } @@ -366,7 +375,7 @@ boolean contains(long item1, long item2, int hash) { } } - bucket = (bucket + 2) & (table.length - 1); + bucket = (bucket + ITEM_SIZE) & (table.length - 1); } } finally { if (acquiredLock) { @@ -410,7 +419,7 @@ boolean add(long item1, long item2, long hash) { } } - bucket = (bucket + 2) & (table.length - 1); + bucket = (bucket + ITEM_SIZE) & (table.length - 1); } } finally { if (usedBuckets > resizeThresholdUp) { @@ -445,7 +454,7 @@ private boolean remove(long item1, long item2, int hash) { return false; } - bucket = (bucket + 2) & (table.length - 1); + bucket = (bucket + ITEM_SIZE) & (table.length - 1); } } finally { tryShrinkThenUnlock(stamp); @@ -459,7 +468,7 @@ private int removeIf(LongPairPredicate filter) { // Go through all the buckets for this section long stamp = writeLock(); try { - for (int bucket = 0; bucket < table.length; bucket += 2) { + for (int bucket = 0; bucket < table.length; bucket += ITEM_SIZE) { long storedItem1 = table[bucket]; long storedItem2 = table[bucket + 1]; if (storedItem1 != DeletedItem && storedItem1 != EmptyItem) { @@ -498,7 +507,7 @@ private void tryShrinkThenUnlock(long stamp) { } private void cleanBucket(int bucket) { - int nextInArray = (bucket + 2) & (table.length - 1); + int nextInArray = (bucket + ITEM_SIZE) & (table.length - 1); if (table[nextInArray] == EmptyItem) { table[bucket] = EmptyItem; table[bucket + 1] = EmptyItem; @@ -506,13 +515,13 @@ private void cleanBucket(int bucket) { // Cleanup all the buckets that were in `DeletedItem` state, // so that we can reduce unnecessary expansions - int lastBucket = (bucket - 2) & (table.length - 1); + int lastBucket = (bucket - ITEM_SIZE) & (table.length - 1); while (table[lastBucket] == DeletedItem) { table[lastBucket] = EmptyItem; table[lastBucket + 1] = EmptyItem; --usedBuckets; - lastBucket = (lastBucket - 2) & (table.length - 1); + lastBucket = (lastBucket - ITEM_SIZE) & (table.length - 1); } } else { table[bucket] = DeletedItem; @@ -542,7 +551,7 @@ public void forEach(LongPairConsumer processor) { // 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 += 2) { + for (int bucket = 0; bucket < table.length; bucket += ITEM_SIZE) { if (stamp == 0) { stamp = tryOptimisticRead(); } @@ -572,11 +581,11 @@ public void forEach(LongPairConsumer processor) { private void rehash(int newCapacity) { // Expand the hashmap - long[] newTable = new long[2 * newCapacity]; + long[] newTable = new long[ITEM_SIZE * newCapacity]; Arrays.fill(newTable, EmptyItem); // Re-hash table - for (int i = 0; i < table.length; i += 2) { + for (int i = 0; i < table.length; i += ITEM_SIZE) { long storedItem1 = table[i]; long storedItem2 = table[i + 1]; if (storedItem1 != EmptyItem && storedItem1 != DeletedItem) { @@ -595,7 +604,7 @@ private void rehash(int newCapacity) { private void shrinkToInitCapacity() { // Expand the hashmap - long[] newTable = new long[2 * initCapacity]; + long[] newTable = new long[ITEM_SIZE * initCapacity]; Arrays.fill(newTable, EmptyItem); table = newTable; @@ -621,7 +630,7 @@ private static void insertKeyValueNoLock(long[] table, int capacity, long item1, return; } - bucket = (bucket + 2) & (table.length - 1); + bucket = (bucket + ITEM_SIZE) & (table.length - 1); } } } @@ -640,6 +649,8 @@ static final long hash(long key1, long key2) { } 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; } 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 index ea2e01768ac7e..1aa95d3090eb2 100644 --- 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 @@ -296,6 +296,9 @@ public List 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; @@ -317,7 +320,7 @@ private static final class Section extends StampedLock { float expandFactor, float shrinkFactor) { this.capacity = alignToPowerOfTwo(capacity); this.initCapacity = this.capacity; - this.table = new Object[2 * this.capacity]; + this.table = new Object[ITEM_SIZE * this.capacity]; this.size = 0; this.usedBuckets = 0; this.autoShrink = autoShrink; @@ -332,7 +335,11 @@ private static final class Section extends StampedLock { V get(K key, int keyHash) { long stamp = tryOptimisticRead(); boolean acquiredLock = false; - int bucket = signSafeMod(keyHash, capacity); + + // 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) { @@ -354,7 +361,9 @@ V get(K key, int keyHash) { stamp = readLock(); acquiredLock = true; - bucket = signSafeMod(keyHash, capacity); + // update local variable + table = this.table; + bucket = signSafeMod(keyHash, table.length / ITEM_SIZE); storedKey = (K) table[bucket]; storedValue = (V) table[bucket + 1]; } @@ -367,7 +376,7 @@ V get(K key, int keyHash) { } } - bucket = (bucket + 2) & (table.length - 1); + bucket = (bucket + ITEM_SIZE) & (table.length - 1); } } finally { if (acquiredLock) { @@ -420,7 +429,7 @@ V put(K key, V value, int keyHash, boolean onlyIfAbsent, Function valuePro } } - bucket = (bucket + 2) & (table.length - 1); + bucket = (bucket + ITEM_SIZE) & (table.length - 1); } } finally { if (usedBuckets > resizeThresholdUp) { @@ -449,7 +458,7 @@ private V remove(K key, Object value, int keyHash) { if (value == null || value.equals(storedValue)) { SIZE_UPDATER.decrementAndGet(this); - int nextInArray = (bucket + 2) & (table.length - 1); + int nextInArray = (bucket + ITEM_SIZE) & (table.length - 1); if (table[nextInArray] == EmptyKey) { table[bucket] = EmptyKey; table[bucket + 1] = null; @@ -457,13 +466,13 @@ private V remove(K key, Object value, int keyHash) { // Cleanup all the buckets that were in `DeletedKey` state, // so that we can reduce unnecessary expansions - int lastBucket = (bucket - 2) & (table.length - 1); + int lastBucket = (bucket - ITEM_SIZE) & (table.length - 1); while (table[lastBucket] == DeletedKey) { table[lastBucket] = EmptyKey; table[lastBucket + 1] = null; --usedBuckets; - lastBucket = (lastBucket - 2) & (table.length - 1); + lastBucket = (lastBucket - ITEM_SIZE) & (table.length - 1); } } else { table[bucket] = DeletedKey; @@ -479,7 +488,7 @@ private V remove(K key, Object value, int keyHash) { return null; } - bucket = (bucket + 2) & (table.length - 1); + bucket = (bucket + ITEM_SIZE) & (table.length - 1); } } finally { @@ -528,7 +537,7 @@ public void forEach(BiConsumer processor) { // 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 += 2) { + for (int bucket = 0; bucket < table.length; bucket += ITEM_SIZE) { if (stamp == 0) { stamp = tryOptimisticRead(); } @@ -558,10 +567,10 @@ public void forEach(BiConsumer processor) { private void rehash(int newCapacity) { // Expand the hashmap - Object[] newTable = new Object[2 * newCapacity]; + Object[] newTable = new Object[ITEM_SIZE * newCapacity]; // Re-hash table - for (int i = 0; i < table.length; i += 2) { + 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) { @@ -577,7 +586,7 @@ private void rehash(int newCapacity) { } private void shrinkToInitCapacity() { - Object[] newTable = new Object[2 * initCapacity]; + Object[] newTable = new Object[ITEM_SIZE * initCapacity]; table = newTable; size = 0; @@ -602,7 +611,7 @@ private static void insertKeyValueNoLock(Object[] table, int capacity, K return; } - bucket = (bucket + 2) & (table.length - 1); + bucket = (bucket + ITEM_SIZE) & (table.length - 1); } } } @@ -618,6 +627,8 @@ static final long hash(K key) { } 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; } 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 index cc8bc07b43095..162cbed5a5ddc 100644 --- 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 @@ -294,16 +294,16 @@ private static final class Section extends StampedLock { } boolean contains(V value, int keyHash) { - int bucket = 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) { - int capacity = this.capacity; - bucket = signSafeMod(bucket, capacity); - // First try optimistic locking V storedValue = values[bucket]; @@ -321,15 +321,12 @@ boolean contains(V value, int keyHash) { stamp = readLock(); acquiredLock = true; + // update local variable + values = this.values; + bucket = signSafeMod(keyHash, values.length); storedValue = values[bucket]; } - if (capacity != this.capacity) { - // There has been a rehashing. We need to restart the search - bucket = keyHash; - continue; - } - if (value.equals(storedValue)) { return true; } else if (storedValue == EmptyValue) { @@ -337,8 +334,7 @@ boolean contains(V value, int keyHash) { return false; } } - - ++bucket; + bucket = (bucket + 1) & (values.length - 1); } } finally { if (acquiredLock) { diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/FutureUtilTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/FutureUtilTest.java index 6df4494edf886..7d44c187d7355 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/FutureUtilTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/FutureUtilTest.java @@ -181,7 +181,6 @@ public void testWaitForAny() { } } - @Test public void testSequencer() { int concurrentNum = 1000; final ScheduledExecutorService executor = Executors.newScheduledThreadPool(concurrentNum); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongHashMapTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongHashMapTest.java index a317fa63c0986..e1f947ad8c4f6 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongHashMapTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongHashMapTest.java @@ -38,6 +38,7 @@ 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.LongFunction; import lombok.Cleanup; @@ -213,6 +214,66 @@ public void testExpandShrinkAndClear() { assertTrue(map.capacity() == initCapacity); } + @Test + public void testConcurrentExpandAndShrinkAndGet() throws Throwable { + ConcurrentLongHashMap map = ConcurrentLongHashMap.newBuilder() + .expectedItems(2) + .concurrencyLevel(1) + .autoShrink(true) + .mapIdleFactor(0.25f) + .build(); + assertEquals(map.capacity(), 4); + + 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); + } + try { + map.get(1); + } catch (Exception e) { + ex.set(e); + } + }); + } + + assertNull(map.put(1,"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(2, "v2")); + assertNull(map.put(3, "v3")); + assertEquals(map.capacity(), 8); + + // shrink hashmap + assertTrue(map.remove(2, "v2")); + assertTrue(map.remove(3, "v3")); + assertEquals(map.capacity(), 4); + } + }); + + future.get(); + assertTrue(ex.get() == null); + // shut down pool + executor.shutdown(); + } + @Test public void testRemove() { ConcurrentLongHashMap map = ConcurrentLongHashMap.newBuilder() @@ -361,7 +422,6 @@ public void concurrentInsertionsAndReads() throws Throwable { assertEquals(map.size(), N * nThreads); } - @Test public void stressConcurrentInsertionsAndReads() throws Throwable { ConcurrentLongHashMap map = ConcurrentLongHashMap.newBuilder() .expectedItems(4) diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongLongPairHashMapTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongLongPairHashMapTest.java index 8e74d285ffb9b..0de3fdb5c84bf 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongLongPairHashMapTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongLongPairHashMapTest.java @@ -31,9 +31,12 @@ import java.util.List; import java.util.Map; 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 org.apache.pulsar.common.util.collections.ConcurrentLongLongPairHashMap.LongPair; import org.testng.annotations.Test; @@ -173,6 +176,68 @@ public void testExpandAndShrink() { assertEquals(map.capacity(), 8); } + @Test + public void testConcurrentExpandAndShrinkAndGet() throws Throwable { + ConcurrentLongLongPairHashMap map = ConcurrentLongLongPairHashMap.newBuilder() + .expectedItems(2) + .concurrencyLevel(1) + .autoShrink(true) + .mapIdleFactor(0.25f) + .build(); + assertEquals(map.capacity(), 4); + + 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 (true) { + try { + map.get(1, 1); + } catch (Exception e) { + ex.set(e); + } + } + }); + } + + assertTrue(map.put(1, 1, 11, 11)); + future = executor.submit(() -> { + try { + barrier.await(); + } catch (Exception e) { + throw new RuntimeException(e); + } + + for (int i = 0; i < n; i++) { + // expand hashmap + assertTrue(map.put(2, 2, 22, 22)); + assertTrue(map.put(3, 3, 33, 33)); + assertEquals(map.capacity(), 8); + + // shrink hashmap + assertTrue(map.remove(2, 2, 22, 22)); + assertTrue(map.remove(3, 3, 33, 33)); + assertEquals(map.capacity(), 4); + } + }); + + future.get(); + assertTrue(ex.get() == null); + // shut down pool + executor.shutdown(); + } + @Test public void testExpandShrinkAndClear() { ConcurrentLongLongPairHashMap map = ConcurrentLongLongPairHashMap.newBuilder() diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongPairSetTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongPairSetTest.java index 7e947ae6e6aa3..bce2b8993835f 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongPairSetTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentLongPairSetTest.java @@ -30,9 +30,11 @@ import java.util.List; import java.util.Random; import java.util.Set; +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.apache.pulsar.common.util.collections.ConcurrentLongPairSet.LongPair; @@ -211,6 +213,68 @@ public void testExpandShrinkAndClear() { assertTrue(map.capacity() == initCapacity); } + @Test + public void testConcurrentExpandAndShrinkAndGet() throws Throwable { + ConcurrentLongPairSet set = ConcurrentLongPairSet.newBuilder() + .expectedItems(2) + .concurrencyLevel(1) + .autoShrink(true) + .mapIdleFactor(0.25f) + .build(); + assertEquals(set.capacity(), 4); + + 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 (true) { + try { + set.contains(1, 1); + } catch (Exception e) { + ex.set(e); + } + } + }); + } + + assertTrue(set.add(1, 1)); + 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(2, 2)); + assertTrue(set.add(3, 3)); + assertEquals(set.capacity(), 8); + + // shrink hashmap + assertTrue(set.remove(2, 2)); + assertTrue(set.remove(3, 3)); + assertEquals(set.capacity(), 4); + } + }); + + future.get(); + assertTrue(ex.get() == null); + // shut down pool + executor.shutdown(); + } + @Test public void testRemove() { 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 index 198a3f4c5c38b..410d490b98faa 100644 --- 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 @@ -32,11 +32,13 @@ 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; @@ -215,6 +217,68 @@ public void testExpandShrinkAndClear() { 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); + + 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 (true) { + 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 = 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 index 27c18abb8b347..6a40095ab0647 100644 --- 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 @@ -28,9 +28,11 @@ 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; @@ -186,6 +188,68 @@ public void testExpandShrinkAndClear() { 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); + + 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 (true) { + 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 = From ad1a6ee97f28978ea364fe4fa657c6fe85b8b74f Mon Sep 17 00:00:00 2001 From: Hang Chen Date: Wed, 13 Sep 2023 17:31:28 +0800 Subject: [PATCH 058/699] [fix] [bookie] Fix RocksDB configuration (#21157) --- conf/entry_location_rocksdb.conf | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/conf/entry_location_rocksdb.conf b/conf/entry_location_rocksdb.conf index 31bd58506ef75..42d916ded378f 100644 --- a/conf/entry_location_rocksdb.conf +++ b/conf/entry_location_rocksdb.conf @@ -52,6 +52,8 @@ max_bytes_for_level_base=268435456 # set by jni: options.setTargetFileSizeBase target_file_size_base=67108864 + # set by jni: options.setLevelCompactionDynamicLevelBytes + level_compaction_dynamic_level_bytes=true [TableOptions/BlockBasedTable "default"] # set by jni: tableOptions.setBlockSize @@ -66,5 +68,3 @@ filter_policy=rocksdb.BloomFilter:10:false # set by jni: tableOptions.setCacheIndexAndFilterBlocks cache_index_and_filter_blocks=true - # set by jni: options.setLevelCompactionDynamicLevelBytes - level_compaction_dynamic_level_bytes=true \ No newline at end of file From 68a66096910cdfc7d941e34026db6d5d8b75fc22 Mon Sep 17 00:00:00 2001 From: Yan Zhao Date: Sun, 17 Sep 2023 21:44:05 +0800 Subject: [PATCH 059/699] [fix][auto-recovery] Fix metadata store deadlock due to BookkeeperInternalCallbacks.Processor (#21159) --- pom.xml | 14 ++ pulsar-metadata/pom.xml | 21 ++ .../AbstractHierarchicalLedgerManager.java | 9 +- .../replication/AuditorPeriodicCheckTest.java | 219 ++++++++++++++++++ 4 files changed, 259 insertions(+), 4 deletions(-) create mode 100644 pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java diff --git a/pom.xml b/pom.xml index d67c48c6308c7..0aebdc6a10e30 100644 --- a/pom.xml +++ b/pom.xml @@ -1501,6 +1501,20 @@ flexible messaging model and an intuitive client API. + + + org.apache.bookkeeper + bookkeeper-common + ${bookkeeper.version} + test + tests + + + com.fasterxml.jackson.core + * + + + diff --git a/pulsar-metadata/pom.xml b/pulsar-metadata/pom.xml index eaff294dff860..c79431ffdd6f0 100644 --- a/pulsar-metadata/pom.xml +++ b/pulsar-metadata/pom.xml @@ -64,6 +64,27 @@ test + + org.apache.zookeeper + zookeeper + tests + test + + + ch.qos.logback + logback-core + + + ch.qos.logback + logback-classic + + + io.netty + netty-tcnative + + + + org.xerial.snappy diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/AbstractHierarchicalLedgerManager.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/AbstractHierarchicalLedgerManager.java index a33c8761cba9e..4db7f4798c309 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/AbstractHierarchicalLedgerManager.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/AbstractHierarchicalLedgerManager.java @@ -206,10 +206,11 @@ protected void asyncProcessLedgersInSingleNode( mcb = new BookkeeperInternalCallbacks.MultiCallback(activeLedgers.size(), finalCb, ctx, successRc, failureRc); // start loop over all ledgers - for (Long ledger : activeLedgers) { - processor.process(ledger, mcb); - } - + scheduler.submit(() -> { + for (Long ledger : activeLedgers) { + processor.process(ledger, mcb); + } + }); }).exceptionally(ex -> { finalCb.processResult(failureRc, null, ctx); return null; diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java new file mode 100644 index 0000000000000..c761d46c62266 --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java @@ -0,0 +1,219 @@ +/* + * 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.replication; + +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertFalse; +import static org.testng.AssertJUnit.assertNotSame; +import io.netty.buffer.ByteBuf; +import java.io.IOException; +import java.net.URI; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.BookieException; +import org.apache.bookkeeper.bookie.TestBookieImpl; +import org.apache.bookkeeper.client.BookKeeper.DigestType; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.meta.MetadataBookieDriver; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterTest; +import org.testng.annotations.BeforeTest; +import org.testng.annotations.Test; + +/** + * This test verifies that the period check on the auditor + * will pick up on missing data in the client. + */ +public class AuditorPeriodicCheckTest extends BookKeeperClusterTestCase { + private static final Logger LOG = LoggerFactory + .getLogger(AuditorPeriodicCheckTest.class); + + private MetadataBookieDriver driver; + private HashMap auditorElectors = new HashMap(); + + private static final int CHECK_INTERVAL = 1; // run every second + + public AuditorPeriodicCheckTest() throws Exception { + super(3); + baseConf.setPageLimit(1); // to make it easy to push ledger out of cache + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataClientDriver"); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataBookieDriver"); + } + + @BeforeTest + @Override + public void setUp() throws Exception { + super.setUp(); + + for (int i = 0; i < numBookies; i++) { + ServerConfiguration conf = new ServerConfiguration(confByIndex(i)); + conf.setAuditorPeriodicCheckInterval(CHECK_INTERVAL); + conf.setMetadataServiceUri( + zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + + String addr = addressByIndex(i).toString(); + + AuditorElector auditorElector = new AuditorElector(addr, conf); + auditorElectors.put(addr, auditorElector); + auditorElector.start(); + if (LOG.isDebugEnabled()) { + LOG.debug("Starting Auditor Elector"); + } + } + + URI uri = URI.create(confByIndex(0).getMetadataServiceUri().replaceAll("zk://", "metadata-store:") + .replaceAll("/ledgers", "")); + driver = MetadataDrivers.getBookieDriver(uri); + ServerConfiguration serverConfiguration = new ServerConfiguration(confByIndex(0)); + serverConfiguration.setMetadataServiceUri( + serverConfiguration.getMetadataServiceUri().replaceAll("zk://", "metadata-store:") + .replaceAll("/ledgers", "")); + driver.initialize(serverConfiguration, NullStatsLogger.INSTANCE); + } + + @AfterTest + @Override + public void tearDown() throws Exception { + if (null != driver) { + driver.close(); + } + + for (AuditorElector e : auditorElectors.values()) { + e.shutdown(); + } + super.tearDown(); + } + + private BookieId replaceBookieWithWriteFailingBookie(LedgerHandle lh) throws Exception { + int bookieIdx = -1; + Long entryId = lh.getLedgerMetadata().getAllEnsembles().firstKey(); + List curEnsemble = lh.getLedgerMetadata().getAllEnsembles().get(entryId); + + // Identify a bookie in the current ledger ensemble to be replaced + BookieId replacedBookie = null; + for (int i = 0; i < numBookies; i++) { + if (curEnsemble.contains(addressByIndex(i))) { + bookieIdx = i; + replacedBookie = addressByIndex(i); + break; + } + } + assertNotSame("Couldn't find ensemble bookie in bookie list", -1, bookieIdx); + + LOG.info("Killing bookie " + addressByIndex(bookieIdx)); + ServerConfiguration conf = killBookie(bookieIdx); + Bookie writeFailingBookie = new TestBookieImpl(conf) { + @Override + public void addEntry(ByteBuf entry, boolean ackBeforeSync, WriteCallback cb, + Object ctx, byte[] masterKey) + throws IOException, BookieException { + try { + LOG.info("Failing write to entry "); + // sleep a bit so that writes to other bookies succeed before + // the client hears about the failure on this bookie. If the + // client gets ack-quorum number of acks first, it won't care + // about any failures and won't reform the ensemble. + Thread.sleep(100); + throw new IOException(); + } catch (InterruptedException ie) { + // ignore, only interrupted if shutting down, + // and an exception would spam the logs + Thread.currentThread().interrupt(); + } + } + }; + startAndAddBookie(conf, writeFailingBookie); + return replacedBookie; + } + + /* + * Validates that the periodic ledger check will fix entries with a failed write. + */ + @Test + public void testFailedWriteRecovery() throws Exception { + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerUnderreplicationManager underReplicationManager = mFactory.newLedgerUnderreplicationManager(); + underReplicationManager.disableLedgerReplication(); + + LedgerHandle lh = bkc.createLedger(2, 2, 1, DigestType.CRC32, "passwd".getBytes()); + + // kill one of the bookies and replace it with one that rejects write; + // This way we get into the under replication state + BookieId replacedBookie = replaceBookieWithWriteFailingBookie(lh); + + // Write a few entries; this should cause under replication + byte[] data = "foobar".getBytes(); + data = "foobar".getBytes(); + lh.addEntry(data); + lh.addEntry(data); + lh.addEntry(data); + + lh.close(); + + // enable under replication detection and wait for it to report + // under replicated ledger + underReplicationManager.enableLedgerReplication(); + long underReplicatedLedger = -1; + for (int i = 0; i < 5; i++) { + underReplicatedLedger = underReplicationManager.pollLedgerToRereplicate(); + if (underReplicatedLedger != -1) { + break; + } + Thread.sleep(CHECK_INTERVAL * 1000); + } + assertEquals("Ledger should be under replicated", lh.getId(), underReplicatedLedger); + + // now start the replication workers + List l = new ArrayList(); + for (int i = 0; i < numBookies; i++) { + ReplicationWorker rw = new ReplicationWorker(confByIndex(i), NullStatsLogger.INSTANCE); + rw.start(); + l.add(rw); + } + underReplicationManager.close(); + + // Wait for ensemble to change after replication + Thread.sleep(3000); + for (ReplicationWorker rw : l) { + rw.shutdown(); + } + + // check that ensemble has changed and the bookie that rejected writes has + // been replaced in the ensemble + LedgerHandle newLh = bkc.openLedger(lh.getId(), DigestType.CRC32, "passwd".getBytes()); + for (Map.Entry> e : + newLh.getLedgerMetadata().getAllEnsembles().entrySet()) { + List ensemble = e.getValue(); + assertFalse("Ensemble hasn't been updated", ensemble.contains(replacedBookie)); + } + newLh.close(); + } +} From 4139291d21214b40a08884b0c487633c01c65ff8 Mon Sep 17 00:00:00 2001 From: thetumbled <52550727+thetumbled@users.noreply.github.com> Date: Thu, 14 Sep 2023 10:44:06 +0800 Subject: [PATCH 060/699] [improve] [broker] disable balancing based on DirectMemory. (#21168) --- conf/broker.conf | 4 +++- .../java/org/apache/pulsar/broker/ServiceConfiguration.java | 5 +++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index 40eb1bcc32485..4ad8536fd8d68 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1372,7 +1372,9 @@ loadBalancerCPUResourceWeight=1.0 # The direct memory usage weight when calculating new resource usage. # It only takes effect in the ThresholdShedder strategy. -loadBalancerDirectMemoryResourceWeight=1.0 +# Direct memory usage cannot accurately reflect the machine's load, +# and it is not recommended to use it to score the machine's load. +loadBalancerDirectMemoryResourceWeight=0 # Bundle unload minimum throughput threshold (MB), avoiding bundle unload frequently. # It only takes effect in the ThresholdShedder strategy. 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 3d73734e82767..fb26775591345 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 @@ -2425,9 +2425,10 @@ The delayed message index time step(in seconds) in per bucket snapshot segment, @FieldContext( dynamic = true, category = CATEGORY_LOAD_BALANCER, - doc = "Direct Memory Resource Usage Weight" + doc = "Direct Memory Resource Usage Weight. Direct memory usage cannot accurately reflect the " + + "machine's load, and it is not recommended to use it to score the machine's load." ) - private double loadBalancerDirectMemoryResourceWeight = 1.0; + private double loadBalancerDirectMemoryResourceWeight = 0; @FieldContext( dynamic = true, From ad69decab1545d3cf63e40e88fd957dca4991459 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Sun, 17 Sep 2023 20:44:13 -0500 Subject: [PATCH 061/699] [fix][broker] Fix PulsarService.getLookupServiceAddress returns wrong port if TLS is enabled (#21015) --- .../apache/pulsar/broker/PulsarService.java | 10 ++--- .../broker/namespace/NamespaceService.java | 6 +-- .../pulsar/broker/PulsarServiceTest.java | 2 + .../pulsar/broker/admin/AdminApi2Test.java | 14 ++----- .../pulsar/broker/admin/AdminApiTest.java | 2 +- .../auth/MockedPulsarServiceBaseTest.java | 4 -- .../loadbalance/AdvertisedListenersTest.java | 2 - .../broker/loadbalance/LoadBalancerTest.java | 1 - .../SimpleLoadManagerImplTest.java | 1 - .../impl/ModularLoadManagerImplTest.java | 6 --- .../OwnerShipForCurrentServerTestBase.java | 2 - .../broker/service/BrokerServiceTest.java | 9 +++++ .../broker/service/ClusterMigrationTest.java | 9 +++++ .../pulsar/broker/service/ReplicatorTest.java | 2 +- .../transaction/TransactionTestBase.java | 2 - .../client/api/BrokerServiceLookupTest.java | 7 +++- .../api/ClientAuthenticationTlsTest.java | 4 +- .../proxy/server/ProxyRefreshAuthTest.java | 1 + .../server/ProxyServiceTlsStarterTest.java | 1 + .../containers/BrokerContainer.java | 9 ++++- .../containers/ProxyContainer.java | 9 ++++- .../tests/integration/tls/ClientTlsTest.java | 9 +++++ .../integration/topologies/PulsarCluster.java | 40 +++++++++++++------ .../topologies/PulsarClusterSpec.java | 6 +++ 24 files changed, 101 insertions(+), 57 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 4ffb5b77d5424..46f0756a82aea 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 @@ -1729,18 +1729,18 @@ public static String webAddressTls(String host, int port) { } public String getSafeWebServiceAddress() { - return webServiceAddress != null ? webServiceAddress : webServiceAddressTls; + return webServiceAddressTls != null ? webServiceAddressTls : webServiceAddress; } @Deprecated public String getSafeBrokerServiceUrl() { - return brokerServiceUrl != null ? brokerServiceUrl : brokerServiceUrlTls; + return brokerServiceUrlTls != null ? brokerServiceUrlTls : brokerServiceUrl; } public String getLookupServiceAddress() { - return String.format("%s:%s", advertisedAddress, config.getWebServicePort().isPresent() - ? config.getWebServicePort().get() - : config.getWebServicePortTls().orElseThrow()); + return String.format("%s:%s", advertisedAddress, config.getWebServicePortTls().isPresent() + ? config.getWebServicePortTls().get() + : config.getWebServicePort().orElseThrow()); } public TopicPoliciesService getTopicPoliciesService() { 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 d66e3c3b65d76..d80ad5b7a60d1 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 @@ -1591,10 +1591,10 @@ public static NamespaceName getHeartbeatNamespaceV2(String lookupBroker, Service public static NamespaceName getSLAMonitorNamespace(String host, ServiceConfiguration config) { Integer port = null; - if (config.getWebServicePort().isPresent()) { - port = config.getWebServicePort().get(); - } else if (config.getWebServicePortTls().isPresent()) { + if (config.getWebServicePortTls().isPresent()) { port = config.getWebServicePortTls().get(); + } else if (config.getWebServicePort().isPresent()) { + port = config.getWebServicePort().get(); } return NamespaceName.get(String.format(SLA_NAMESPACE_FMT, config.getClusterName(), host, port)); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java index 37a7310ae17ca..3e0887646e119 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java @@ -54,6 +54,8 @@ protected void cleanup() throws Exception { @Override protected void doInitConf() throws Exception { super.doInitConf(); + conf.setBrokerServicePortTls(Optional.of(0)); + conf.setWebServicePortTls(Optional.of(0)); if (useStaticPorts) { conf.setBrokerServicePortTls(Optional.of(6651)); conf.setBrokerServicePort(Optional.of(6660)); 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 99009a1988206..c68010f967b9b 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 @@ -439,19 +439,13 @@ public void testTopicPoliciesWithMultiBroker() throws Exception { String tenantName = newUniqueName("prop-xyz2"); admin.tenants().createTenant(tenantName, tenantInfo); admin.namespaces().createNamespace(tenantName + "/ns1", Set.of("test")); - conf.setBrokerServicePort(Optional.of(1024)); - conf.setBrokerServicePortTls(Optional.of(1025)); - conf.setWebServicePort(Optional.of(1026)); - conf.setWebServicePortTls(Optional.of(1027)); + ServiceConfiguration config2 = super.getDefaultConf(); @Cleanup - PulsarTestContext pulsarTestContext2 = createAdditionalPulsarTestContext(conf); + PulsarTestContext pulsarTestContext2 = createAdditionalPulsarTestContext(config2); PulsarService pulsar2 = pulsarTestContext2.getPulsarService(); - conf.setBrokerServicePort(Optional.of(2048)); - conf.setBrokerServicePortTls(Optional.of(2049)); - conf.setWebServicePort(Optional.of(2050)); - conf.setWebServicePortTls(Optional.of(2051)); + ServiceConfiguration config3 = super.getDefaultConf(); @Cleanup - PulsarTestContext pulsarTestContext3 = createAdditionalPulsarTestContext(conf); + PulsarTestContext pulsarTestContext3 = createAdditionalPulsarTestContext(config3); PulsarService pulsar3 = pulsarTestContext.getPulsarService(); @Cleanup PulsarAdmin admin2 = PulsarAdmin.builder().serviceHttpUrl(pulsar2.getWebServiceAddress()).build(); 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 9fb3cb6fb2328..e97707710d743 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 @@ -931,7 +931,7 @@ public void persistentTopics(String topicName) throws Exception { assertEquals(topicStats.getSubscriptions().get(subName).getMsgBacklog(), 10); assertEquals(topicStats.getPublishers().size(), 0); assertEquals(topicStats.getOwnerBroker(), - pulsar.getAdvertisedAddress() + ":" + pulsar.getConfiguration().getWebServicePort().get()); + pulsar.getAdvertisedAddress() + ":" + pulsar.getConfiguration().getWebServicePortTls().get()); PersistentTopicInternalStats internalStats = admin.topics().getInternalStats(persistentTopicName, false); assertEquals(internalStats.cursors.keySet(), Set.of(Codec.encode(subName))); 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 c32d3fc3b0b27..fd8ef0f40b115 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 @@ -221,10 +221,8 @@ protected void doInitConf() throws Exception { this.conf.setBrokerShutdownTimeoutMs(0L); this.conf.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); this.conf.setBrokerServicePort(Optional.of(0)); - this.conf.setBrokerServicePortTls(Optional.of(0)); this.conf.setAdvertisedAddress("localhost"); this.conf.setWebServicePort(Optional.of(0)); - this.conf.setWebServicePortTls(Optional.of(0)); this.conf.setNumExecutorThreadPoolSize(5); this.conf.setExposeBundlesMetricsInPrometheus(true); } @@ -464,9 +462,7 @@ protected ServiceConfiguration getDefaultConf() { configuration.setBrokerShutdownTimeoutMs(0L); configuration.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); configuration.setBrokerServicePort(Optional.of(0)); - configuration.setBrokerServicePortTls(Optional.of(0)); configuration.setWebServicePort(Optional.of(0)); - configuration.setWebServicePortTls(Optional.of(0)); configuration.setBookkeeperClientExposeStatsToPrometheus(true); configuration.setNumExecutorThreadPoolSize(5); configuration.setBrokerMaxConnections(0); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AdvertisedListenersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AdvertisedListenersTest.java index 7a8154312e4dc..a88ccd60ae4c4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AdvertisedListenersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AdvertisedListenersTest.java @@ -78,7 +78,6 @@ private void updateConfig(ServiceConfiguration conf, String advertisedAddress) { ",public_https:https://localhost:" + httpsPort); conf.setBrokerServicePort(Optional.of(pulsarPort)); conf.setWebServicePort(Optional.of(httpPort)); - conf.setWebServicePortTls(Optional.of(httpsPort)); } @Test @@ -101,7 +100,6 @@ public void testLookup() throws Exception { assertEquals(new URI(ld.getBrokerUrl()).getHost(), "localhost"); assertEquals(new URI(ld.getHttpUrl()).getHost(), "localhost"); - assertEquals(new URI(ld.getHttpUrlTls()).getHost(), "localhost"); // Produce data diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java index 68902c73e5717..7cc4499df9710 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java @@ -124,7 +124,6 @@ void setup() throws Exception { config.setAdvertisedAddress("localhost"); config.setWebServicePort(Optional.of(0)); config.setBrokerServicePortTls(Optional.of(0)); - config.setWebServicePortTls(Optional.of(0)); config.setMetadataStoreUrl("zk:127.0.0.1:" + bkEnsemble.getZookeeperPort()); config.setBrokerShutdownTimeoutMs(0L); config.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java index c4898786e3e03..6303c70b4dc77 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java @@ -113,7 +113,6 @@ void setup() throws Exception { config1.setBrokerServicePort(Optional.of(0)); config1.setLoadManagerClassName(SimpleLoadManagerImpl.class.getName()); config1.setBrokerServicePortTls(Optional.of(0)); - config1.setWebServicePortTls(Optional.of(0)); config1.setAdvertisedAddress("localhost"); pulsar1 = new PulsarService(config1); pulsar1.start(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java index 786c9027c944d..d8acb6d24e9ef 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java @@ -169,8 +169,6 @@ void setup() throws Exception { config1.setBrokerShutdownTimeoutMs(0L); config1.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); config1.setBrokerServicePort(Optional.of(0)); - config1.setBrokerServicePortTls(Optional.of(0)); - config1.setWebServicePortTls(Optional.of(0)); pulsar1 = new PulsarService(config1); pulsar1.start(); @@ -189,8 +187,6 @@ void setup() throws Exception { config2.setBrokerShutdownTimeoutMs(0L); config2.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); config2.setBrokerServicePort(Optional.of(0)); - config2.setBrokerServicePortTls(Optional.of(0)); - config2.setWebServicePortTls(Optional.of(0)); pulsar2 = new PulsarService(config2); pulsar2.start(); @@ -204,8 +200,6 @@ void setup() throws Exception { config.setBrokerShutdownTimeoutMs(0L); config.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); config.setBrokerServicePort(Optional.of(0)); - config.setBrokerServicePortTls(Optional.of(0)); - config.setWebServicePortTls(Optional.of(0)); pulsar3 = new PulsarService(config); secondaryHost = String.format("%s:%d", "localhost", pulsar2.getListenPortHTTP().get()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipForCurrentServerTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipForCurrentServerTestBase.java index 8dd4f53db8240..46e8989ac3df4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipForCurrentServerTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipForCurrentServerTestBase.java @@ -80,10 +80,8 @@ protected void startBroker() throws Exception { conf.setBrokerShutdownTimeoutMs(0L); conf.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); conf.setBrokerServicePort(Optional.of(0)); - conf.setBrokerServicePortTls(Optional.of(0)); conf.setAdvertisedAddress("localhost"); conf.setWebServicePort(Optional.of(0)); - conf.setWebServicePortTls(Optional.of(0)); serviceConfigurationList.add(conf); PulsarTestContext.Builder testContextBuilder = 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 6c0bc6efec758..1c24146b18440 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 @@ -1786,4 +1786,13 @@ public void testUnsubscribeNonDurableSub() throws Exception { fail("Unsubscribe failed"); } } + + @Test + public void testGetLookupServiceAddress() throws Exception { + cleanup(); + setup(); + conf.setWebServicePortTls(Optional.of(8081)); + assertEquals(pulsar.getLookupServiceAddress(), "localhost:8081"); + resetState(); + } } 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 469e155d409b3..76209294a0e22 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 @@ -28,6 +28,7 @@ import java.lang.reflect.Method; import java.net.URL; +import java.util.Optional; import java.util.concurrent.TimeUnit; import org.apache.pulsar.broker.BrokerTestUtil; @@ -477,6 +478,14 @@ protected void setup() throws Exception { super.setupWithClusterName(clusterName); } + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + this.conf.setWebServicePortTls(Optional.of(0)); + this.conf.setBrokerServicePortTls(Optional.of(0)); + } + + public PulsarService getPulsarService() { return pulsar; } 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 f710c8541d1b5..1139bb9e0bfb7 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 @@ -247,7 +247,7 @@ public void activeBrokerParse() throws Exception { -> admin2.clusters().getCluster(cluster2) != null); List list = admin1.brokers().getActiveBrokers(cluster2); - assertEquals(list.get(0), url2.toString().replace("http://", "")); + assertEquals(list.get(0), urlTls2.toString().replace("https://", "")); //restore configuration pulsar1.getConfiguration().setAuthorizationEnabled(false); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTestBase.java index c0300c63b3587..1ff835732aab5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTestBase.java @@ -157,10 +157,8 @@ protected void startBroker() throws Exception { conf.setBrokerShutdownTimeoutMs(0L); conf.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); conf.setBrokerServicePort(Optional.of(0)); - conf.setBrokerServicePortTls(Optional.of(0)); conf.setAdvertisedAddress("localhost"); conf.setWebServicePort(Optional.of(0)); - conf.setWebServicePortTls(Optional.of(0)); conf.setTransactionCoordinatorEnabled(true); conf.setBrokerDeduplicationEnabled(true); conf.setTransactionBufferSnapshotMaxTransactionCount(2); 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 792f419ee997e..accdd2a335f39 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 @@ -474,7 +474,7 @@ public void testWebserviceServiceTls() throws Exception { // request [3] doReturn(true).when(loadManager1).isCentralized(); doReturn(true).when(loadManager2).isCentralized(); - SimpleResourceUnit resourceUnit = new SimpleResourceUnit(pulsar.getWebServiceAddress(), null); + SimpleResourceUnit resourceUnit = new SimpleResourceUnit(pulsar.getWebServiceAddressTls(), null); doReturn(Optional.of(resourceUnit)).when(loadManager2).getLeastLoaded(any(ServiceUnitId.class)); doReturn(Optional.of(resourceUnit)).when(loadManager1).getLeastLoaded(any(ServiceUnitId.class)); @@ -507,6 +507,9 @@ public void testWebserviceServiceTls() throws Exception { loadManager1 = null; loadManager2 = null; + + conf.setBrokerServicePortTls(Optional.empty()); + conf.setWebServicePortTls(Optional.empty()); } /** @@ -840,6 +843,8 @@ public void testPartitionedMetadataWithDeprecatedVersion() throws Exception { admin.topics().createPartitionedTopic(dest.toString(), totalPartitions); stopBroker(); + conf.setBrokerServicePortTls(Optional.empty()); + conf.setWebServicePortTls(Optional.empty()); conf.setClientLibraryVersionCheckEnabled(true); startBroker(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientAuthenticationTlsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientAuthenticationTlsTest.java index c9b243257c4e1..d716d5a806392 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientAuthenticationTlsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientAuthenticationTlsTest.java @@ -22,6 +22,7 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.expectThrows; import java.util.HashSet; +import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; @@ -50,7 +51,8 @@ protected void doInitConf() throws Exception { Set providers = new HashSet<>(); providers.add(AuthenticationProviderTls.class.getName()); conf.setAuthenticationProviders(providers); - + conf.setWebServicePortTls(Optional.of(0)); + conf.setBrokerServicePortTls(Optional.of(0)); conf.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); conf.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); conf.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java index bde989fc432f9..2f36cc679f1f2 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java @@ -69,6 +69,7 @@ protected void doInitConf() throws Exception { conf.setAdvertisedAddress(null); conf.setAuthenticateOriginalAuthData(true); conf.setBrokerServicePort(Optional.of(0)); + conf.setWebServicePortTls(Optional.of(0)); conf.setWebServicePort(Optional.of(0)); Set superUserRoles = new HashSet<>(); 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 01c06fbf52f4e..6247c2a66e874 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 @@ -75,6 +75,7 @@ protected void setup() throws Exception { protected void doInitConf() throws Exception { super.doInitConf(); + this.conf.setBrokerServicePortTls(Optional.of(0)); this.conf.setTlsCertificateFilePath(PROXY_CERT_FILE_PATH); this.conf.setTlsKeyFilePath(PROXY_KEY_FILE_PATH); } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/BrokerContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/BrokerContainer.java index 616d45554d75c..a51397050b97f 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/BrokerContainer.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/BrokerContainer.java @@ -28,8 +28,13 @@ public class BrokerContainer extends PulsarContainer { public static final String NAME = "pulsar-broker"; public BrokerContainer(String clusterName, String hostName) { - super(clusterName, hostName, hostName, "bin/run-broker.sh", BROKER_PORT, BROKER_PORT_TLS, - BROKER_HTTP_PORT, BROKER_HTTPS_PORT, DEFAULT_HTTP_PATH, DEFAULT_IMAGE_NAME); + this(clusterName, hostName, false); + } + + public BrokerContainer(String clusterName, String hostName, boolean enableTls) { + super(clusterName, hostName, hostName, "bin/run-broker.sh", BROKER_PORT, + enableTls ? BROKER_PORT_TLS : 0, BROKER_HTTP_PORT, + enableTls ? BROKER_HTTPS_PORT : 0, DEFAULT_HTTP_PATH, DEFAULT_IMAGE_NAME); tailContainerLog(); } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/ProxyContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/ProxyContainer.java index 53283447378f5..f3926878f37c5 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/ProxyContainer.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/ProxyContainer.java @@ -28,8 +28,13 @@ public class ProxyContainer extends PulsarContainer { public static final String NAME = "pulsar-proxy"; public ProxyContainer(String clusterName, String hostName) { - super(clusterName, hostName, hostName, "bin/run-proxy.sh", BROKER_PORT, BROKER_PORT_TLS, BROKER_HTTP_PORT, - BROKER_HTTPS_PORT, DEFAULT_HTTP_PATH, DEFAULT_IMAGE_NAME); + this(clusterName, hostName, false); + } + + public ProxyContainer(String clusterName, String hostName, boolean enableTls) { + super(clusterName, hostName, hostName, "bin/run-proxy.sh", BROKER_PORT, + enableTls ? BROKER_PORT_TLS : 0, BROKER_HTTP_PORT, + enableTls ? BROKER_HTTPS_PORT : 0, DEFAULT_HTTP_PATH, DEFAULT_IMAGE_NAME); } @Override diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/tls/ClientTlsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/tls/ClientTlsTest.java index 59ff978cafa06..080912cd49262 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/tls/ClientTlsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/tls/ClientTlsTest.java @@ -29,6 +29,7 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.tests.integration.suites.PulsarTestSuite; +import org.apache.pulsar.tests.integration.topologies.PulsarClusterSpec; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -41,6 +42,14 @@ private static String loadCertificateAuthorityFile(String name) { return Resources.getResource("certificate-authority/" + name).getPath(); } + @Override + protected PulsarClusterSpec.PulsarClusterSpecBuilder beforeSetupCluster( + String clusterName, + PulsarClusterSpec.PulsarClusterSpecBuilder specBuilder) { + specBuilder.enableTls(true); + return specBuilder; + } + @DataProvider(name = "adminUrls") public Object[][] adminUrls() { return new Object[][]{ 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 9b4823f46d4cc..769f135599b01 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 @@ -38,6 +38,7 @@ import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.commons.io.IOUtils; +import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.tests.integration.containers.BKContainer; import org.apache.pulsar.tests.integration.containers.BrokerContainer; import org.apache.pulsar.tests.integration.containers.CSContainer; @@ -132,14 +133,16 @@ private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean s this.brokerContainers = Maps.newTreeMap(); this.workerContainers = Maps.newTreeMap(); - this.proxyContainer = new ProxyContainer(appendClusterName("pulsar-proxy"), ProxyContainer.NAME) + this.proxyContainer = new ProxyContainer(appendClusterName("pulsar-proxy"), ProxyContainer.NAME, spec.enableTls) .withNetwork(network) .withNetworkAliases(appendClusterName("pulsar-proxy")) .withEnv("zkServers", appendClusterName(ZKContainer.NAME)) .withEnv("zookeeperServers", appendClusterName(ZKContainer.NAME)) .withEnv("configurationStoreServers", CSContainer.NAME + ":" + CS_PORT) - .withEnv("clusterName", clusterName) + .withEnv("clusterName", clusterName); // enable mTLS + if (spec.enableTls) { + proxyContainer .withEnv("webServicePortTls", String.valueOf(BROKER_HTTPS_PORT)) .withEnv("servicePortTls", String.valueOf(BROKER_PORT_TLS)) .withEnv("forwardAuthorizationCredentials", "true") @@ -147,7 +150,15 @@ private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean s .withEnv("tlsAllowInsecureConnection", "false") .withEnv("tlsCertificateFilePath", "/pulsar/certificate-authority/server-keys/proxy.cert.pem") .withEnv("tlsKeyFilePath", "/pulsar/certificate-authority/server-keys/proxy.key-pk8.pem") - .withEnv("tlsTrustCertsFilePath", "/pulsar/certificate-authority/certs/ca.cert.pem"); + .withEnv("tlsTrustCertsFilePath", "/pulsar/certificate-authority/certs/ca.cert.pem") + .withEnv("brokerClientAuthenticationPlugin", AuthenticationTls.class.getName()) + .withEnv("brokerClientAuthenticationParameters", String.format("tlsCertFile:%s,tlsKeyFile:%s", "/pulsar/certificate-authority/client-keys/admin.cert.pem", "/pulsar/certificate-authority/client-keys/admin.key-pk8.pem")) + .withEnv("tlsEnabledWithBroker", "true") + .withEnv("brokerClientTrustCertsFilePath", "/pulsar/certificate-authority/certs/ca.cert.pem") + .withEnv("brokerClientCertificateFilePath", "/pulsar/certificate-authority/server-keys/proxy.cert.pem") + .withEnv("brokerClientKeyFilePath", "/pulsar/certificate-authority/server-keys/proxy.key-pk8.pem"); + + } if (spec.proxyEnvs != null) { spec.proxyEnvs.forEach(this.proxyContainer::withEnv); } @@ -184,7 +195,7 @@ private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean s // create brokers brokerContainers.putAll( runNumContainers("broker", spec.numBrokers(), (name) -> { - BrokerContainer brokerContainer = new BrokerContainer(clusterName, appendClusterName(name)) + BrokerContainer brokerContainer = new BrokerContainer(clusterName, appendClusterName(name), spec.enableTls) .withNetwork(network) .withNetworkAliases(appendClusterName(name)) .withEnv("zkServers", appendClusterName(ZKContainer.NAME)) @@ -195,16 +206,19 @@ private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean s .withEnv("loadBalancerOverrideBrokerNicSpeedGbps", "1") // used in s3 tests .withEnv("AWS_ACCESS_KEY_ID", "accesskey").withEnv("AWS_SECRET_KEY", "secretkey") - .withEnv("maxMessageSize", "" + spec.maxMessageSize) + .withEnv("maxMessageSize", "" + spec.maxMessageSize); + if (spec.enableTls) { // enable mTLS - .withEnv("webServicePortTls", String.valueOf(BROKER_HTTPS_PORT)) - .withEnv("brokerServicePortTls", String.valueOf(BROKER_PORT_TLS)) - .withEnv("authenticateOriginalAuthData", "true") - .withEnv("tlsRequireTrustedClientCertOnConnect", "true") - .withEnv("tlsAllowInsecureConnection", "false") - .withEnv("tlsCertificateFilePath", "/pulsar/certificate-authority/server-keys/broker.cert.pem") - .withEnv("tlsKeyFilePath", "/pulsar/certificate-authority/server-keys/broker.key-pk8.pem") - .withEnv("tlsTrustCertsFilePath", "/pulsar/certificate-authority/certs/ca.cert.pem"); + brokerContainer + .withEnv("webServicePortTls", String.valueOf(BROKER_HTTPS_PORT)) + .withEnv("brokerServicePortTls", String.valueOf(BROKER_PORT_TLS)) + .withEnv("authenticateOriginalAuthData", "true") + .withEnv("tlsAllowInsecureConnection", "false") + .withEnv("tlsRequireTrustedClientCertOnConnect", "true") + .withEnv("tlsTrustCertsFilePath", "/pulsar/certificate-authority/certs/ca.cert.pem") + .withEnv("tlsCertificateFilePath", "/pulsar/certificate-authority/server-keys/broker.cert.pem") + .withEnv("tlsKeyFilePath", "/pulsar/certificate-authority/server-keys/broker.key-pk8.pem"); + } if (spec.queryLastMessage) { brokerContainer.withEnv("bookkeeperExplicitLacIntervalInMills", "10"); brokerContainer.withEnv("bookkeeperUseV2WireProtocol", "false"); 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 fa28d20e6b356..c141e990d62e0 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 @@ -177,4 +177,10 @@ public class PulsarClusterSpec { * Additional ports to expose on bookie containers. */ List bookieAdditionalPorts; + + /** + * Enable TLS for connection. + */ + @Default + boolean enableTls = false; } From 9828326f08038c5789a28b548d2285678f56c1a1 Mon Sep 17 00:00:00 2001 From: Yan Zhao Date: Mon, 18 Sep 2023 23:29:49 +0800 Subject: [PATCH 062/699] [fix] [auto-recovery] Fix PulsarLedgerUnderreplicationManager notify problem. (#21161) --- pulsar-metadata/pom.xml | 2 - .../PulsarLedgerUnderreplicationManager.java | 64 ++-- .../replication/AuditorLedgerCheckerTest.java | 312 ++++++++++++++++++ .../LedgerUnderreplicationManagerTest.java | 19 +- 4 files changed, 371 insertions(+), 26 deletions(-) create mode 100644 pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java diff --git a/pulsar-metadata/pom.xml b/pulsar-metadata/pom.xml index c79431ffdd6f0..3b9e432f5135c 100644 --- a/pulsar-metadata/pom.xml +++ b/pulsar-metadata/pom.xml @@ -57,7 +57,6 @@ - io.dropwizard.metrics metrics-core @@ -85,7 +84,6 @@ - org.xerial.snappy snappy-java diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerUnderreplicationManager.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerUnderreplicationManager.java index aaa247bfb32fc..2673328b81139 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerUnderreplicationManager.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerUnderreplicationManager.java @@ -105,14 +105,17 @@ long getLedgerNodeVersion() { private final String urLockPath; private final String layoutPath; private final String lostBookieRecoveryDelayPath; + private final String replicationDisablePath; private final String checkAllLedgersCtimePath; private final String placementPolicyCheckCtimePath; private final String replicasCheckCtimePath; private final MetadataStoreExtended store; - private BookkeeperInternalCallbacks.GenericCallback replicationEnabledListener; - private BookkeeperInternalCallbacks.GenericCallback lostBookieRecoveryDelayListener; + private final List> replicationEnabledCallbacks = + new ArrayList<>(); + private final List> lostBookieRecoveryDelayCallbacks = + new ArrayList<>(); private static class PulsarUnderreplicatedLedger extends UnderreplicatedLedger { PulsarUnderreplicatedLedger(long ledgerId) { @@ -139,6 +142,7 @@ public PulsarLedgerUnderreplicationManager(AbstractConfiguration conf, Metada urLedgerPath = basePath + BookKeeperConstants.DEFAULT_ZK_LEDGERS_ROOT_PATH; urLockPath = basePath + '/' + BookKeeperConstants.UNDER_REPLICATION_LOCK; lostBookieRecoveryDelayPath = basePath + '/' + BookKeeperConstants.LOSTBOOKIERECOVERYDELAY_NODE; + replicationDisablePath = basePath + '/' + BookKeeperConstants.DISABLE_NODE; checkAllLedgersCtimePath = basePath + '/' + BookKeeperConstants.CHECK_ALL_LEDGERS_CTIME; placementPolicyCheckCtimePath = basePath + '/' + BookKeeperConstants.PLACEMENT_POLICY_CHECK_CTIME; replicasCheckCtimePath = basePath + '/' + BookKeeperConstants.REPLICAS_CHECK_CTIME; @@ -232,17 +236,34 @@ private void handleNotification(Notification n) { synchronized (this) { // Notify that there were some changes on the under-replicated z-nodes notifyAll(); - - if (n.getType() == NotificationType.Deleted) { - if (n.getPath().equals(basePath + '/' + BookKeeperConstants.DISABLE_NODE)) { - log.info("LedgerReplication is enabled externally through MetadataStore, " - + "since DISABLE_NODE ZNode is deleted"); - if (replicationEnabledListener != null) { - replicationEnabledListener.operationComplete(0, null); + if (lostBookieRecoveryDelayPath.equals(n.getPath())) { + final List> callbackList; + synchronized (lostBookieRecoveryDelayCallbacks) { + callbackList = new ArrayList<>(lostBookieRecoveryDelayCallbacks); + lostBookieRecoveryDelayCallbacks.clear(); + } + for (BookkeeperInternalCallbacks.GenericCallback callback : callbackList) { + try { + callback.operationComplete(0, null); + } catch (Exception e) { + log.warn("lostBookieRecoveryDelayCallbacks handle error", e); } - } else if (n.getPath().equals(lostBookieRecoveryDelayPath)) { - if (lostBookieRecoveryDelayListener != null) { - lostBookieRecoveryDelayListener.operationComplete(0, null); + } + return; + } + if (replicationDisablePath.equals(n.getPath()) && n.getType() == NotificationType.Deleted) { + log.info("LedgerReplication is enabled externally through MetadataStore, " + + "since DISABLE_NODE ZNode is deleted"); + final List> callbackList; + synchronized (replicationEnabledCallbacks) { + callbackList = new ArrayList<>(replicationEnabledCallbacks); + replicationEnabledCallbacks.clear(); + } + for (BookkeeperInternalCallbacks.GenericCallback callback : callbackList) { + try { + callback.operationComplete(0, null); + } catch (Exception e) { + log.warn("replicationEnabledCallbacks handle error", e); } } } @@ -688,8 +709,7 @@ public void disableLedgerReplication() log.debug("disableLedegerReplication()"); } try { - String path = basePath + '/' + BookKeeperConstants.DISABLE_NODE; - store.put(path, "".getBytes(UTF_8), Optional.of(-1L)) + store.put(replicationDisablePath, "".getBytes(UTF_8), Optional.of(-1L)) .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); log.info("Auto ledger re-replication is disabled!"); } catch (ExecutionException | TimeoutException ee) { @@ -710,7 +730,7 @@ public void enableLedgerReplication() log.debug("enableLedegerReplication()"); } try { - store.delete(basePath + '/' + BookKeeperConstants.DISABLE_NODE, Optional.empty()) + store.delete(replicationDisablePath, Optional.empty()) .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); log.info("Resuming automatic ledger re-replication"); } catch (ExecutionException | TimeoutException ee) { @@ -731,7 +751,7 @@ public boolean isLedgerReplicationEnabled() log.debug("isLedgerReplicationEnabled()"); } try { - return !store.exists(basePath + '/' + BookKeeperConstants.DISABLE_NODE) + return !store.exists(replicationDisablePath) .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); } catch (ExecutionException | TimeoutException ee) { log.error("Error while checking the state of " @@ -751,13 +771,11 @@ public void notifyLedgerReplicationEnabled(final BookkeeperInternalCallbacks.Gen if (log.isDebugEnabled()) { log.debug("notifyLedgerReplicationEnabled()"); } - - synchronized (this) { - replicationEnabledListener = cb; + synchronized (replicationEnabledCallbacks) { + replicationEnabledCallbacks.add(cb); } - try { - if (!store.exists(basePath + '/' + BookKeeperConstants.DISABLE_NODE) + if (!store.exists(replicationDisablePath) .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS)) { log.info("LedgerReplication is enabled externally through metadata store, " + "since DISABLE_NODE node is deleted"); @@ -851,8 +869,8 @@ public int getLostBookieRecoveryDelay() throws ReplicationException.UnavailableE public void notifyLostBookieRecoveryDelayChanged(BookkeeperInternalCallbacks.GenericCallback cb) throws ReplicationException.UnavailableException { log.debug("notifyLostBookieRecoveryDelayChanged()"); - synchronized (this) { - lostBookieRecoveryDelayListener = cb; + synchronized (lostBookieRecoveryDelayCallbacks) { + lostBookieRecoveryDelayCallbacks.add(cb); } try { if (!store.exists(lostBookieRecoveryDelayPath).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS)) { diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java new file mode 100644 index 0000000000000..d394e4ae7d1d5 --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java @@ -0,0 +1,312 @@ +/* + * 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.replication; + +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertFalse; +import static org.testng.AssertJUnit.assertNotNull; +import static org.testng.AssertJUnit.assertNotSame; +import static org.testng.AssertJUnit.assertTrue; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import org.apache.bookkeeper.client.AsyncCallback.AddCallback; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper.DigestType; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; +import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.proto.BookieServer; +import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +/** + * Tests publishing of under replicated ledgers by the Auditor bookie node when + * corresponding bookies identifes as not running. + */ +public class AuditorLedgerCheckerTest extends BookKeeperClusterTestCase { + + // Depending on the taste, select the amount of logging + // by decommenting one of the two lines below + // private static final Logger LOG = Logger.getRootLogger(); + private static final Logger LOG = LoggerFactory + .getLogger(AuditorLedgerCheckerTest.class); + + private static final byte[] ledgerPassword = "aaa".getBytes(); + private Random rng; // Random Number Generator + + private DigestType digestType; + + private String underreplicatedPath; + private Map auditorElectors = new ConcurrentHashMap<>(); + private ZkLedgerUnderreplicationManager urLedgerMgr; + private Set urLedgerList; + private String electionPath; + + private List ledgerList; + + public AuditorLedgerCheckerTest() + throws IOException, KeeperException, InterruptedException, + CompatibilityException { + this("org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory"); + } + + AuditorLedgerCheckerTest(String ledgerManagerFactoryClass) + throws IOException, KeeperException, InterruptedException, + CompatibilityException { + super(3); + LOG.info("Running test case using ledger manager : " + + ledgerManagerFactoryClass); + this.digestType = DigestType.CRC32; + // set ledger manager name + baseConf.setLedgerManagerFactoryClassName(ledgerManagerFactoryClass); + baseClientConf + .setLedgerManagerFactoryClassName(ledgerManagerFactoryClass); + } + + @BeforeMethod + public void setUp() throws Exception { + super.setUp(); + underreplicatedPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(baseClientConf) + + "/underreplication/ledgers"; + electionPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(baseConf) + + "/underreplication/auditorelection"; + + urLedgerMgr = new ZkLedgerUnderreplicationManager(baseClientConf, zkc); + urLedgerMgr.setCheckAllLedgersCTime(System.currentTimeMillis()); + startAuditorElectors(); + rng = new Random(System.currentTimeMillis()); // Initialize the Random + urLedgerList = new HashSet(); + ledgerList = new ArrayList(2); + baseClientConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri()); + baseConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri()); + } + + @Override + public void tearDown() throws Exception { + stopAuditorElectors(); + super.tearDown(); + } + + private void startAuditorElectors() throws Exception { + for (String addr : bookieAddresses().stream().map(Object::toString) + .collect(Collectors.toList())) { + AuditorElector auditorElector = new AuditorElector(addr, baseConf); + auditorElectors.put(addr, auditorElector); + auditorElector.start(); + if (LOG.isDebugEnabled()) { + LOG.debug("Starting Auditor Elector"); + } + } + } + + private void stopAuditorElectors() throws Exception { + for (AuditorElector auditorElector : auditorElectors.values()) { + auditorElector.shutdown(); + if (LOG.isDebugEnabled()) { + LOG.debug("Stopping Auditor Elector!"); + } + } + } + + @Test + public void testTriggerAuditorBySettingDelayToZeroWithPendingAuditTask() throws Exception { + // wait for a second so that the initial periodic check finishes + Thread.sleep(1000); + + Auditor auditorBookiesAuditor = getAuditorBookiesAuditor(); + LedgerHandle lh1 = createAndAddEntriesToLedger(); + Long ledgerId = lh1.getId(); + if (LOG.isDebugEnabled()) { + LOG.debug("Created ledger : " + ledgerId); + } + ledgerList.add(ledgerId); + lh1.close(); + + final CountDownLatch underReplicaLatch = registerUrLedgerWatcher(ledgerList + .size()); + + int lostBookieRecoveryDelay = 5; + // wait for 5 seconds before starting the recovery work when a bookie fails + urLedgerMgr.setLostBookieRecoveryDelay(lostBookieRecoveryDelay); + + // shutdown a non auditor bookie; choosing non-auditor to avoid another election + String shutdownBookie = shutDownNonAuditorBookie(); + + if (LOG.isDebugEnabled()) { + LOG.debug("Waiting for ledgers to be marked as under replicated"); + } + assertFalse("audit of lost bookie isn't delayed", underReplicaLatch.await(2, TimeUnit.SECONDS)); + assertEquals("under replicated ledgers identified when it was not expected", 0, + urLedgerList.size()); + + Future auditTask = auditorBookiesAuditor.getAuditTask(); + assertNotSame("auditTask is not supposed to be null", null, auditTask); + assertEquals( + "lostBookieRecoveryDelayBeforeChange of Auditor should be equal to what we set", + lostBookieRecoveryDelay, auditorBookiesAuditor.getLostBookieRecoveryDelayBeforeChange()); + + // set lostBookieRecoveryDelay to 0, so that Auditor is triggered immediately + urLedgerMgr.setLostBookieRecoveryDelay(0); + assertTrue("audit of lost bookie shouldn't be delayed", underReplicaLatch.await(1, TimeUnit.SECONDS)); + assertEquals("all under replicated ledgers should be identified", ledgerList.size(), + urLedgerList.size()); + + Thread.sleep(100); + auditTask = auditorBookiesAuditor.getAuditTask(); + assertEquals("auditTask is supposed to be null", null, auditTask); + assertEquals( + "lostBookieRecoveryDelayBeforeChange of Auditor should be equal to previously set value", + 0, auditorBookiesAuditor.getLostBookieRecoveryDelayBeforeChange()); + } + + private CountDownLatch registerUrLedgerWatcher(int count) + throws KeeperException, InterruptedException { + final CountDownLatch underReplicaLatch = new CountDownLatch(count); + for (Long ledgerId : ledgerList) { + Watcher urLedgerWatcher = new ChildWatcher(underReplicaLatch); + String znode = ZkLedgerUnderreplicationManager.getUrLedgerZnode(underreplicatedPath, + ledgerId); + zkc.exists(znode, urLedgerWatcher); + } + return underReplicaLatch; + } + + private String shutdownBookie(int bkShutdownIndex) throws Exception { + BookieServer bkServer = serverByIndex(bkShutdownIndex); + String bookieAddr = bkServer.getBookieId().toString(); + if (LOG.isDebugEnabled()) { + LOG.debug("Shutting down bookie:" + bookieAddr); + } + killBookie(bkShutdownIndex); + auditorElectors.get(bookieAddr).shutdown(); + auditorElectors.remove(bookieAddr); + return bookieAddr; + } + + private LedgerHandle createAndAddEntriesToLedger() throws BKException, + InterruptedException { + int numEntriesToWrite = 100; + // Create a ledger + LedgerHandle lh = bkc.createLedger(digestType, ledgerPassword); + LOG.info("Ledger ID: " + lh.getId()); + addEntry(numEntriesToWrite, lh); + return lh; + } + + private void addEntry(int numEntriesToWrite, LedgerHandle lh) + throws InterruptedException, BKException { + final CountDownLatch completeLatch = new CountDownLatch(numEntriesToWrite); + final AtomicInteger rc = new AtomicInteger(BKException.Code.OK); + + for (int i = 0; i < numEntriesToWrite; i++) { + ByteBuffer entry = ByteBuffer.allocate(4); + entry.putInt(rng.nextInt(Integer.MAX_VALUE)); + entry.position(0); + lh.asyncAddEntry(entry.array(), new AddCallback() { + public void addComplete(int rc2, LedgerHandle lh, long entryId, Object ctx) { + rc.compareAndSet(BKException.Code.OK, rc2); + completeLatch.countDown(); + } + }, null); + } + completeLatch.await(); + if (rc.get() != BKException.Code.OK) { + throw BKException.create(rc.get()); + } + + } + + private class ChildWatcher implements Watcher { + private final CountDownLatch underReplicaLatch; + + public ChildWatcher(CountDownLatch underReplicaLatch) { + this.underReplicaLatch = underReplicaLatch; + } + + @Override + public void process(WatchedEvent event) { + LOG.info("Received notification for the ledger path : " + + event.getPath()); + for (Long ledgerId : ledgerList) { + if (event.getPath().contains(ledgerId + "")) { + urLedgerList.add(ledgerId); + } + } + if (LOG.isDebugEnabled()) { + LOG.debug("Count down and waiting for next notification"); + } + // count down and waiting for next notification + underReplicaLatch.countDown(); + } + } + + private BookieServer getAuditorBookie() throws Exception { + List auditors = new LinkedList(); + byte[] data = zkc.getData(electionPath, false, null); + assertNotNull("Auditor election failed", data); + for (int i = 0; i < bookieCount(); i++) { + BookieId bookieId = addressByIndex(i); + if (new String(data).contains(bookieId + "")) { + auditors.add(serverByIndex(i)); + } + } + assertEquals("Multiple Bookies acting as Auditor!", 1, auditors + .size()); + return auditors.get(0); + } + + private Auditor getAuditorBookiesAuditor() throws Exception { + BookieServer auditorBookieServer = getAuditorBookie(); + String bookieAddr = auditorBookieServer.getBookieId().toString(); + return auditorElectors.get(bookieAddr).auditor; + } + + private String shutDownNonAuditorBookie() throws Exception { + // shutdown bookie which is not an auditor + int indexOf = indexOfServer(getAuditorBookie()); + int bkIndexDownBookie; + if (indexOf < lastBookieIndex()) { + bkIndexDownBookie = indexOf + 1; + } else { + bkIndexDownBookie = indexOf - 1; + } + return shutdownBookie(bkIndexDownBookie); + } +} 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 649dc1663c68f..0e9c781fb9143 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 @@ -40,6 +40,7 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; import lombok.Cleanup; @@ -614,6 +615,8 @@ public void testDisableLedgerReplication(String provider, Supplier urlSu final String missingReplica = "localhost:3181"; // disabling replication + AtomicInteger callbackCount = new AtomicInteger(); + lum.notifyLedgerReplicationEnabled((rc, result) -> callbackCount.incrementAndGet()); lum.disableLedgerReplication(); log.info("Disabled Ledeger Replication"); @@ -631,6 +634,7 @@ public void testDisableLedgerReplication(String provider, Supplier urlSu } catch (TimeoutException te) { // expected behaviour, as the replication is disabled } + assertEquals(callbackCount.get(), 1, "Notify callback times mismatch"); } /** @@ -651,7 +655,8 @@ public void testEnableLedgerReplication(String provider, Supplier urlSup log.debug("Unexpected exception while marking urLedger", e); fail("Unexpected exception while marking urLedger" + e.getMessage()); } - + AtomicInteger callbackCount = new AtomicInteger(); + lum.notifyLedgerReplicationEnabled((rc, result) -> callbackCount.incrementAndGet()); // disabling replication lum.disableLedgerReplication(); log.debug("Disabled Ledeger Replication"); @@ -688,6 +693,7 @@ public void testEnableLedgerReplication(String provider, Supplier urlSup znodeLatch.await(5, TimeUnit.SECONDS); log.debug("Enabled Ledeger Replication"); assertEquals(znodeLatch.getCount(), 0, "Failed to disable ledger replication!"); + assertEquals(callbackCount.get(), 2, "Notify callback times mismatch"); } finally { thread1.interrupt(); } @@ -749,6 +755,17 @@ public void testReplicasCheckCTime(String provider, Supplier urlSupplier assertEquals(underReplicaMgr1.getReplicasCheckCTime(), curTime); } + @Test(timeOut = 60000, dataProvider = "impl") + public void testLostBookieRecoveryDelay(String provider, Supplier urlSupplier) throws Exception { + methodSetup(urlSupplier); + + AtomicInteger callbackCount = new AtomicInteger(); + lum.notifyLostBookieRecoveryDelayChanged((rc, result) -> callbackCount.incrementAndGet()); + // disabling replication + lum.setLostBookieRecoveryDelay(10); + Awaitility.await().until(() -> callbackCount.get() == 2); + } + private void verifyMarkLedgerUnderreplicated(Collection missingReplica) throws Exception { Long ledgerA = 0xfeadeefdacL; String znodeA = getUrLedgerZnode(ledgerA); From 1212c7743f2bc3c9696a7703daf56dfceac206eb Mon Sep 17 00:00:00 2001 From: Yan Zhao Date: Tue, 19 Sep 2023 11:47:44 +0800 Subject: [PATCH 063/699] [fix] [auto-recovery] Fix pulsar ledger auditor dead lock problem. (#21181) --- .../PulsarLedgerAuditorManager.java | 13 ++ .../replication/AuditorPeriodicCheckTest.java | 8 +- .../replication/AutoRecoveryMainTest.java | 201 ++++++++++++++++++ 3 files changed, 218 insertions(+), 4 deletions(-) create mode 100644 pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerAuditorManager.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerAuditorManager.java index bc35380fec19c..d664ecdcd2016 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerAuditorManager.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerAuditorManager.java @@ -27,6 +27,7 @@ import org.apache.pulsar.metadata.api.coordination.LeaderElection; import org.apache.pulsar.metadata.api.coordination.LeaderElectionState; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; +import org.apache.pulsar.metadata.api.extended.SessionEvent; import org.apache.pulsar.metadata.coordination.impl.CoordinationServiceImpl; @Slf4j @@ -38,6 +39,7 @@ class PulsarLedgerAuditorManager implements LedgerAuditorManager { private final LeaderElection leaderElection; private LeaderElectionState leaderElectionState; private String bookieId; + private boolean sessionExpired = false; PulsarLedgerAuditorManager(MetadataStoreExtended store, String ledgersRoot) { this.coordinationService = new CoordinationServiceImpl(store); @@ -47,6 +49,14 @@ class PulsarLedgerAuditorManager implements LedgerAuditorManager { this.leaderElection = coordinationService.getLeaderElection(String.class, electionPath, this::handleStateChanges); this.leaderElectionState = LeaderElectionState.NoLeader; + store.registerSessionListener(event -> { + if (SessionEvent.SessionLost == event) { + synchronized (this) { + sessionExpired = true; + notifyAll(); + } + } + }); } private void handleStateChanges(LeaderElectionState state) { @@ -71,6 +81,9 @@ public void tryToBecomeAuditor(String bookieId, Consumer listener) while (true) { try { synchronized (this) { + if (sessionExpired) { + throw new IllegalStateException("Zookeeper session expired, give up to become auditor."); + } if (leaderElectionState == LeaderElectionState.Leading) { return; } else { diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java index c761d46c62266..901361dd3a277 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java @@ -44,8 +44,8 @@ import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.testng.annotations.AfterTest; -import org.testng.annotations.BeforeTest; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; /** @@ -68,7 +68,7 @@ public AuditorPeriodicCheckTest() throws Exception { Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataBookieDriver"); } - @BeforeTest + @BeforeMethod @Override public void setUp() throws Exception { super.setUp(); @@ -99,7 +99,7 @@ public void setUp() throws Exception { driver.initialize(serverConfiguration, NullStatsLogger.INSTANCE); } - @AfterTest + @AfterMethod @Override public void tearDown() throws Exception { if (null != driver) { diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java new file mode 100644 index 0000000000000..d12ee177ece69 --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java @@ -0,0 +1,201 @@ +/* + * 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.replication; + +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertFalse; +import static org.testng.AssertJUnit.assertNotNull; +import static org.testng.AssertJUnit.assertTrue; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.concurrent.TimeUnit; +import org.apache.bookkeeper.bookie.BookieImpl; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.bookkeeper.util.TestUtils; +import org.apache.pulsar.metadata.bookkeeper.PulsarLedgerManagerFactory; +import org.apache.pulsar.metadata.bookkeeper.PulsarMetadataClientDriver; +import org.apache.pulsar.metadata.impl.ZKMetadataStore; +import org.apache.zookeeper.ZooKeeper; +import org.awaitility.Awaitility; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +/** + * Test the AuditorPeer. + */ +public class AutoRecoveryMainTest extends BookKeeperClusterTestCase { + + public AutoRecoveryMainTest() throws Exception { + super(3); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataClientDriver"); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataBookieDriver"); + } + + @BeforeMethod + @Override + public void setUp() throws Exception { + super.setUp(); + } + + /** + * Test that, if an autorecovery looses its ZK connection/session it will + * shutdown. + */ + @Test + public void testAutoRecoverySessionLoss() throws Exception { + confByIndex(0).setMetadataServiceUri( + zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + confByIndex(1).setMetadataServiceUri( + zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + confByIndex(2).setMetadataServiceUri( + zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + /* + * initialize three AutoRecovery instances. + */ + AutoRecoveryMain main1 = new AutoRecoveryMain(confByIndex(0)); + AutoRecoveryMain main2 = new AutoRecoveryMain(confByIndex(1)); + AutoRecoveryMain main3 = new AutoRecoveryMain(confByIndex(2)); + + /* + * start main1, make sure all the components are started and main1 is + * the current Auditor + */ + PulsarMetadataClientDriver pulsarMetadataClientDriver1 = startAutoRecoveryMain(main1); + ZooKeeper zk1 = getZk(pulsarMetadataClientDriver1); + + // Wait until auditor gets elected + for (int i = 0; i < 10; i++) { + try { + if (main1.auditorElector.getCurrentAuditor() != null) { + break; + } else { + Thread.sleep(1000); + } + } catch (IOException e) { + Thread.sleep(1000); + } + } + BookieId currentAuditor = main1.auditorElector.getCurrentAuditor(); + assertNotNull(currentAuditor); + Auditor auditor1 = main1.auditorElector.getAuditor(); + assertEquals("Current Auditor should be AR1", currentAuditor, BookieImpl.getBookieId(confByIndex(0))); + Awaitility.waitAtMost(30, TimeUnit.SECONDS).untilAsserted(() -> { + assertNotNull(auditor1); + assertTrue("Auditor of AR1 should be running", auditor1.isRunning()); + }); + + + /* + * start main2 and main3 + */ + PulsarMetadataClientDriver pulsarMetadataClientDriver2 = startAutoRecoveryMain(main2); + ZooKeeper zk2 = getZk(pulsarMetadataClientDriver2); + + PulsarMetadataClientDriver pulsarMetadataClientDriver3 = startAutoRecoveryMain(main3); + ZooKeeper zk3 = getZk(pulsarMetadataClientDriver3); + + + /* + * make sure AR1 is still the current Auditor and AR2's and AR3's + * auditors are not running. + */ + assertEquals("Current Auditor should still be AR1", currentAuditor, BookieImpl.getBookieId(confByIndex(0))); + Awaitility.await().untilAsserted(() -> { + assertTrue("AR2's Auditor should not be running", (main2.auditorElector.getAuditor() == null + || !main2.auditorElector.getAuditor().isRunning())); + assertTrue("AR3's Auditor should not be running", (main3.auditorElector.getAuditor() == null + || !main3.auditorElector.getAuditor().isRunning())); + }); + + + /* + * expire zk2 and zk1 sessions. + */ + zkUtil.expireSession(zk2); + zkUtil.expireSession(zk1); + + /* + * wait for some time for all the components of AR1 and AR2 are + * shutdown. + */ + for (int i = 0; i < 10; i++) { + if (!main1.auditorElector.isRunning() && !main1.replicationWorker.isRunning() + && !main1.isAutoRecoveryRunning() && !main2.auditorElector.isRunning() + && !main2.replicationWorker.isRunning() && !main2.isAutoRecoveryRunning()) { + break; + } + Thread.sleep(1000); + } + + /* + * the AR3 should be current auditor. + */ + currentAuditor = main3.auditorElector.getCurrentAuditor(); + assertEquals("Current Auditor should be AR3", currentAuditor, BookieImpl.getBookieId(confByIndex(2))); + Awaitility.await().untilAsserted(() -> { + assertNotNull(main3.auditorElector.getAuditor()); + assertTrue("Auditor of AR3 should be running", main3.auditorElector.getAuditor().isRunning()); + }); + + Awaitility.waitAtMost(100, TimeUnit.SECONDS).untilAsserted(() -> { + /* + * since AR3 is current auditor, AR1's auditor should not be running + * anymore. + */ + assertFalse("AR1's auditor should not be running", auditor1.isRunning()); + + /* + * components of AR2 and AR3 should not be running since zk1 and zk2 + * sessions are expired. + */ + assertFalse("Elector1 should have shutdown", main1.auditorElector.isRunning()); + assertFalse("RW1 should have shutdown", main1.replicationWorker.isRunning()); + assertFalse("AR1 should have shutdown", main1.isAutoRecoveryRunning()); + assertFalse("Elector2 should have shutdown", main2.auditorElector.isRunning()); + assertFalse("RW2 should have shutdown", main2.replicationWorker.isRunning()); + assertFalse("AR2 should have shutdown", main2.isAutoRecoveryRunning()); + }); + + } + + /* + * start autoRecoveryMain and make sure all its components are running and + * myVote node is existing + */ + PulsarMetadataClientDriver startAutoRecoveryMain(AutoRecoveryMain autoRecoveryMain) throws Exception { + autoRecoveryMain.start(); + PulsarMetadataClientDriver pulsarMetadataClientDriver = (PulsarMetadataClientDriver) autoRecoveryMain.bkc + .getMetadataClientDriver(); + TestUtils.assertEventuallyTrue("autoRecoveryMain components should be running", + () -> autoRecoveryMain.auditorElector.isRunning() + && autoRecoveryMain.replicationWorker.isRunning() && autoRecoveryMain.isAutoRecoveryRunning()); + return pulsarMetadataClientDriver; + } + + private ZooKeeper getZk(PulsarMetadataClientDriver pulsarMetadataClientDriver) throws Exception { + PulsarLedgerManagerFactory pulsarLedgerManagerFactory = + (PulsarLedgerManagerFactory) pulsarMetadataClientDriver.getLedgerManagerFactory(); + Field field = pulsarLedgerManagerFactory.getClass().getDeclaredField("store"); + field.setAccessible(true); + ZKMetadataStore zkMetadataStore = (ZKMetadataStore) field.get(pulsarLedgerManagerFactory); + return zkMetadataStore.getZkClient(); + } +} From 0da5e4604cd628a61ad9caf53406967787614e85 Mon Sep 17 00:00:00 2001 From: Jiwe Guo Date: Thu, 21 Sep 2023 18:29:51 +0800 Subject: [PATCH 064/699] Revert "[fix][broker] Fix PulsarService.getLookupServiceAddress returns wrong port if TLS is enabled (#21015)" This reverts commit ad69decab1545d3cf63e40e88fd957dca4991459. --- .../apache/pulsar/broker/PulsarService.java | 10 ++--- .../broker/namespace/NamespaceService.java | 6 +-- .../pulsar/broker/PulsarServiceTest.java | 2 - .../pulsar/broker/admin/AdminApi2Test.java | 14 +++++-- .../pulsar/broker/admin/AdminApiTest.java | 2 +- .../auth/MockedPulsarServiceBaseTest.java | 4 ++ .../loadbalance/AdvertisedListenersTest.java | 2 + .../broker/loadbalance/LoadBalancerTest.java | 1 + .../SimpleLoadManagerImplTest.java | 1 + .../impl/ModularLoadManagerImplTest.java | 6 +++ .../OwnerShipForCurrentServerTestBase.java | 2 + .../broker/service/BrokerServiceTest.java | 9 ----- .../broker/service/ClusterMigrationTest.java | 9 ----- .../pulsar/broker/service/ReplicatorTest.java | 2 +- .../transaction/TransactionTestBase.java | 2 + .../client/api/BrokerServiceLookupTest.java | 7 +--- .../api/ClientAuthenticationTlsTest.java | 4 +- .../proxy/server/ProxyRefreshAuthTest.java | 1 - .../server/ProxyServiceTlsStarterTest.java | 1 - .../containers/BrokerContainer.java | 9 +---- .../containers/ProxyContainer.java | 9 +---- .../tests/integration/tls/ClientTlsTest.java | 9 ----- .../integration/topologies/PulsarCluster.java | 40 ++++++------------- .../topologies/PulsarClusterSpec.java | 6 --- 24 files changed, 57 insertions(+), 101 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 46f0756a82aea..4ffb5b77d5424 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 @@ -1729,18 +1729,18 @@ public static String webAddressTls(String host, int port) { } public String getSafeWebServiceAddress() { - return webServiceAddressTls != null ? webServiceAddressTls : webServiceAddress; + return webServiceAddress != null ? webServiceAddress : webServiceAddressTls; } @Deprecated public String getSafeBrokerServiceUrl() { - return brokerServiceUrlTls != null ? brokerServiceUrlTls : brokerServiceUrl; + return brokerServiceUrl != null ? brokerServiceUrl : brokerServiceUrlTls; } public String getLookupServiceAddress() { - return String.format("%s:%s", advertisedAddress, config.getWebServicePortTls().isPresent() - ? config.getWebServicePortTls().get() - : config.getWebServicePort().orElseThrow()); + return String.format("%s:%s", advertisedAddress, config.getWebServicePort().isPresent() + ? config.getWebServicePort().get() + : config.getWebServicePortTls().orElseThrow()); } public TopicPoliciesService getTopicPoliciesService() { 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 d80ad5b7a60d1..d66e3c3b65d76 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 @@ -1591,10 +1591,10 @@ public static NamespaceName getHeartbeatNamespaceV2(String lookupBroker, Service public static NamespaceName getSLAMonitorNamespace(String host, ServiceConfiguration config) { Integer port = null; - if (config.getWebServicePortTls().isPresent()) { - port = config.getWebServicePortTls().get(); - } else if (config.getWebServicePort().isPresent()) { + if (config.getWebServicePort().isPresent()) { port = config.getWebServicePort().get(); + } else if (config.getWebServicePortTls().isPresent()) { + port = config.getWebServicePortTls().get(); } return NamespaceName.get(String.format(SLA_NAMESPACE_FMT, config.getClusterName(), host, port)); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java index 3e0887646e119..37a7310ae17ca 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java @@ -54,8 +54,6 @@ protected void cleanup() throws Exception { @Override protected void doInitConf() throws Exception { super.doInitConf(); - conf.setBrokerServicePortTls(Optional.of(0)); - conf.setWebServicePortTls(Optional.of(0)); if (useStaticPorts) { conf.setBrokerServicePortTls(Optional.of(6651)); conf.setBrokerServicePort(Optional.of(6660)); 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 c68010f967b9b..99009a1988206 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 @@ -439,13 +439,19 @@ public void testTopicPoliciesWithMultiBroker() throws Exception { String tenantName = newUniqueName("prop-xyz2"); admin.tenants().createTenant(tenantName, tenantInfo); admin.namespaces().createNamespace(tenantName + "/ns1", Set.of("test")); - ServiceConfiguration config2 = super.getDefaultConf(); + conf.setBrokerServicePort(Optional.of(1024)); + conf.setBrokerServicePortTls(Optional.of(1025)); + conf.setWebServicePort(Optional.of(1026)); + conf.setWebServicePortTls(Optional.of(1027)); @Cleanup - PulsarTestContext pulsarTestContext2 = createAdditionalPulsarTestContext(config2); + PulsarTestContext pulsarTestContext2 = createAdditionalPulsarTestContext(conf); PulsarService pulsar2 = pulsarTestContext2.getPulsarService(); - ServiceConfiguration config3 = super.getDefaultConf(); + conf.setBrokerServicePort(Optional.of(2048)); + conf.setBrokerServicePortTls(Optional.of(2049)); + conf.setWebServicePort(Optional.of(2050)); + conf.setWebServicePortTls(Optional.of(2051)); @Cleanup - PulsarTestContext pulsarTestContext3 = createAdditionalPulsarTestContext(config3); + PulsarTestContext pulsarTestContext3 = createAdditionalPulsarTestContext(conf); PulsarService pulsar3 = pulsarTestContext.getPulsarService(); @Cleanup PulsarAdmin admin2 = PulsarAdmin.builder().serviceHttpUrl(pulsar2.getWebServiceAddress()).build(); 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 e97707710d743..9fb3cb6fb2328 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 @@ -931,7 +931,7 @@ public void persistentTopics(String topicName) throws Exception { assertEquals(topicStats.getSubscriptions().get(subName).getMsgBacklog(), 10); assertEquals(topicStats.getPublishers().size(), 0); assertEquals(topicStats.getOwnerBroker(), - pulsar.getAdvertisedAddress() + ":" + pulsar.getConfiguration().getWebServicePortTls().get()); + pulsar.getAdvertisedAddress() + ":" + pulsar.getConfiguration().getWebServicePort().get()); PersistentTopicInternalStats internalStats = admin.topics().getInternalStats(persistentTopicName, false); assertEquals(internalStats.cursors.keySet(), Set.of(Codec.encode(subName))); 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 fd8ef0f40b115..c32d3fc3b0b27 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 @@ -221,8 +221,10 @@ protected void doInitConf() throws Exception { this.conf.setBrokerShutdownTimeoutMs(0L); this.conf.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); this.conf.setBrokerServicePort(Optional.of(0)); + this.conf.setBrokerServicePortTls(Optional.of(0)); this.conf.setAdvertisedAddress("localhost"); this.conf.setWebServicePort(Optional.of(0)); + this.conf.setWebServicePortTls(Optional.of(0)); this.conf.setNumExecutorThreadPoolSize(5); this.conf.setExposeBundlesMetricsInPrometheus(true); } @@ -462,7 +464,9 @@ protected ServiceConfiguration getDefaultConf() { configuration.setBrokerShutdownTimeoutMs(0L); configuration.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); configuration.setBrokerServicePort(Optional.of(0)); + configuration.setBrokerServicePortTls(Optional.of(0)); configuration.setWebServicePort(Optional.of(0)); + configuration.setWebServicePortTls(Optional.of(0)); configuration.setBookkeeperClientExposeStatsToPrometheus(true); configuration.setNumExecutorThreadPoolSize(5); configuration.setBrokerMaxConnections(0); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AdvertisedListenersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AdvertisedListenersTest.java index a88ccd60ae4c4..7a8154312e4dc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AdvertisedListenersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AdvertisedListenersTest.java @@ -78,6 +78,7 @@ private void updateConfig(ServiceConfiguration conf, String advertisedAddress) { ",public_https:https://localhost:" + httpsPort); conf.setBrokerServicePort(Optional.of(pulsarPort)); conf.setWebServicePort(Optional.of(httpPort)); + conf.setWebServicePortTls(Optional.of(httpsPort)); } @Test @@ -100,6 +101,7 @@ public void testLookup() throws Exception { assertEquals(new URI(ld.getBrokerUrl()).getHost(), "localhost"); assertEquals(new URI(ld.getHttpUrl()).getHost(), "localhost"); + assertEquals(new URI(ld.getHttpUrlTls()).getHost(), "localhost"); // Produce data diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java index 7cc4499df9710..68902c73e5717 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java @@ -124,6 +124,7 @@ void setup() throws Exception { config.setAdvertisedAddress("localhost"); config.setWebServicePort(Optional.of(0)); config.setBrokerServicePortTls(Optional.of(0)); + config.setWebServicePortTls(Optional.of(0)); config.setMetadataStoreUrl("zk:127.0.0.1:" + bkEnsemble.getZookeeperPort()); config.setBrokerShutdownTimeoutMs(0L); config.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java index 6303c70b4dc77..c4898786e3e03 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java @@ -113,6 +113,7 @@ void setup() throws Exception { config1.setBrokerServicePort(Optional.of(0)); config1.setLoadManagerClassName(SimpleLoadManagerImpl.class.getName()); config1.setBrokerServicePortTls(Optional.of(0)); + config1.setWebServicePortTls(Optional.of(0)); config1.setAdvertisedAddress("localhost"); pulsar1 = new PulsarService(config1); pulsar1.start(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java index d8acb6d24e9ef..786c9027c944d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java @@ -169,6 +169,8 @@ void setup() throws Exception { config1.setBrokerShutdownTimeoutMs(0L); config1.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); config1.setBrokerServicePort(Optional.of(0)); + config1.setBrokerServicePortTls(Optional.of(0)); + config1.setWebServicePortTls(Optional.of(0)); pulsar1 = new PulsarService(config1); pulsar1.start(); @@ -187,6 +189,8 @@ void setup() throws Exception { config2.setBrokerShutdownTimeoutMs(0L); config2.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); config2.setBrokerServicePort(Optional.of(0)); + config2.setBrokerServicePortTls(Optional.of(0)); + config2.setWebServicePortTls(Optional.of(0)); pulsar2 = new PulsarService(config2); pulsar2.start(); @@ -200,6 +204,8 @@ void setup() throws Exception { config.setBrokerShutdownTimeoutMs(0L); config.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); config.setBrokerServicePort(Optional.of(0)); + config.setBrokerServicePortTls(Optional.of(0)); + config.setWebServicePortTls(Optional.of(0)); pulsar3 = new PulsarService(config); secondaryHost = String.format("%s:%d", "localhost", pulsar2.getListenPortHTTP().get()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipForCurrentServerTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipForCurrentServerTestBase.java index 46e8989ac3df4..8dd4f53db8240 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipForCurrentServerTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipForCurrentServerTestBase.java @@ -80,8 +80,10 @@ protected void startBroker() throws Exception { conf.setBrokerShutdownTimeoutMs(0L); conf.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); conf.setBrokerServicePort(Optional.of(0)); + conf.setBrokerServicePortTls(Optional.of(0)); conf.setAdvertisedAddress("localhost"); conf.setWebServicePort(Optional.of(0)); + conf.setWebServicePortTls(Optional.of(0)); serviceConfigurationList.add(conf); PulsarTestContext.Builder testContextBuilder = 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 1c24146b18440..6c0bc6efec758 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 @@ -1786,13 +1786,4 @@ public void testUnsubscribeNonDurableSub() throws Exception { fail("Unsubscribe failed"); } } - - @Test - public void testGetLookupServiceAddress() throws Exception { - cleanup(); - setup(); - conf.setWebServicePortTls(Optional.of(8081)); - assertEquals(pulsar.getLookupServiceAddress(), "localhost:8081"); - resetState(); - } } 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 76209294a0e22..469e155d409b3 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 @@ -28,7 +28,6 @@ import java.lang.reflect.Method; import java.net.URL; -import java.util.Optional; import java.util.concurrent.TimeUnit; import org.apache.pulsar.broker.BrokerTestUtil; @@ -478,14 +477,6 @@ protected void setup() throws Exception { super.setupWithClusterName(clusterName); } - @Override - protected void doInitConf() throws Exception { - super.doInitConf(); - this.conf.setWebServicePortTls(Optional.of(0)); - this.conf.setBrokerServicePortTls(Optional.of(0)); - } - - public PulsarService getPulsarService() { return pulsar; } 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 1139bb9e0bfb7..f710c8541d1b5 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 @@ -247,7 +247,7 @@ public void activeBrokerParse() throws Exception { -> admin2.clusters().getCluster(cluster2) != null); List list = admin1.brokers().getActiveBrokers(cluster2); - assertEquals(list.get(0), urlTls2.toString().replace("https://", "")); + assertEquals(list.get(0), url2.toString().replace("http://", "")); //restore configuration pulsar1.getConfiguration().setAuthorizationEnabled(false); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTestBase.java index 1ff835732aab5..c0300c63b3587 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTestBase.java @@ -157,8 +157,10 @@ protected void startBroker() throws Exception { conf.setBrokerShutdownTimeoutMs(0L); conf.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); conf.setBrokerServicePort(Optional.of(0)); + conf.setBrokerServicePortTls(Optional.of(0)); conf.setAdvertisedAddress("localhost"); conf.setWebServicePort(Optional.of(0)); + conf.setWebServicePortTls(Optional.of(0)); conf.setTransactionCoordinatorEnabled(true); conf.setBrokerDeduplicationEnabled(true); conf.setTransactionBufferSnapshotMaxTransactionCount(2); 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 accdd2a335f39..792f419ee997e 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 @@ -474,7 +474,7 @@ public void testWebserviceServiceTls() throws Exception { // request [3] doReturn(true).when(loadManager1).isCentralized(); doReturn(true).when(loadManager2).isCentralized(); - SimpleResourceUnit resourceUnit = new SimpleResourceUnit(pulsar.getWebServiceAddressTls(), null); + SimpleResourceUnit resourceUnit = new SimpleResourceUnit(pulsar.getWebServiceAddress(), null); doReturn(Optional.of(resourceUnit)).when(loadManager2).getLeastLoaded(any(ServiceUnitId.class)); doReturn(Optional.of(resourceUnit)).when(loadManager1).getLeastLoaded(any(ServiceUnitId.class)); @@ -507,9 +507,6 @@ public void testWebserviceServiceTls() throws Exception { loadManager1 = null; loadManager2 = null; - - conf.setBrokerServicePortTls(Optional.empty()); - conf.setWebServicePortTls(Optional.empty()); } /** @@ -843,8 +840,6 @@ public void testPartitionedMetadataWithDeprecatedVersion() throws Exception { admin.topics().createPartitionedTopic(dest.toString(), totalPartitions); stopBroker(); - conf.setBrokerServicePortTls(Optional.empty()); - conf.setWebServicePortTls(Optional.empty()); conf.setClientLibraryVersionCheckEnabled(true); startBroker(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientAuthenticationTlsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientAuthenticationTlsTest.java index d716d5a806392..c9b243257c4e1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientAuthenticationTlsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientAuthenticationTlsTest.java @@ -22,7 +22,6 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.expectThrows; import java.util.HashSet; -import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; @@ -51,8 +50,7 @@ protected void doInitConf() throws Exception { Set providers = new HashSet<>(); providers.add(AuthenticationProviderTls.class.getName()); conf.setAuthenticationProviders(providers); - conf.setWebServicePortTls(Optional.of(0)); - conf.setBrokerServicePortTls(Optional.of(0)); + conf.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); conf.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); conf.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java index 2f36cc679f1f2..bde989fc432f9 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java @@ -69,7 +69,6 @@ protected void doInitConf() throws Exception { conf.setAdvertisedAddress(null); conf.setAuthenticateOriginalAuthData(true); conf.setBrokerServicePort(Optional.of(0)); - conf.setWebServicePortTls(Optional.of(0)); conf.setWebServicePort(Optional.of(0)); Set superUserRoles = new HashSet<>(); 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 6247c2a66e874..01c06fbf52f4e 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 @@ -75,7 +75,6 @@ protected void setup() throws Exception { protected void doInitConf() throws Exception { super.doInitConf(); - this.conf.setBrokerServicePortTls(Optional.of(0)); this.conf.setTlsCertificateFilePath(PROXY_CERT_FILE_PATH); this.conf.setTlsKeyFilePath(PROXY_KEY_FILE_PATH); } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/BrokerContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/BrokerContainer.java index a51397050b97f..616d45554d75c 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/BrokerContainer.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/BrokerContainer.java @@ -28,13 +28,8 @@ public class BrokerContainer extends PulsarContainer { public static final String NAME = "pulsar-broker"; public BrokerContainer(String clusterName, String hostName) { - this(clusterName, hostName, false); - } - - public BrokerContainer(String clusterName, String hostName, boolean enableTls) { - super(clusterName, hostName, hostName, "bin/run-broker.sh", BROKER_PORT, - enableTls ? BROKER_PORT_TLS : 0, BROKER_HTTP_PORT, - enableTls ? BROKER_HTTPS_PORT : 0, DEFAULT_HTTP_PATH, DEFAULT_IMAGE_NAME); + super(clusterName, hostName, hostName, "bin/run-broker.sh", BROKER_PORT, BROKER_PORT_TLS, + BROKER_HTTP_PORT, BROKER_HTTPS_PORT, DEFAULT_HTTP_PATH, DEFAULT_IMAGE_NAME); tailContainerLog(); } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/ProxyContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/ProxyContainer.java index f3926878f37c5..53283447378f5 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/ProxyContainer.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/ProxyContainer.java @@ -28,13 +28,8 @@ public class ProxyContainer extends PulsarContainer { public static final String NAME = "pulsar-proxy"; public ProxyContainer(String clusterName, String hostName) { - this(clusterName, hostName, false); - } - - public ProxyContainer(String clusterName, String hostName, boolean enableTls) { - super(clusterName, hostName, hostName, "bin/run-proxy.sh", BROKER_PORT, - enableTls ? BROKER_PORT_TLS : 0, BROKER_HTTP_PORT, - enableTls ? BROKER_HTTPS_PORT : 0, DEFAULT_HTTP_PATH, DEFAULT_IMAGE_NAME); + super(clusterName, hostName, hostName, "bin/run-proxy.sh", BROKER_PORT, BROKER_PORT_TLS, BROKER_HTTP_PORT, + BROKER_HTTPS_PORT, DEFAULT_HTTP_PATH, DEFAULT_IMAGE_NAME); } @Override diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/tls/ClientTlsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/tls/ClientTlsTest.java index 080912cd49262..59ff978cafa06 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/tls/ClientTlsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/tls/ClientTlsTest.java @@ -29,7 +29,6 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.tests.integration.suites.PulsarTestSuite; -import org.apache.pulsar.tests.integration.topologies.PulsarClusterSpec; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -42,14 +41,6 @@ private static String loadCertificateAuthorityFile(String name) { return Resources.getResource("certificate-authority/" + name).getPath(); } - @Override - protected PulsarClusterSpec.PulsarClusterSpecBuilder beforeSetupCluster( - String clusterName, - PulsarClusterSpec.PulsarClusterSpecBuilder specBuilder) { - specBuilder.enableTls(true); - return specBuilder; - } - @DataProvider(name = "adminUrls") public Object[][] adminUrls() { return new Object[][]{ 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 769f135599b01..9b4823f46d4cc 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 @@ -38,7 +38,6 @@ import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.commons.io.IOUtils; -import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.tests.integration.containers.BKContainer; import org.apache.pulsar.tests.integration.containers.BrokerContainer; import org.apache.pulsar.tests.integration.containers.CSContainer; @@ -133,16 +132,14 @@ private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean s this.brokerContainers = Maps.newTreeMap(); this.workerContainers = Maps.newTreeMap(); - this.proxyContainer = new ProxyContainer(appendClusterName("pulsar-proxy"), ProxyContainer.NAME, spec.enableTls) + this.proxyContainer = new ProxyContainer(appendClusterName("pulsar-proxy"), ProxyContainer.NAME) .withNetwork(network) .withNetworkAliases(appendClusterName("pulsar-proxy")) .withEnv("zkServers", appendClusterName(ZKContainer.NAME)) .withEnv("zookeeperServers", appendClusterName(ZKContainer.NAME)) .withEnv("configurationStoreServers", CSContainer.NAME + ":" + CS_PORT) - .withEnv("clusterName", clusterName); + .withEnv("clusterName", clusterName) // enable mTLS - if (spec.enableTls) { - proxyContainer .withEnv("webServicePortTls", String.valueOf(BROKER_HTTPS_PORT)) .withEnv("servicePortTls", String.valueOf(BROKER_PORT_TLS)) .withEnv("forwardAuthorizationCredentials", "true") @@ -150,15 +147,7 @@ private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean s .withEnv("tlsAllowInsecureConnection", "false") .withEnv("tlsCertificateFilePath", "/pulsar/certificate-authority/server-keys/proxy.cert.pem") .withEnv("tlsKeyFilePath", "/pulsar/certificate-authority/server-keys/proxy.key-pk8.pem") - .withEnv("tlsTrustCertsFilePath", "/pulsar/certificate-authority/certs/ca.cert.pem") - .withEnv("brokerClientAuthenticationPlugin", AuthenticationTls.class.getName()) - .withEnv("brokerClientAuthenticationParameters", String.format("tlsCertFile:%s,tlsKeyFile:%s", "/pulsar/certificate-authority/client-keys/admin.cert.pem", "/pulsar/certificate-authority/client-keys/admin.key-pk8.pem")) - .withEnv("tlsEnabledWithBroker", "true") - .withEnv("brokerClientTrustCertsFilePath", "/pulsar/certificate-authority/certs/ca.cert.pem") - .withEnv("brokerClientCertificateFilePath", "/pulsar/certificate-authority/server-keys/proxy.cert.pem") - .withEnv("brokerClientKeyFilePath", "/pulsar/certificate-authority/server-keys/proxy.key-pk8.pem"); - - } + .withEnv("tlsTrustCertsFilePath", "/pulsar/certificate-authority/certs/ca.cert.pem"); if (spec.proxyEnvs != null) { spec.proxyEnvs.forEach(this.proxyContainer::withEnv); } @@ -195,7 +184,7 @@ private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean s // create brokers brokerContainers.putAll( runNumContainers("broker", spec.numBrokers(), (name) -> { - BrokerContainer brokerContainer = new BrokerContainer(clusterName, appendClusterName(name), spec.enableTls) + BrokerContainer brokerContainer = new BrokerContainer(clusterName, appendClusterName(name)) .withNetwork(network) .withNetworkAliases(appendClusterName(name)) .withEnv("zkServers", appendClusterName(ZKContainer.NAME)) @@ -206,19 +195,16 @@ private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean s .withEnv("loadBalancerOverrideBrokerNicSpeedGbps", "1") // used in s3 tests .withEnv("AWS_ACCESS_KEY_ID", "accesskey").withEnv("AWS_SECRET_KEY", "secretkey") - .withEnv("maxMessageSize", "" + spec.maxMessageSize); - if (spec.enableTls) { + .withEnv("maxMessageSize", "" + spec.maxMessageSize) // enable mTLS - brokerContainer - .withEnv("webServicePortTls", String.valueOf(BROKER_HTTPS_PORT)) - .withEnv("brokerServicePortTls", String.valueOf(BROKER_PORT_TLS)) - .withEnv("authenticateOriginalAuthData", "true") - .withEnv("tlsAllowInsecureConnection", "false") - .withEnv("tlsRequireTrustedClientCertOnConnect", "true") - .withEnv("tlsTrustCertsFilePath", "/pulsar/certificate-authority/certs/ca.cert.pem") - .withEnv("tlsCertificateFilePath", "/pulsar/certificate-authority/server-keys/broker.cert.pem") - .withEnv("tlsKeyFilePath", "/pulsar/certificate-authority/server-keys/broker.key-pk8.pem"); - } + .withEnv("webServicePortTls", String.valueOf(BROKER_HTTPS_PORT)) + .withEnv("brokerServicePortTls", String.valueOf(BROKER_PORT_TLS)) + .withEnv("authenticateOriginalAuthData", "true") + .withEnv("tlsRequireTrustedClientCertOnConnect", "true") + .withEnv("tlsAllowInsecureConnection", "false") + .withEnv("tlsCertificateFilePath", "/pulsar/certificate-authority/server-keys/broker.cert.pem") + .withEnv("tlsKeyFilePath", "/pulsar/certificate-authority/server-keys/broker.key-pk8.pem") + .withEnv("tlsTrustCertsFilePath", "/pulsar/certificate-authority/certs/ca.cert.pem"); if (spec.queryLastMessage) { brokerContainer.withEnv("bookkeeperExplicitLacIntervalInMills", "10"); brokerContainer.withEnv("bookkeeperUseV2WireProtocol", "false"); 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 c141e990d62e0..fa28d20e6b356 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 @@ -177,10 +177,4 @@ public class PulsarClusterSpec { * Additional ports to expose on bookie containers. */ List bookieAdditionalPorts; - - /** - * Enable TLS for connection. - */ - @Default - boolean enableTls = false; } From 0eb1267077d8580eae14bbcedfdf35a9a3419bea Mon Sep 17 00:00:00 2001 From: hanmz Date: Thu, 21 Sep 2023 10:04:41 +0800 Subject: [PATCH 065/699] [fix][broker] replicator leak when removeReplicator in NonPersistentTopic (#21205) --- .../pulsar/broker/service/nonpersistent/NonPersistentTopic.java | 1 + 1 file changed, 1 insertion(+) 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 c764283cb4459..42c11fa14cae1 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 @@ -634,6 +634,7 @@ CompletableFuture removeReplicator(String remoteCluster) { replicators.get(remoteCluster).disconnect().thenRun(() -> { log.info("[{}] Successfully removed replicator {}", name, remoteCluster); + replicators.remove(remoteCluster); }).exceptionally(e -> { log.error("[{}] Failed to close replication producer {} {}", topic, name, e.getMessage(), e); From b442caea736f632a7fb4a021b0b13a1acea4232a Mon Sep 17 00:00:00 2001 From: Asaf Mesika Date: Sun, 24 Sep 2023 06:43:40 +0300 Subject: [PATCH 066/699] [fix][broker] Fixed reset for AggregatedNamespaceStats (#21225) --- pulsar-broker/pom.xml | 1 + .../prometheus/AggregatedNamespaceStats.java | 24 +++- .../AggregatedNamespaceStatsTest.java | 103 +++++++++++++++++- 3 files changed, 124 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index 54fb4c4a3c04c..6be64cf92f415 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -582,6 +582,7 @@ protobuf-maven-plugin ${protobuf-maven-plugin.version} + com.google.protobuf:protoc:${protoc3.version}:exe:${os.detected.classifier} true diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java index 9fe5588044d2f..d0dc4fe2a7e7d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java @@ -34,7 +34,7 @@ public class AggregatedNamespaceStats { public double throughputIn; public double throughputOut; - public long messageAckRate; + public double messageAckRate; public long bytesInCounter; public long msgInCounter; public long bytesOutCounter; @@ -64,7 +64,7 @@ public class AggregatedNamespaceStats { long compactionCompactedEntriesCount; long compactionCompactedEntriesSize; StatsBuckets compactionLatencyBuckets = new StatsBuckets(CompactionRecord.WRITE_LATENCY_BUCKETS_USEC); - int delayedMessageIndexSizeInBytes; + long delayedMessageIndexSizeInBytes; Map bucketDelayedIndexStats = new HashMap<>(); @@ -182,14 +182,34 @@ public void reset() { rateOut = 0; throughputIn = 0; throughputOut = 0; + messageAckRate = 0; + bytesInCounter = 0; + msgInCounter = 0; + + bytesOutCounter = 0; + msgOutCounter = 0; msgBacklog = 0; msgDelayed = 0; + ongoingTxnCount = 0; + abortedTxnCount = 0; + committedTxnCount = 0; + backlogQuotaLimit = 0; backlogQuotaLimitTime = -1; replicationStats.clear(); subscriptionStats.clear(); + + compactionRemovedEventCount = 0; + compactionSucceedCount = 0; + compactionFailedCount = 0; + compactionDurationTimeInMills = 0; + compactionReadThroughput = 0; + compactionWriteThroughput = 0; + compactionCompactedEntriesCount = 0; + compactionCompactedEntriesSize = 0; + delayedMessageIndexSizeInBytes = 0; bucketDelayedIndexStats.clear(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java index b5933f9ecf529..0e12d75f74fa0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java @@ -20,10 +20,12 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; - +import java.util.HashMap; +import org.apache.bookkeeper.mledger.util.StatsBuckets; +import org.apache.pulsar.common.policies.data.stats.TopicMetricBean; import org.testng.annotations.Test; -@Test(groups = "broker") +@Test(groups = {"broker"}) public class AggregatedNamespaceStatsTest { @Test @@ -157,4 +159,101 @@ public void testSimpleAggregation() { assertEquals(nsSubStats.unackedMessages, 2); } + + @Test + public void testReset() { + AggregatedNamespaceStats stats = new AggregatedNamespaceStats(); + stats.topicsCount = 8; + stats.subscriptionsCount = 3; + stats.producersCount = 1; + stats.consumersCount = 8; + stats.rateIn = 1.3; + stats.rateOut = 3.5; + stats.throughputIn = 3.2; + stats.throughputOut = 5.8; + stats.messageAckRate = 12; + stats.bytesInCounter = 1234; + stats.msgInCounter = 3889; + stats.bytesOutCounter = 89775; + stats.msgOutCounter = 28983; + stats.msgBacklog = 39; + stats.msgDelayed = 31; + + stats.ongoingTxnCount = 87; + stats.abortedTxnCount = 74; + stats.committedTxnCount = 34; + + stats.backlogQuotaLimit = 387; + stats.backlogQuotaLimitTime = 8771; + + stats.replicationStats = new HashMap<>(); + stats.replicationStats.put("r", new AggregatedReplicationStats()); + + stats.subscriptionStats = new HashMap<>(); + stats.subscriptionStats.put("r", new AggregatedSubscriptionStats()); + + stats.compactionRemovedEventCount = 124; + stats.compactionSucceedCount = 487; + stats.compactionFailedCount = 84857; + stats.compactionDurationTimeInMills = 2384; + stats.compactionReadThroughput = 355423; + stats.compactionWriteThroughput = 23299; + stats.compactionCompactedEntriesCount = 37522; + stats.compactionCompactedEntriesSize = 8475; + + stats.compactionLatencyBuckets = new StatsBuckets(5); + stats.compactionLatencyBuckets.addValue(3); + + stats.delayedMessageIndexSizeInBytes = 45223; + + stats.bucketDelayedIndexStats = new HashMap<>(); + stats.bucketDelayedIndexStats.put("t", new TopicMetricBean()); + + stats.reset(); + + assertEquals(stats.bytesOutCounter, 0); + assertEquals(stats.topicsCount, 0); + assertEquals(stats.subscriptionsCount, 0); + assertEquals(stats.producersCount, 0); + assertEquals(stats.consumersCount, 0); + assertEquals(stats.rateIn, 0); + assertEquals(stats.rateOut, 0); + assertEquals(stats.throughputIn, 0); + assertEquals(stats.throughputOut, 0); + assertEquals(stats.messageAckRate, 0); + assertEquals(stats.bytesInCounter, 0); + assertEquals(stats.msgInCounter, 0); + assertEquals(stats.bytesOutCounter, 0); + assertEquals(stats.msgOutCounter, 0); + + assertEquals(stats.managedLedgerStats.storageSize, 0); + + assertEquals(stats.msgBacklog, 0); + assertEquals(stats.msgDelayed, 0); + + assertEquals(stats.ongoingTxnCount, 0); + assertEquals(stats.abortedTxnCount, 0); + assertEquals(stats.committedTxnCount, 0); + + assertEquals(stats.backlogQuotaLimit, 0); + assertEquals(stats.backlogQuotaLimitTime, -1); + + assertEquals(stats.replicationStats.size(), 0); + assertEquals(stats.subscriptionStats.size(), 0); + + assertEquals(stats.compactionRemovedEventCount, 0); + assertEquals(stats.compactionSucceedCount, 0); + assertEquals(stats.compactionFailedCount, 0); + assertEquals(stats.compactionDurationTimeInMills, 0); + assertEquals(stats.compactionReadThroughput, 0); + assertEquals(stats.compactionWriteThroughput, 0); + assertEquals(stats.compactionCompactedEntriesCount, 0); + assertEquals(stats.compactionCompactedEntriesSize, 0); + + assertEquals(stats.compactionLatencyBuckets.getSum(), 0); + + assertEquals(stats.delayedMessageIndexSizeInBytes, 0); + assertEquals(stats.bucketDelayedIndexStats.size(), 0); + } + } From c8b8639be47c3634017fd4a841318d5e37358be0 Mon Sep 17 00:00:00 2001 From: Guangning E Date: Sun, 24 Sep 2023 20:22:59 +0800 Subject: [PATCH 067/699] [fix][broker]Fixed produce and consume when anonymousUserRole enabled (#21237) --- .../pulsar/broker/service/ServerCnx.java | 19 +++++++--- .../pulsar/broker/service/ServerCnxTest.java | 35 +++++++++++++++++++ 2 files changed, 50 insertions(+), 4 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 f3b355d63325f..74c2cee3ec7be 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 @@ -990,7 +990,6 @@ protected void handleConnect(CommandConnect connect) { try { byte[] authData = connect.hasAuthData() ? connect.getAuthData() : emptyArray; AuthData clientData = AuthData.of(authData); - // init authentication if (connect.hasAuthMethodName()) { authMethod = connect.getAuthMethodName(); @@ -1049,10 +1048,22 @@ protected void handleConnect(CommandConnect connect) { .getAuthenticationService() .getAuthenticationProvider(originalAuthMethod); + /** + * When both the broker and the proxy are configured with anonymousUserRole + * if the client does not configure an authentication method + * the proxy side will set the value of anonymousUserRole to clientAuthRole when it creates a connection + * and the value of clientAuthMethod will be none. + * Similarly, should also set the value of authRole to anonymousUserRole on the broker side. + */ if (originalAuthenticationProvider == null) { - throw new AuthenticationException( - String.format("Can't find AuthenticationProvider for original role" - + " using auth method [%s] is not available", originalAuthMethod)); + authRole = getBrokerService().getAuthenticationService().getAnonymousUserRole() + .orElseThrow(() -> + new AuthenticationException("No anonymous role, and can't find " + + "AuthenticationProvider for original role using auth method " + + "[" + originalAuthMethod + "] is not available")); + originalPrincipal = authRole; + completeConnect(clientProtocolVersion, clientVersion); + return; } originalAuthDataCopy = AuthData.of(connect.getOriginalAuthData().getBytes()); 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 c3bab634a42c1..e66abc34e3650 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 @@ -496,6 +496,41 @@ public void testConnectCommandWithPassingOriginalAuthData() throws Exception { channel.finish(); } + @Test(timeOut = 30000) + public void testConnectCommandWithPassingOriginalAuthDataAndSetAnonymousUserRole() throws Exception { + AuthenticationService authenticationService = mock(AuthenticationService.class); + AuthenticationProvider authenticationProvider = new MockAuthenticationProvider(); + String authMethodName = authenticationProvider.getAuthMethodName(); + + String anonymousUserRole = "admin"; + when(brokerService.getAuthenticationService()).thenReturn(authenticationService); + when(authenticationService.getAuthenticationProvider(authMethodName)).thenReturn(authenticationProvider); + when(authenticationService.getAnonymousUserRole()).thenReturn(Optional.of(anonymousUserRole)); + svcConfig.setAuthenticationEnabled(true); + svcConfig.setAuthenticateOriginalAuthData(true); + svcConfig.setProxyRoles(Collections.singleton("pass.proxy")); + svcConfig.setAnonymousUserRole(anonymousUserRole); + + resetChannel(); + assertTrue(channel.isActive()); + assertEquals(serverCnx.getState(), State.Start); + + // When both the proxy and the broker set the anonymousUserRole option + // the proxy will use anonymousUserRole to delegate the client's role when connecting. + ByteBuf clientCommand = Commands.newConnect(authMethodName, "pass.proxy", 1, null, + null, anonymousUserRole, null, null); + channel.writeInbound(clientCommand); + + Object response1 = getResponse(); + assertTrue(response1 instanceof CommandConnected); + assertEquals(serverCnx.getState(), State.Connected); + assertEquals(serverCnx.getAuthRole(), anonymousUserRole); + assertEquals(serverCnx.getPrincipal(), anonymousUserRole); + assertEquals(serverCnx.getOriginalPrincipal(), anonymousUserRole); + assertTrue(serverCnx.isActive()); + channel.finish(); + } + @Test(timeOut = 30000) public void testConnectCommandWithPassingOriginalPrincipal() throws Exception { AuthenticationService authenticationService = mock(AuthenticationService.class); From 72c6308e4e0e563fa970732b4c3f676f3c254e3d Mon Sep 17 00:00:00 2001 From: hanmz Date: Mon, 25 Sep 2023 20:53:18 +0800 Subject: [PATCH 068/699] [fix][fn] fix functions_log4j2.xml delete strategy config (#21215) --- conf/functions_log4j2.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/conf/functions_log4j2.xml b/conf/functions_log4j2.xml index 6902a3acd8736..fd4042e82e82f 100644 --- a/conf/functions_log4j2.xml +++ b/conf/functions_log4j2.xml @@ -68,7 +68,7 @@ ${sys:pulsar.function.log.dir} 2 - */${sys:pulsar.function.log.file}*log.gz + ${sys:pulsar.function.log.file}*log.gz 30d @@ -101,7 +101,7 @@ ${sys:pulsar.function.log.dir} 2 - */${sys:pulsar.function.log.file}.bk*log.gz + ${sys:pulsar.function.log.file}.bk*log.gz 30d From a514a1f67346978056ed924805f7123a1b5bdfc1 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 22 Sep 2023 16:26:04 +0800 Subject: [PATCH 069/699] [improve] [client] Merge lookup requests for the same topic (#21232) Motivation: Multiple consumers and producers can be maintained by the same Pulsar Client. In some cases, multiple consumers or producers might attempt to connect to the same topic. To optimize the process, it is recommended to perform the topic lookup only once for each topic. Modifications: - Merge lookup requests for the same topic. - Merge get partitioned metadata request for the same partitioned topic. --- .../client/api/BrokerServiceLookupTest.java | 103 ++++++++++++++++++ .../client/impl/BinaryProtoLookupService.java | 40 ++++++- 2 files changed, 141 insertions(+), 2 deletions(-) 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 792f419ee997e..c0cc06795ac24 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 @@ -31,6 +31,7 @@ import com.google.common.util.concurrent.MoreExecutors; import io.netty.handler.codec.http.HttpRequest; import io.netty.handler.codec.http.HttpResponse; +import io.prometheus.client.CollectorRegistry; import java.io.IOException; import java.io.InputStream; import java.lang.reflect.Field; @@ -72,6 +73,9 @@ import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.testcontext.PulsarTestContext; +import org.apache.pulsar.client.impl.BinaryProtoLookupService; +import org.apache.pulsar.client.impl.LookupService; +import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.ServiceUnitId; @@ -94,6 +98,7 @@ import org.asynchttpclient.Response; import org.asynchttpclient.channel.DefaultKeepAliveStrategy; import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.annotations.AfterMethod; @@ -823,6 +828,104 @@ public void testSkipSplitBundleIfOnlyOneBroker() throws Exception { } } + @Test + public void testMergeGetPartitionedMetadataRequests() throws Exception { + // Assert the lookup service is a "BinaryProtoLookupService". + final PulsarClientImpl pulsarClientImpl = (PulsarClientImpl) pulsarClient; + final LookupService lookupService = pulsarClientImpl.getLookup(); + assertTrue(lookupService instanceof BinaryProtoLookupService); + + final String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final int topicPartitions = 10; + admin.topics().createPartitionedTopic(tpName, topicPartitions); + + // Verify the request is works after merge the requests. + List> futures = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + futures.add(lookupService.getPartitionedTopicMetadata(TopicName.get(tpName))); + } + for (CompletableFuture future : futures) { + assertEquals(future.join().partitions, topicPartitions); + } + + // cleanup. + admin.topics().deletePartitionedTopic(tpName); + } + + @Test + public void testMergeLookupRequests() throws Exception { + // Assert the lookup service is a "BinaryProtoLookupService". + final PulsarClientImpl pulsarClientImpl = (PulsarClientImpl) pulsarClient; + final LookupService lookupService = pulsarClientImpl.getLookup(); + assertTrue(lookupService instanceof BinaryProtoLookupService); + + final String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(tpName); + + // Create 1 producer and 100 consumers. + List> producers = new ArrayList<>(); + List> consumers = new ArrayList<>(); + for (int i = 0; i < 20; i++) { + producers.add(pulsarClient.newProducer(Schema.STRING).topic(tpName).create()); + } + for (int i = 0; i < 20; i++) { + consumers.add(pulsarClient.newConsumer(Schema.STRING).topic(tpName).subscriptionName("s" + i).subscribe()); + } + + // Verify the lookup count will be smaller than before improve. + int lookupCountBeforeUnload = calculateLookupRequestCount(); + admin.namespaces().unload(TopicName.get(tpName).getNamespace()); + Awaitility.await().untilAsserted(() -> { + for (Producer p : producers) { + assertEquals(WhiteboxImpl.getInternalState(p, "state").toString(), "Ready"); + } + for (Consumer c : consumers) { + assertEquals(WhiteboxImpl.getInternalState(c, "state").toString(), "Ready"); + } + }); + int lookupCountAfterUnload = calculateLookupRequestCount(); + log.info("lookup count before unload: {}, after unload: {}", lookupCountBeforeUnload, lookupCountAfterUnload); + assertTrue(lookupCountAfterUnload < lookupCountBeforeUnload * 2, + "the lookup count should be smaller than before improve"); + + // Verify the producers and consumers is still works. + List messagesSent = new ArrayList<>(); + int index = 0; + for (Producer producer: producers) { + String message = Integer.valueOf(index++).toString(); + producer.send(message); + messagesSent.add(message); + } + HashSet messagesReceived = new HashSet<>(); + for (Consumer consumer : consumers) { + while (true) { + Message msg = consumer.receive(2, TimeUnit.SECONDS); + if (msg == null) { + break; + } + messagesReceived.add(msg.getValue()); + } + } + assertEquals(messagesReceived.size(), producers.size()); + + // cleanup. + for (Producer producer: producers) { + producer.close(); + } + for (Consumer consumer : consumers) { + consumer.close(); + } + admin.topics().delete(tpName); + } + + private int calculateLookupRequestCount() throws Exception { + int failures = CollectorRegistry.defaultRegistry.getSampleValue("pulsar_broker_lookup_failures_total") + .intValue(); + int answers = CollectorRegistry.defaultRegistry.getSampleValue("pulsar_broker_lookup_answers_total") + .intValue(); + return failures + answers; + } + @Test(timeOut = 10000) public void testPartitionedMetadataWithDeprecatedVersion() throws Exception { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index d5ce9213211dd..8ceb8e44975c8 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -26,10 +26,12 @@ import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import org.apache.commons.lang3.mutable.MutableObject; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SchemaSerializationException; @@ -56,6 +58,12 @@ public class BinaryProtoLookupService implements LookupService { private final String listenerName; private final int maxLookupRedirects; + private final ConcurrentHashMap>> + lookupInProgress = new ConcurrentHashMap<>(); + + private final ConcurrentHashMap> + partitionedMetadataInProgress = new ConcurrentHashMap<>(); + public BinaryProtoLookupService(PulsarClientImpl client, String serviceUrl, boolean useTls, @@ -92,7 +100,21 @@ public void updateServiceUrl(String serviceUrl) throws PulsarClientException { * @return broker-socket-address that serves given topic */ public CompletableFuture> getBroker(TopicName topicName) { - return findBroker(serviceNameResolver.resolveHost(), false, topicName, 0); + final MutableObject newFutureCreated = new MutableObject<>(); + try { + return lookupInProgress.computeIfAbsent(topicName, tpName -> { + CompletableFuture> newFuture = + findBroker(serviceNameResolver.resolveHost(), false, topicName, 0); + newFutureCreated.setValue(newFuture); + return newFuture; + }); + } finally { + if (newFutureCreated.getValue() != null) { + newFutureCreated.getValue().whenComplete((v, ex) -> { + lookupInProgress.remove(topicName, newFutureCreated.getValue()); + }); + } + } } /** @@ -100,7 +122,21 @@ public CompletableFuture> getBroker(T * */ public CompletableFuture getPartitionedTopicMetadata(TopicName topicName) { - return getPartitionedTopicMetadata(serviceNameResolver.resolveHost(), topicName); + final MutableObject newFutureCreated = new MutableObject<>(); + try { + return partitionedMetadataInProgress.computeIfAbsent(topicName, tpName -> { + CompletableFuture newFuture = + getPartitionedTopicMetadata(serviceNameResolver.resolveHost(), topicName); + newFutureCreated.setValue(newFuture); + return newFuture; + }); + } finally { + if (newFutureCreated.getValue() != null) { + newFutureCreated.getValue().whenComplete((v, ex) -> { + partitionedMetadataInProgress.remove(topicName, newFutureCreated.getValue()); + }); + } + } } private CompletableFuture> findBroker(InetSocketAddress socketAddress, From 31880ec2eb571c4b38097d6e0e8e679200918b6b Mon Sep 17 00:00:00 2001 From: hrzzzz <64506104+hrzzzz@users.noreply.github.com> Date: Wed, 27 Sep 2023 15:45:20 +0800 Subject: [PATCH 070/699] [fix][txn] fix the consumer stuck due to deduplicated messages in pending ack state (#21177) --- .../service/AbstractBaseDispatcher.java | 11 ++-- .../client/impl/TransactionEndToEndTest.java | 59 +++++++++++++++++++ 2 files changed, 65 insertions(+), 5 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java index 50f05f80e3f8b..eb8b015139586 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java @@ -213,12 +213,7 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i this.filterAcceptedMsgs.add(entryMsgCnt); } - totalEntries++; int batchSize = msgMetadata.getNumMessagesInBatch(); - totalMessages += batchSize; - totalBytes += metadataAndPayload.readableBytes(); - totalChunkedMessages += msgMetadata.hasChunkId() ? 1 : 0; - batchSizes.setBatchSize(i, batchSize); long[] ackSet = null; if (indexesAcks != null && cursor != null) { PositionImpl position = PositionImpl.get(entry.getLedgerId(), entry.getEntryId()); @@ -262,6 +257,12 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i } } + totalEntries++; + totalMessages += batchSize; + totalBytes += metadataAndPayload.readableBytes(); + totalChunkedMessages += msgMetadata.hasChunkId() ? 1 : 0; + batchSizes.setBatchSize(i, batchSize); + BrokerInterceptor interceptor = subscription.interceptor(); if (null != interceptor) { // keep for compatibility if users has implemented the old interface diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java index 34cc3bc1ca526..348fb04b7dd23 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java @@ -254,6 +254,65 @@ private void testFilterMsgsInPendingAckStateWhenConsumerDisconnect(boolean enabl Assert.assertEquals(receiveCounter, count / 2); } + @Test + private void testMsgsInPendingAckStateWouldNotGetTheConsumerStuck() throws Exception { + final String topicName = NAMESPACE1 + "/testMsgsInPendingAckStateWouldNotGetTheConsumerStuck"; + final String subscription = "test"; + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.INT32) + .topic(topicName) + .create(); + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.INT32) + .topic(topicName) + .subscriptionName(subscription) + .subscriptionType(SubscriptionType.Shared) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + + int numStep1Receive = 2, numStep2Receive = 2, numStep3Receive = 2; + int numTotalMessage = numStep1Receive + numStep2Receive + numStep3Receive; + + for (int i = 0; i < numTotalMessage; i++) { + producer.send(i); + } + + Transaction step1Txn = getTxn(); + Transaction step2Txn = getTxn(); + + // Step 1, try to consume some messages but do not commit the transaction + for (int i = 0; i < numStep1Receive; i++) { + consumer.acknowledgeAsync(consumer.receive().getMessageId(), step1Txn).get(); + } + + // Step 2, try to consume some messages and commit the transaction + for (int i = 0; i < numStep2Receive; i++) { + consumer.acknowledgeAsync(consumer.receive().getMessageId(), step2Txn).get(); + } + + // commit step2Txn + step2Txn.commit().get(); + + // close and re-create consumer + consumer.close(); + @Cleanup + Consumer consumer2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topicName) + .receiverQueueSize(numStep3Receive) + .subscriptionName(subscription) + .subscriptionType(SubscriptionType.Shared) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + + // Step 3, try to consume the rest messages and should receive all of them + for (int i = 0; i < numStep3Receive; i++) { + // should get the message instead of timeout + Message msg = consumer2.receive(3, TimeUnit.SECONDS); + Assert.assertEquals(msg.getValue(), numStep1Receive + numStep2Receive + i); + } + } + @Test(dataProvider="enableBatch") private void produceCommitTest(boolean enableBatch) throws Exception { @Cleanup From 80fb39085b4e49ff31f2df17b10addcca5abdccb Mon Sep 17 00:00:00 2001 From: Jiwe Guo Date: Wed, 27 Sep 2023 16:29:30 +0800 Subject: [PATCH 071/699] Release 3.1.1 --- 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 +- managed-ledger/pom.xml | 2 +- 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-client-1x-base/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-1x/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-2x-shaded/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/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/hdfs2/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-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-sql/pom.xml | 2 +- pulsar-sql/presto-distribution/pom.xml | 2 +- pulsar-sql/presto-pulsar-plugin/pom.xml | 2 +- pulsar-sql/presto-pulsar/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 +- 133 files changed, 135 insertions(+), 135 deletions(-) diff --git a/bouncy-castle/bc/pom.xml b/bouncy-castle/bc/pom.xml index 6170221e9432e..46b65929d4d5d 100644 --- a/bouncy-castle/bc/pom.xml +++ b/bouncy-castle/bc/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar bouncy-castle-parent - 3.1.0 + 3.1.1 .. diff --git a/bouncy-castle/bcfips-include-test/pom.xml b/bouncy-castle/bcfips-include-test/pom.xml index 678a5c3c804e3..f5b1d1fbad597 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 - 3.1.0 + 3.1.1 .. diff --git a/bouncy-castle/bcfips/pom.xml b/bouncy-castle/bcfips/pom.xml index 0654d746ed9bf..7211174a47228 100644 --- a/bouncy-castle/bcfips/pom.xml +++ b/bouncy-castle/bcfips/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar bouncy-castle-parent - 3.1.0 + 3.1.1 .. diff --git a/bouncy-castle/pom.xml b/bouncy-castle/pom.xml index be5e058fd396a..b06d763c62bc4 100644 --- a/bouncy-castle/pom.xml +++ b/bouncy-castle/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 3.1.0 + 3.1.1 .. diff --git a/buildtools/pom.xml b/buildtools/pom.xml index b4827e63cfaa1..b1cc0dc49c405 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -31,12 +31,12 @@ org.apache.pulsar buildtools - 3.1.0 + 3.1.1 jar Pulsar Build Tools - 2023-08-02T03:24:30Z + 2023-09-27T08:29:08Z 1.8 1.8 3.1.0 diff --git a/distribution/io/pom.xml b/distribution/io/pom.xml index 93aa65c08101f..4a4e626f66ba3 100644 --- a/distribution/io/pom.xml +++ b/distribution/io/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 3.1.0 + 3.1.1 .. diff --git a/distribution/offloaders/pom.xml b/distribution/offloaders/pom.xml index 5f0965cca9585..f1ff90f84d66e 100644 --- a/distribution/offloaders/pom.xml +++ b/distribution/offloaders/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 3.1.0 + 3.1.1 .. diff --git a/distribution/pom.xml b/distribution/pom.xml index 9cd66e14d51f2..f47149bdb0553 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 3.1.0 + 3.1.1 .. diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index 129d05891e2d5..195beac8269a7 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 3.1.0 + 3.1.1 .. diff --git a/distribution/shell/pom.xml b/distribution/shell/pom.xml index d625cdcc1796b..5373ad48659b5 100644 --- a/distribution/shell/pom.xml +++ b/distribution/shell/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar distribution - 3.1.0 + 3.1.1 .. diff --git a/docker/pom.xml b/docker/pom.xml index 87a810f4f7b0a..240c23fba2ba2 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 3.1.0 + 3.1.1 docker-images Apache Pulsar :: Docker Images diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index b269754e6336f..6973b5b04a3fa 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar docker-images - 3.1.0 + 3.1.1 4.0.0 pulsar-all-docker-image diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 4747f05aa8e0c..b1abce3d5506b 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar docker-images - 3.1.0 + 3.1.1 4.0.0 pulsar-docker-image diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml index dc8c0d376c425..b8d5dcbdcfb07 100644 --- a/jclouds-shaded/pom.xml +++ b/jclouds-shaded/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar pulsar - 3.1.0 + 3.1.1 .. diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index 0558edcf69b28..db8f26afba29d 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar pulsar - 3.1.0 + 3.1.1 .. diff --git a/pom.xml b/pom.xml index 0aebdc6a10e30..d405def766890 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ org.apache.pulsar pulsar - 3.1.0 + 3.1.1 Pulsar Pulsar is a distributed pub-sub messaging platform with a very @@ -92,7 +92,7 @@ flexible messaging model and an intuitive client API. UTF-8 UTF-8 - 2023-08-02T03:24:30Z + 2023-09-27T08:29:08Z true + 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 + com.datastax.oss pulsar 3.1.1 .. @@ -128,7 +128,7 @@ - org.apache.pulsar:pulsar-client-original + com.datastax.oss:pulsar-client-original org.apache.bookkeeper:* org.apache.commons:commons-lang3 commons-codec:commons-codec @@ -155,7 +155,7 @@ io.swagger:* io.airlift:* - org.apache.pulsar:pulsar-common + com.datastax.oss:pulsar-common com.yahoo.datasketches:sketches-core org.objenesis:* org.yaml:snakeyaml @@ -166,7 +166,7 @@ org.apache.commons:commons-compress org.tukaani:xz - org.apache.pulsar:pulsar-client-messagecrypto-bc + com.datastax.oss:pulsar-client-messagecrypto-bc com.fasterxml.jackson.core:jackson-annotations @@ -174,7 +174,7 @@ - org.apache.pulsar:pulsar-client-original + com.datastax.oss:pulsar-client-original ** diff --git a/pulsar-client-tools-api/pom.xml b/pulsar-client-tools-api/pom.xml index a6de23a8d581a..b9f8ded828dd3 100644 --- a/pulsar-client-tools-api/pom.xml +++ b/pulsar-client-tools-api/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar 3.1.1 .. diff --git a/pulsar-client-tools-customcommand-example/pom.xml b/pulsar-client-tools-customcommand-example/pom.xml index f8f149637bb03..abac5000a889b 100644 --- a/pulsar-client-tools-customcommand-example/pom.xml +++ b/pulsar-client-tools-customcommand-example/pom.xml @@ -20,7 +20,7 @@ --> - org.apache.pulsar + com.datastax.oss pulsar 3.1.1 .. diff --git a/pulsar-client-tools-test/pom.xml b/pulsar-client-tools-test/pom.xml index 634091270ec22..0312da9b63d84 100644 --- a/pulsar-client-tools-test/pom.xml +++ b/pulsar-client-tools-test/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar 3.1.1 .. diff --git a/pulsar-client-tools/pom.xml b/pulsar-client-tools/pom.xml index 69025edbc495f..f0113e8100b64 100644 --- a/pulsar-client-tools/pom.xml +++ b/pulsar-client-tools/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar 3.1.1 .. diff --git a/pulsar-client/pom.xml b/pulsar-client/pom.xml index 07564b2b153e3..a707fc57aed43 100644 --- a/pulsar-client/pom.xml +++ b/pulsar-client/pom.xml @@ -23,7 +23,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar 3.1.1 .. diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 5b52f6b68fc5d..affc86822a4b6 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -24,7 +24,7 @@ xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"> 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar 3.1.1 .. diff --git a/pulsar-config-validation/pom.xml b/pulsar-config-validation/pom.xml index d36b743c5da2f..992ec42a74b6f 100644 --- a/pulsar-config-validation/pom.xml +++ b/pulsar-config-validation/pom.xml @@ -24,7 +24,7 @@ xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"> 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar 3.1.1 .. diff --git a/pulsar-docs-tools/pom.xml b/pulsar-docs-tools/pom.xml index 36012733d26d8..735ac5e98508d 100644 --- a/pulsar-docs-tools/pom.xml +++ b/pulsar-docs-tools/pom.xml @@ -25,7 +25,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar 3.1.1 diff --git a/pulsar-functions/api-java/pom.xml b/pulsar-functions/api-java/pom.xml index cd9810e08fb64..23abbf55385bc 100644 --- a/pulsar-functions/api-java/pom.xml +++ b/pulsar-functions/api-java/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-functions 3.1.1 diff --git a/pulsar-functions/instance/pom.xml b/pulsar-functions/instance/pom.xml index 1f42912e22af7..f077371efe0f7 100644 --- a/pulsar-functions/instance/pom.xml +++ b/pulsar-functions/instance/pom.xml @@ -23,7 +23,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-functions 3.1.1 diff --git a/pulsar-functions/java-examples-builtin/pom.xml b/pulsar-functions/java-examples-builtin/pom.xml index 2ee1968c04fe7..3006a54bd6e0d 100644 --- a/pulsar-functions/java-examples-builtin/pom.xml +++ b/pulsar-functions/java-examples-builtin/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-functions 3.1.1 diff --git a/pulsar-functions/java-examples/pom.xml b/pulsar-functions/java-examples/pom.xml index 38df16b1b9732..3227b6e98198c 100644 --- a/pulsar-functions/java-examples/pom.xml +++ b/pulsar-functions/java-examples/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-functions 3.1.1 diff --git a/pulsar-functions/localrun-shaded/pom.xml b/pulsar-functions/localrun-shaded/pom.xml index 8ffaed69b5640..a1d200b0fb178 100644 --- a/pulsar-functions/localrun-shaded/pom.xml +++ b/pulsar-functions/localrun-shaded/pom.xml @@ -24,7 +24,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-functions 3.1.1 .. diff --git a/pulsar-functions/localrun/pom.xml b/pulsar-functions/localrun/pom.xml index d88083782a66b..012d56400d69b 100644 --- a/pulsar-functions/localrun/pom.xml +++ b/pulsar-functions/localrun/pom.xml @@ -24,7 +24,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-functions 3.1.1 .. diff --git a/pulsar-functions/pom.xml b/pulsar-functions/pom.xml index efdee839f389c..72c61013eb11d 100644 --- a/pulsar-functions/pom.xml +++ b/pulsar-functions/pom.xml @@ -23,7 +23,7 @@ 4.0.0 pom - org.apache.pulsar + com.datastax.oss pulsar 3.1.1 diff --git a/pulsar-functions/proto/pom.xml b/pulsar-functions/proto/pom.xml index 121f64f4cbd5e..99a12d248f9e9 100644 --- a/pulsar-functions/proto/pom.xml +++ b/pulsar-functions/proto/pom.xml @@ -25,7 +25,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-functions 3.1.1 diff --git a/pulsar-functions/runtime-all/pom.xml b/pulsar-functions/runtime-all/pom.xml index 7a6bf3e381709..ec542f169aac8 100644 --- a/pulsar-functions/runtime-all/pom.xml +++ b/pulsar-functions/runtime-all/pom.xml @@ -24,7 +24,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-functions 3.1.1 .. diff --git a/pulsar-functions/runtime/pom.xml b/pulsar-functions/runtime/pom.xml index 1c88160b1a8e9..b06fd4bb1cda1 100644 --- a/pulsar-functions/runtime/pom.xml +++ b/pulsar-functions/runtime/pom.xml @@ -23,7 +23,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-functions 3.1.1 diff --git a/pulsar-functions/secrets/pom.xml b/pulsar-functions/secrets/pom.xml index a339e0d4fb023..ba27a8a9f0e19 100644 --- a/pulsar-functions/secrets/pom.xml +++ b/pulsar-functions/secrets/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-functions 3.1.1 diff --git a/pulsar-functions/utils/pom.xml b/pulsar-functions/utils/pom.xml index 9036c236d8e0f..1475b81f3b4b8 100644 --- a/pulsar-functions/utils/pom.xml +++ b/pulsar-functions/utils/pom.xml @@ -23,7 +23,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-functions 3.1.1 diff --git a/pulsar-functions/worker/pom.xml b/pulsar-functions/worker/pom.xml index 4181e78fe2818..fba52eedb79a3 100644 --- a/pulsar-functions/worker/pom.xml +++ b/pulsar-functions/worker/pom.xml @@ -23,7 +23,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-functions 3.1.1 .. diff --git a/pulsar-io/aerospike/pom.xml b/pulsar-io/aerospike/pom.xml index 7e511b3bab072..dde84a9df63db 100644 --- a/pulsar-io/aerospike/pom.xml +++ b/pulsar-io/aerospike/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/alluxio/pom.xml b/pulsar-io/alluxio/pom.xml index 8d9bb7c0c96f1..28ca96642329b 100644 --- a/pulsar-io/alluxio/pom.xml +++ b/pulsar-io/alluxio/pom.xml @@ -24,7 +24,7 @@ 4.0.0 pulsar-io - org.apache.pulsar + com.datastax.oss 3.1.1 diff --git a/pulsar-io/aws/pom.xml b/pulsar-io/aws/pom.xml index b7e50635b0512..5613cb9e0bb48 100644 --- a/pulsar-io/aws/pom.xml +++ b/pulsar-io/aws/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/batch-data-generator/pom.xml b/pulsar-io/batch-data-generator/pom.xml index d752f21b9ec11..d012b69bfd351 100644 --- a/pulsar-io/batch-data-generator/pom.xml +++ b/pulsar-io/batch-data-generator/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/batch-discovery-triggerers/pom.xml b/pulsar-io/batch-discovery-triggerers/pom.xml index afd8a07354f7e..7c13c09921289 100644 --- a/pulsar-io/batch-discovery-triggerers/pom.xml +++ b/pulsar-io/batch-discovery-triggerers/pom.xml @@ -23,7 +23,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/canal/pom.xml b/pulsar-io/canal/pom.xml index 37dfd136676b0..065985ef8465a 100644 --- a/pulsar-io/canal/pom.xml +++ b/pulsar-io/canal/pom.xml @@ -23,7 +23,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> - org.apache.pulsar + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/cassandra/pom.xml b/pulsar-io/cassandra/pom.xml index c6d11bdd14eac..e1d1056e82b1d 100644 --- a/pulsar-io/cassandra/pom.xml +++ b/pulsar-io/cassandra/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/common/pom.xml b/pulsar-io/common/pom.xml index ba9fbb4fb9141..8acfa4397e8bd 100644 --- a/pulsar-io/common/pom.xml +++ b/pulsar-io/common/pom.xml @@ -25,7 +25,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/core/pom.xml b/pulsar-io/core/pom.xml index 1296d19a0b66d..920282159bb8a 100644 --- a/pulsar-io/core/pom.xml +++ b/pulsar-io/core/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/data-generator/pom.xml b/pulsar-io/data-generator/pom.xml index 466f11173fb05..54d45ebeee34b 100644 --- a/pulsar-io/data-generator/pom.xml +++ b/pulsar-io/data-generator/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/debezium/core/pom.xml b/pulsar-io/debezium/core/pom.xml index 4d33d3fac4df7..e265de906f3e4 100644 --- a/pulsar-io/debezium/core/pom.xml +++ b/pulsar-io/debezium/core/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io-debezium 3.1.1 diff --git a/pulsar-io/debezium/mongodb/pom.xml b/pulsar-io/debezium/mongodb/pom.xml index 208b6888078aa..f62b9208aa8ac 100644 --- a/pulsar-io/debezium/mongodb/pom.xml +++ b/pulsar-io/debezium/mongodb/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io-debezium 3.1.1 diff --git a/pulsar-io/debezium/mssql/pom.xml b/pulsar-io/debezium/mssql/pom.xml index 45c02ae5076e0..8580d17cd8b64 100644 --- a/pulsar-io/debezium/mssql/pom.xml +++ b/pulsar-io/debezium/mssql/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io-debezium 3.1.1 diff --git a/pulsar-io/debezium/mysql/pom.xml b/pulsar-io/debezium/mysql/pom.xml index 4cadf4f59f765..338bcd028d86e 100644 --- a/pulsar-io/debezium/mysql/pom.xml +++ b/pulsar-io/debezium/mysql/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io-debezium 3.1.1 diff --git a/pulsar-io/debezium/oracle/pom.xml b/pulsar-io/debezium/oracle/pom.xml index d0fe74f7cac3e..ab0cdde410a47 100644 --- a/pulsar-io/debezium/oracle/pom.xml +++ b/pulsar-io/debezium/oracle/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io-debezium 3.1.1 diff --git a/pulsar-io/debezium/pom.xml b/pulsar-io/debezium/pom.xml index 112b197f5aae3..1effdd6c0c07d 100644 --- a/pulsar-io/debezium/pom.xml +++ b/pulsar-io/debezium/pom.xml @@ -23,7 +23,7 @@ 4.0.0 pom - org.apache.pulsar + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/debezium/postgres/pom.xml b/pulsar-io/debezium/postgres/pom.xml index 9e6abf9724f28..feb70de92fbae 100644 --- a/pulsar-io/debezium/postgres/pom.xml +++ b/pulsar-io/debezium/postgres/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io-debezium 3.1.1 diff --git a/pulsar-io/docs/pom.xml b/pulsar-io/docs/pom.xml index f67d351053acf..0f5f20ec34665 100644 --- a/pulsar-io/docs/pom.xml +++ b/pulsar-io/docs/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/dynamodb/pom.xml b/pulsar-io/dynamodb/pom.xml index e63be2e564f32..5ad7867ec4053 100644 --- a/pulsar-io/dynamodb/pom.xml +++ b/pulsar-io/dynamodb/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/elastic-search/pom.xml b/pulsar-io/elastic-search/pom.xml index af63119016e7e..ca2d40169d5ed 100644 --- a/pulsar-io/elastic-search/pom.xml +++ b/pulsar-io/elastic-search/pom.xml @@ -21,7 +21,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/file/pom.xml b/pulsar-io/file/pom.xml index e3f6aba4c8288..6723a6ee351b3 100644 --- a/pulsar-io/file/pom.xml +++ b/pulsar-io/file/pom.xml @@ -21,7 +21,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/flume/pom.xml b/pulsar-io/flume/pom.xml index c60751a6180f8..833f8f9009ed7 100644 --- a/pulsar-io/flume/pom.xml +++ b/pulsar-io/flume/pom.xml @@ -23,7 +23,7 @@ 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 + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/hbase/pom.xml b/pulsar-io/hbase/pom.xml index 31eeb6872247f..c674544992cbc 100644 --- a/pulsar-io/hbase/pom.xml +++ b/pulsar-io/hbase/pom.xml @@ -24,7 +24,7 @@ 4.0.0 pulsar-io - org.apache.pulsar + com.datastax.oss 3.1.1 pulsar-io-hbase diff --git a/pulsar-io/hdfs2/pom.xml b/pulsar-io/hdfs2/pom.xml index 592d3b05de257..9466406978af9 100644 --- a/pulsar-io/hdfs2/pom.xml +++ b/pulsar-io/hdfs2/pom.xml @@ -21,7 +21,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/hdfs3/pom.xml b/pulsar-io/hdfs3/pom.xml index 43672b261d6fe..18aac80c8356e 100644 --- a/pulsar-io/hdfs3/pom.xml +++ b/pulsar-io/hdfs3/pom.xml @@ -21,7 +21,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/http/pom.xml b/pulsar-io/http/pom.xml index 789a0bab1b886..3744e2f6fc4a0 100644 --- a/pulsar-io/http/pom.xml +++ b/pulsar-io/http/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/influxdb/pom.xml b/pulsar-io/influxdb/pom.xml index 3130999a31a99..6b609208ea7e6 100644 --- a/pulsar-io/influxdb/pom.xml +++ b/pulsar-io/influxdb/pom.xml @@ -24,7 +24,7 @@ 4.0.0 pulsar-io - org.apache.pulsar + com.datastax.oss 3.1.1 diff --git a/pulsar-io/jdbc/clickhouse/pom.xml b/pulsar-io/jdbc/clickhouse/pom.xml index 4ecf066027ff7..c2f97c4720b6b 100644 --- a/pulsar-io/jdbc/clickhouse/pom.xml +++ b/pulsar-io/jdbc/clickhouse/pom.xml @@ -23,7 +23,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> pulsar-io-jdbc - org.apache.pulsar + com.datastax.oss 3.1.1 4.0.0 diff --git a/pulsar-io/jdbc/core/pom.xml b/pulsar-io/jdbc/core/pom.xml index 52caee78fc160..af5bb33647b8c 100644 --- a/pulsar-io/jdbc/core/pom.xml +++ b/pulsar-io/jdbc/core/pom.xml @@ -23,7 +23,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> pulsar-io-jdbc - org.apache.pulsar + com.datastax.oss 3.1.1 4.0.0 diff --git a/pulsar-io/jdbc/mariadb/pom.xml b/pulsar-io/jdbc/mariadb/pom.xml index fc70db7044d75..7baf21d53bbb9 100644 --- a/pulsar-io/jdbc/mariadb/pom.xml +++ b/pulsar-io/jdbc/mariadb/pom.xml @@ -23,7 +23,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> pulsar-io-jdbc - org.apache.pulsar + com.datastax.oss 3.1.1 4.0.0 diff --git a/pulsar-io/jdbc/openmldb/pom.xml b/pulsar-io/jdbc/openmldb/pom.xml index 6017cf7399574..32377db9999c8 100644 --- a/pulsar-io/jdbc/openmldb/pom.xml +++ b/pulsar-io/jdbc/openmldb/pom.xml @@ -23,7 +23,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> pulsar-io-jdbc - org.apache.pulsar + com.datastax.oss 3.1.1 4.0.0 diff --git a/pulsar-io/jdbc/pom.xml b/pulsar-io/jdbc/pom.xml index 767d62a569040..bee2df61a2cf3 100644 --- a/pulsar-io/jdbc/pom.xml +++ b/pulsar-io/jdbc/pom.xml @@ -31,7 +31,7 @@ openmldb - org.apache.pulsar + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/jdbc/postgres/pom.xml b/pulsar-io/jdbc/postgres/pom.xml index a3e1492ad801c..246c1b01c8225 100644 --- a/pulsar-io/jdbc/postgres/pom.xml +++ b/pulsar-io/jdbc/postgres/pom.xml @@ -23,7 +23,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> pulsar-io-jdbc - org.apache.pulsar + com.datastax.oss 3.1.1 4.0.0 diff --git a/pulsar-io/jdbc/sqlite/pom.xml b/pulsar-io/jdbc/sqlite/pom.xml index 8daa9f6ccf111..21a3edbb9d4ab 100644 --- a/pulsar-io/jdbc/sqlite/pom.xml +++ b/pulsar-io/jdbc/sqlite/pom.xml @@ -23,7 +23,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> pulsar-io-jdbc - org.apache.pulsar + com.datastax.oss 3.1.1 4.0.0 diff --git a/pulsar-io/kafka-connect-adaptor-nar/pom.xml b/pulsar-io/kafka-connect-adaptor-nar/pom.xml index 124bbf02cb5a1..eb003a9d9f8c1 100644 --- a/pulsar-io/kafka-connect-adaptor-nar/pom.xml +++ b/pulsar-io/kafka-connect-adaptor-nar/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/kafka-connect-adaptor/pom.xml b/pulsar-io/kafka-connect-adaptor/pom.xml index 786efa8fd02f9..59f3ceb2ccf1c 100644 --- a/pulsar-io/kafka-connect-adaptor/pom.xml +++ b/pulsar-io/kafka-connect-adaptor/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/kafka/pom.xml b/pulsar-io/kafka/pom.xml index e501470878357..6ab5b247c137d 100644 --- a/pulsar-io/kafka/pom.xml +++ b/pulsar-io/kafka/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/kinesis/pom.xml b/pulsar-io/kinesis/pom.xml index 16d081fb81647..b859532bc29e3 100644 --- a/pulsar-io/kinesis/pom.xml +++ b/pulsar-io/kinesis/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/mongo/pom.xml b/pulsar-io/mongo/pom.xml index ce81fba71d597..e288d4da0a2ca 100644 --- a/pulsar-io/mongo/pom.xml +++ b/pulsar-io/mongo/pom.xml @@ -24,7 +24,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/netty/pom.xml b/pulsar-io/netty/pom.xml index 71baf7ed36e4f..8b027ab0c6054 100644 --- a/pulsar-io/netty/pom.xml +++ b/pulsar-io/netty/pom.xml @@ -23,7 +23,7 @@ xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"> 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/nsq/pom.xml b/pulsar-io/nsq/pom.xml index 731e05e61886c..f09ad6bd96fd1 100644 --- a/pulsar-io/nsq/pom.xml +++ b/pulsar-io/nsq/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/pom.xml b/pulsar-io/pom.xml index 26f3762c434e1..8d36fd5267e5a 100644 --- a/pulsar-io/pom.xml +++ b/pulsar-io/pom.xml @@ -23,7 +23,7 @@ 4.0.0 pom - org.apache.pulsar + com.datastax.oss pulsar 3.1.1 diff --git a/pulsar-io/rabbitmq/pom.xml b/pulsar-io/rabbitmq/pom.xml index 40b71b91615a9..1dc7b4e056532 100644 --- a/pulsar-io/rabbitmq/pom.xml +++ b/pulsar-io/rabbitmq/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-io/redis/pom.xml b/pulsar-io/redis/pom.xml index 4ec19e977b021..ce95e9f8ae466 100644 --- a/pulsar-io/redis/pom.xml +++ b/pulsar-io/redis/pom.xml @@ -24,7 +24,7 @@ 4.0.0 pulsar-io - org.apache.pulsar + com.datastax.oss 3.1.1 diff --git a/pulsar-io/solr/pom.xml b/pulsar-io/solr/pom.xml index 53ff251bf0def..5fcb97b7c56d1 100644 --- a/pulsar-io/solr/pom.xml +++ b/pulsar-io/solr/pom.xml @@ -24,7 +24,7 @@ 4.0.0 pulsar-io - org.apache.pulsar + com.datastax.oss 3.1.1 diff --git a/pulsar-io/twitter/pom.xml b/pulsar-io/twitter/pom.xml index 3b875ef786546..9b7aac7bb3f37 100644 --- a/pulsar-io/twitter/pom.xml +++ b/pulsar-io/twitter/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar-io 3.1.1 diff --git a/pulsar-metadata/pom.xml b/pulsar-metadata/pom.xml index 429ca2af1ccf9..a868fa2bd9774 100644 --- a/pulsar-metadata/pom.xml +++ b/pulsar-metadata/pom.xml @@ -23,7 +23,7 @@ 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 + com.datastax.oss pulsar 3.1.1 .. diff --git a/pulsar-package-management/bookkeeper-storage/pom.xml b/pulsar-package-management/bookkeeper-storage/pom.xml index 8f8bf4e888f97..72920e30ba513 100644 --- a/pulsar-package-management/bookkeeper-storage/pom.xml +++ b/pulsar-package-management/bookkeeper-storage/pom.xml @@ -24,7 +24,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> pulsar-package-management - org.apache.pulsar + com.datastax.oss 3.1.1 4.0.0 diff --git a/pulsar-package-management/core/pom.xml b/pulsar-package-management/core/pom.xml index de27c1e80859c..e4af2999fad79 100644 --- a/pulsar-package-management/core/pom.xml +++ b/pulsar-package-management/core/pom.xml @@ -24,7 +24,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> pulsar-package-management - org.apache.pulsar + com.datastax.oss 3.1.1 4.0.0 diff --git a/pulsar-package-management/filesystem-storage/pom.xml b/pulsar-package-management/filesystem-storage/pom.xml index 420fc9e8850d2..14aa1cff1c24a 100644 --- a/pulsar-package-management/filesystem-storage/pom.xml +++ b/pulsar-package-management/filesystem-storage/pom.xml @@ -24,7 +24,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> pulsar-package-management - org.apache.pulsar + com.datastax.oss 3.1.1 4.0.0 diff --git a/pulsar-package-management/pom.xml b/pulsar-package-management/pom.xml index 51033cfac74ac..766341c8b0752 100644 --- a/pulsar-package-management/pom.xml +++ b/pulsar-package-management/pom.xml @@ -24,7 +24,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> pulsar - org.apache.pulsar + com.datastax.oss 3.1.1 .. diff --git a/pulsar-proxy/pom.xml b/pulsar-proxy/pom.xml index 8575ba49ca9e4..cbc62c5faed28 100644 --- a/pulsar-proxy/pom.xml +++ b/pulsar-proxy/pom.xml @@ -22,7 +22,7 @@ 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 + com.datastax.oss pulsar 3.1.1 diff --git a/pulsar-sql/pom.xml b/pulsar-sql/pom.xml index 0b96d4c17b7b0..3e2d5574a8af4 100644 --- a/pulsar-sql/pom.xml +++ b/pulsar-sql/pom.xml @@ -23,7 +23,7 @@ 4.0.0 pom - org.apache.pulsar + com.datastax.oss pulsar 3.1.1 diff --git a/pulsar-sql/presto-distribution/pom.xml b/pulsar-sql/presto-distribution/pom.xml index 10defb12d04f8..353445fd436ae 100644 --- a/pulsar-sql/presto-distribution/pom.xml +++ b/pulsar-sql/presto-distribution/pom.xml @@ -23,7 +23,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-sql 3.1.1 diff --git a/pulsar-sql/presto-pulsar-plugin/pom.xml b/pulsar-sql/presto-pulsar-plugin/pom.xml index 4cd2da3480c8e..5ca7f3546b4f8 100644 --- a/pulsar-sql/presto-pulsar-plugin/pom.xml +++ b/pulsar-sql/presto-pulsar-plugin/pom.xml @@ -23,7 +23,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-sql 3.1.1 diff --git a/pulsar-sql/presto-pulsar/pom.xml b/pulsar-sql/presto-pulsar/pom.xml index 7d3174c455ca3..c196e133a7ff7 100644 --- a/pulsar-sql/presto-pulsar/pom.xml +++ b/pulsar-sql/presto-pulsar/pom.xml @@ -23,7 +23,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-sql 3.1.1 diff --git a/pulsar-testclient/pom.xml b/pulsar-testclient/pom.xml index 7936af599325b..969e5e832e1fc 100644 --- a/pulsar-testclient/pom.xml +++ b/pulsar-testclient/pom.xml @@ -23,7 +23,7 @@ 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 + com.datastax.oss pulsar 3.1.1 .. diff --git a/pulsar-transaction/common/pom.xml b/pulsar-transaction/common/pom.xml index 2108130270fb4..095cecf105fc9 100644 --- a/pulsar-transaction/common/pom.xml +++ b/pulsar-transaction/common/pom.xml @@ -25,7 +25,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-transaction-parent 3.1.1 diff --git a/pulsar-transaction/coordinator/pom.xml b/pulsar-transaction/coordinator/pom.xml index de86ff37076ee..c877a653f98eb 100644 --- a/pulsar-transaction/coordinator/pom.xml +++ b/pulsar-transaction/coordinator/pom.xml @@ -25,7 +25,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar-transaction-parent 3.1.1 diff --git a/pulsar-transaction/pom.xml b/pulsar-transaction/pom.xml index 8c11eca0d758c..a43b77ecc1fb1 100644 --- a/pulsar-transaction/pom.xml +++ b/pulsar-transaction/pom.xml @@ -23,7 +23,7 @@ 4.0.0 pom - org.apache.pulsar + com.datastax.oss pulsar 3.1.1 diff --git a/pulsar-websocket/pom.xml b/pulsar-websocket/pom.xml index 50988918dadcb..4f41498ef4a5f 100644 --- a/pulsar-websocket/pom.xml +++ b/pulsar-websocket/pom.xml @@ -23,7 +23,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar 3.1.1 .. diff --git a/structured-event-log/pom.xml b/structured-event-log/pom.xml index 23343f3b64352..382504c32ace5 100644 --- a/structured-event-log/pom.xml +++ b/structured-event-log/pom.xml @@ -23,7 +23,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar 3.1.1 .. diff --git a/testmocks/pom.xml b/testmocks/pom.xml index 345e4dcf50f7a..a76b847d9014f 100644 --- a/testmocks/pom.xml +++ b/testmocks/pom.xml @@ -24,7 +24,7 @@ pulsar - org.apache.pulsar + com.datastax.oss 3.1.1 diff --git a/tests/pom.xml b/tests/pom.xml index c763db5eb4b06..ec19f0156b3c2 100644 --- a/tests/pom.xml +++ b/tests/pom.xml @@ -24,7 +24,7 @@ pom 4.0.0 - org.apache.pulsar + com.datastax.oss pulsar 3.1.1 diff --git a/tiered-storage/file-system/pom.xml b/tiered-storage/file-system/pom.xml index 56c89180756eb..a739e406441b2 100644 --- a/tiered-storage/file-system/pom.xml +++ b/tiered-storage/file-system/pom.xml @@ -23,7 +23,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss tiered-storage-parent 3.1.1 .. diff --git a/tiered-storage/jcloud/pom.xml b/tiered-storage/jcloud/pom.xml index 72db7e29e780c..49ee5ea8f8c46 100644 --- a/tiered-storage/jcloud/pom.xml +++ b/tiered-storage/jcloud/pom.xml @@ -23,7 +23,7 @@ 4.0.0 - org.apache.pulsar + com.datastax.oss tiered-storage-parent 3.1.1 .. diff --git a/tiered-storage/pom.xml b/tiered-storage/pom.xml index b170bb8731045..cbc028da7fd86 100644 --- a/tiered-storage/pom.xml +++ b/tiered-storage/pom.xml @@ -23,7 +23,7 @@ 4.0.0 pom - org.apache.pulsar + com.datastax.oss pulsar 3.1.1 .. From 36fbec5ae0298f7ecbdd1572bbe87de7e9607fd7 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Tue, 5 Dec 2023 17:28:44 +0530 Subject: [PATCH 073/699] Removed binary licenses check --- .github/workflows/pulsar-ci.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index b2681db9602f9..f7c1ea09caba8 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -142,8 +142,8 @@ jobs: run: | mvn -B -T 1C -ntp -Pcore-modules,-main clean install -DskipTests -Dlicense.skip=true -Drat.skip=true -Dcheckstyle.skip=true - - name: Check binary licenses - run: src/check-binary-license.sh ./distribution/server/target/apache-pulsar-*-bin.tar.gz +# - name: Check binary licenses +# run: src/check-binary-license.sh ./distribution/server/target/apache-pulsar-*-bin.tar.gz - name: Install gh-actions-artifact-client.js uses: apache/pulsar-test-infra/gh-actions-artifact-client/dist@master From 34e69b51534db1add89a2f86535040126df6de11 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Tue, 5 Dec 2023 17:41:29 +0530 Subject: [PATCH 074/699] Removed binary full licenses check --- .github/workflows/pulsar-ci.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index f7c1ea09caba8..64708d940a9fb 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -789,8 +789,8 @@ jobs: -Dspotbugs.skip=true -Dlicense.skip=true -Dcheckstyle.skip=true -Drat.skip=true # check full build artifacts licenses - - name: Check binary licenses - 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: Check binary licenses +# 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: Clean up disk space run: | From 26a4af7990beb97e09ad84e301fa83831b7679b6 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Wed, 6 Dec 2023 10:27:40 +0530 Subject: [PATCH 075/699] Updated package name --- build/run_unit_group.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build/run_unit_group.sh b/build/run_unit_group.sh index 69434b011b37e..216b239dbf1b7 100755 --- a/build/run_unit_group.sh +++ b/build/run_unit_group.sh @@ -151,7 +151,7 @@ function test_group_proxy() { function test_group_other() { mvn_test --clean --install \ - -pl '!org.apache.pulsar:distribution,!org.apache.pulsar:pulsar-offloader-distribution,!org.apache.pulsar:pulsar-server-distribution,!org.apache.pulsar:pulsar-io-distribution,!org.apache.pulsar:pulsar-all-docker-image' \ + -pl '!com.datastax.oss:distribution,!com.datastax.oss:pulsar-offloader-distribution,!com.datastax.oss:pulsar-server-distribution,!com.datastax.oss:pulsar-io-distribution,!com.datastax.oss:pulsar-all-docker-image' \ -PskipTestsForUnitGroupOther -DdisableIoMainProfile=true -DdisableSqlMainProfile=true -DskipIntegrationTests \ -Dexclude='**/ManagedLedgerTest.java, **/OffloadersCacheTest.java From 5b8256c94b087932833a2380d6e1e654ca545579 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 9 Oct 2023 18:21:16 +0530 Subject: [PATCH 076/699] [fix][test] Fix flaky test NarUnpackerTest (#21328) (cherry picked from commit e76a86e3cd1c362e9daa1c88eb8b888e6ab38ab4) --- .../pulsar/common/nar/NarUnpackerTest.java | 30 +++++++++++++------ 1 file changed, 21 insertions(+), 9 deletions(-) diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/nar/NarUnpackerTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/nar/NarUnpackerTest.java index c6c5ee180f69a..a1f915c8b7828 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/nar/NarUnpackerTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/nar/NarUnpackerTest.java @@ -38,6 +38,7 @@ import org.testng.annotations.Test; @Slf4j +@Test public class NarUnpackerTest { File sampleZipFile; File extractDirectory; @@ -46,7 +47,7 @@ public class NarUnpackerTest { public void createSampleZipFile() throws IOException { sampleZipFile = Files.createTempFile("sample", ".zip").toFile(); try (ZipOutputStream out = new ZipOutputStream(new FileOutputStream(sampleZipFile))) { - for (int i = 0; i < 10000; i++) { + for (int i = 0; i < 5000; i++) { ZipEntry e = new ZipEntry("hello" + i + ".txt"); out.putNextEntry(e); byte[] msg = "hello world!".getBytes(StandardCharsets.UTF_8); @@ -58,12 +59,20 @@ public void createSampleZipFile() throws IOException { } @AfterMethod(alwaysRun = true) - void deleteSampleZipFile() throws IOException { - if (sampleZipFile != null) { - sampleZipFile.delete(); + void deleteSampleZipFile() { + if (sampleZipFile != null && sampleZipFile.exists()) { + try { + sampleZipFile.delete(); + } catch (Exception e) { + log.warn("Failed to delete file {}", sampleZipFile, e); + } } - if (extractDirectory != null) { - FileUtils.deleteFile(extractDirectory, true); + if (extractDirectory != null && extractDirectory.exists()) { + try { + FileUtils.deleteFile(extractDirectory, true); + } catch (IOException e) { + log.warn("Failed to delete directory {}", extractDirectory, e); + } } } @@ -111,7 +120,7 @@ public static void main(String[] args) { @Test void shouldExtractFilesOnceInDifferentProcess() throws InterruptedException { - int processes = 10; + int processes = 5; String javaExePath = findJavaExe().getAbsolutePath(); CountDownLatch countDownLatch = new CountDownLatch(processes); AtomicInteger exceptionCounter = new AtomicInteger(); @@ -122,7 +131,9 @@ void shouldExtractFilesOnceInDifferentProcess() throws InterruptedException { // fork a new process with the same classpath Process process = new ProcessBuilder() .command(javaExePath, - "-Xmx64m", + "-Xmx96m", + "-XX:TieredStopAtLevel=1", + "-Dlog4j2.disable.jmx=true", "-cp", System.getProperty("java.class.path"), // use NarUnpackerWorker as the main class @@ -130,6 +141,7 @@ void shouldExtractFilesOnceInDifferentProcess() throws InterruptedException { // pass arguments to use for testing sampleZipFile.getAbsolutePath(), extractDirectory.getAbsolutePath()) + .redirectErrorStream(true) .start(); String output = IOUtils.toString(process.getInputStream(), StandardCharsets.UTF_8); int retval = process.waitFor(); @@ -147,7 +159,7 @@ void shouldExtractFilesOnceInDifferentProcess() throws InterruptedException { } }).start(); } - assertTrue(countDownLatch.await(30, TimeUnit.SECONDS)); + assertTrue(countDownLatch.await(30, TimeUnit.SECONDS), "All processes should finish before timeout"); assertEquals(exceptionCounter.get(), 0); assertEquals(extractCounter.get(), 1); } From a904474aa7b048c096f6a9fec5ed44763cd4b370 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 9 Oct 2023 17:48:38 +0300 Subject: [PATCH 077/699] [fix][test] Fix flaky CompactionTest.testDispatcherMaxReadSizeBytes (#21329) (cherry picked from commit c883f50e117ff9da310c879aa048993c28ea955a) --- .../apache/pulsar/compaction/CompactionTest.java | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) 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 4c6db644f1e01..52837cbdcd56a 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 @@ -1898,22 +1898,21 @@ public void testDispatcherMaxReadSizeBytes() throws Exception { admin.topics().unload(topicName); - ConsumerImpl consumer = (ConsumerImpl) client.newConsumer(Schema.BYTES) - .topic(topicName).readCompacted(true).receiverQueueSize(receiveQueueSize).subscriptionName(subName) - .subscribe(); - - PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get(); + PersistentTopic topic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, true, Map.of()).get().get(); TopicCompactionService topicCompactionService = Mockito.spy(topic.getTopicCompactionService()); FieldUtils.writeDeclaredField(topic, "topicCompactionService", topicCompactionService, true); + ConsumerImpl consumer = (ConsumerImpl) client.newConsumer(Schema.BYTES) + .topic(topicName).readCompacted(true).receiverQueueSize(receiveQueueSize).subscriptionName(subName) + .subscribe(); + Awaitility.await().untilAsserted(() -> { assertEquals(consumer.getStats().getMsgNumInReceiverQueue(), 1); }); - consumer.increaseAvailablePermits(2); - Mockito.verify(topicCompactionService, Mockito.times(1)).readCompactedEntries(Mockito.any(), Mockito.same(1)); consumer.close(); From a392c2ba474fd1a675c99a5f689f7db443e37d00 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Wed, 6 Dec 2023 17:17:25 +0530 Subject: [PATCH 078/699] Ignored testCassandraArchiveSink --- .../pulsar/tests/integration/io/sinks/PulsarSinksTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarSinksTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarSinksTest.java index 0663b4d44ff0f..703178b7ce484 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarSinksTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarSinksTest.java @@ -50,7 +50,7 @@ public void testCassandraSink() throws Exception { testSink(CassandraSinkTester.createTester(true), true); } - @Test(groups = "sink") + //@Test(groups = "sink") public void testCassandraArchiveSink() throws Exception { testSink(CassandraSinkTester.createTester(false), false); } From 2c9243c7d790c1273468b17e41f2cdf83ca719cd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Nicol=C3=B2=20Boschi?= Date: Fri, 29 Apr 2022 10:15:40 +0200 Subject: [PATCH 079/699] Remove batch-data-generator from the test docker image (cherry picked from commit 09444147fee3f44eea4ede066b55669a66a6f1ae) --- tests/docker-images/latest-version-image/Dockerfile | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/docker-images/latest-version-image/Dockerfile b/tests/docker-images/latest-version-image/Dockerfile index 99672773dcbc8..ade472ded049e 100644 --- a/tests/docker-images/latest-version-image/Dockerfile +++ b/tests/docker-images/latest-version-image/Dockerfile @@ -94,7 +94,6 @@ 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 From d1baa34336ceeadb4c50103af023e42008648932 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Nicol=C3=B2=20Boschi?= Date: Mon, 9 May 2022 13:27:20 +0200 Subject: [PATCH 080/699] Disable integration tests testing connector not included in the test docker image (cherry picked from commit c6e065031d0ba74f90642f1ab164a68d3eba89ff) --- .../pulsar/tests/integration/io/sinks/PulsarSinksTest.java | 2 +- .../pulsar/tests/integration/io/sources/BatchSourceTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarSinksTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarSinksTest.java index 703178b7ce484..ca21c4f89b9d2 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarSinksTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarSinksTest.java @@ -80,7 +80,7 @@ public void testOpenSearchSinkRawData(boolean withSchema) throws Exception { testSink(new OpenSearchSinkTester(withSchema), true); } - @Test(groups = "sink") + @Test(enabled = false, groups = "sink") public void testRabbitMQSink() throws Exception { final String containerName = "rabbitmq-" + randomName(8); testSink(new RabbitMQSinkTester(containerName), true, new RabbitMQSourceTester(containerName)); diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/BatchSourceTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/BatchSourceTest.java index 19f74bf3878fe..8fc3115f70727 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/BatchSourceTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/BatchSourceTest.java @@ -55,7 +55,7 @@ public class BatchSourceTest extends PulsarStandaloneTestSuite { private static final String BATCH_CONFIG = "{\"discoveryTriggererConfig\": {\"__CRON__\": \"* * * * * *\"}, " + "\"discoveryTriggererClassName\": \"org.apache.pulsar.io.batchdiscovery.CronTriggerer\"}"; - @Test(groups = {"source"}) + @Test(enabled = false, groups = {"source"}) public void testGenericRecordSource() throws Exception { String outputTopicName = "test-state-source-output-" + randomName(8); String sourceName = "test-state-source-" + randomName(8); From 3e1ff14ac9f24572f18b59cabbc3329028bef691 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Nicol=C3=B2=20Boschi?= Date: Thu, 19 May 2022 13:54:33 +0200 Subject: [PATCH 081/699] Add artifacts deployment to Sonatype (cherry picked from commit 4ff4f90ab3f15fd4d5e8de9d39c23d872fcac83a) --- pom.xml | 132 +++++++++++++++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 131 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index fa7352d250f39..b986f246a30c3 100644 --- a/pom.xml +++ b/pom.xml @@ -37,7 +37,7 @@ Pulsar Pulsar is a distributed pub-sub messaging platform with a very flexible messaging model and an intuitive client API. - https://github.com/apache/pulsar + https://github.com/datastax/pulsar Apache Software Foundation @@ -2528,6 +2528,136 @@ flexible messaging model and an intuitive client API. pulsar-sql + + + + + datastax-release + + + + org.apache.maven.plugins + maven-assembly-plugin + + + org.apache.apache.resources + apache-source-release-assembly-descriptor + 1.0.6 + + + + + source-release-assembly + package + + single + + + true + + ${sourceReleaseAssemblyDescriptor} + + posix + + + + + + + true + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + + jar-no-fork + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + + attach-javadocs + + jar + + + + + + + org.apache.maven.plugins + maven-gpg-plugin + + + sign-release-artifacts + + sign + + + + --pinentry-mode + loopback + + + + + + + net.nicoulaj.maven.plugins + checksum-maven-plugin + 1.7 + + + source-release-checksum + + files + + + + + + SHA-512 + + false + + + ${project.build.directory} + + ${project.artifactId}-${project.version}-source-release.zip + ${project.artifactId}-${project.version}-source-release.tar* + + + + false + + + + + + + + ossrh + https://oss.sonatype.org/content/repositories/snapshots + + + ossrh + https://oss.sonatype.org/service/local/staging/deploy/maven2 + + + From 0817e1ee0baaa749c5a184303dcb0a2dfa42fc1a Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Tue, 5 Jul 2022 15:43:00 +0200 Subject: [PATCH 082/699] Offloaders: fix metrics - pass the Scheduler for periodic calculations - add raw brk_ledgeroffloader_read_bytes counter - fix read latency from JClouds calculation (cherry picked from commit cc4c7be3916dec04761ef156ede29f26c776fa34) Conflicts: managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LedgerOffloaderStatsImpl.java pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java --- .../bookkeeper/mledger/impl/LedgerOffloaderStatsImpl.java | 3 ++- .../src/main/java/org/apache/pulsar/broker/PulsarService.java | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LedgerOffloaderStatsImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LedgerOffloaderStatsImpl.java index 5e05e4c8137cd..e8d0b3488f617 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LedgerOffloaderStatsImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/LedgerOffloaderStatsImpl.java @@ -31,12 +31,13 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.LongAdder; import java.util.stream.Collectors; +import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.LedgerOffloaderStats; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.common.naming.TopicName; - +@Slf4j public final class LedgerOffloaderStatsImpl implements LedgerOffloaderStats, Runnable { private static final String TOPIC_LABEL = "topic"; private static final String NAMESPACE_LABEL = "namespace"; 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 4ffb5b77d5424..620eb68fb2143 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 @@ -1,4 +1,4 @@ -/* +/** * 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 From 6e51df85cef8d2867073c19b0e610acba1bf8f02 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Thu, 7 Dec 2023 13:32:23 +0530 Subject: [PATCH 083/699] Fix build issue introduced by 4ea1946ac1a1cf58e7c10dc533c38479465c8c24 (cherry-pick of 4ff4f90ab3f15fd4d5e8de9d39c23d872fcac83a) --- pom.xml | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index b986f246a30c3..eb23574114a01 100644 --- a/pom.xml +++ b/pom.xml @@ -2607,10 +2607,12 @@ flexible messaging model and an intuitive client API. - - --pinentry-mode - loopback - + + + --pinentry-mode + loopback + + From 9c65feae6912b310f5571eee18a4e9ac474d06d6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Nicol=C3=B2=20Boschi?= Date: Mon, 19 Sep 2022 11:35:40 +0200 Subject: [PATCH 084/699] [cli] Topic subscriptions visualizer (#133) (cherry picked from commit 7bba51fab2151ae0b81f12d22da4347f8c696715) Conflicts: pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java --- .../apache/pulsar/admin/cli/CmdTopics.java | 37 ++- .../utils/TopicSubscriptionsVisualizer.java | 247 ++++++++++++++++++ .../TopicSubscriptionsVisualizerHtmlUtil.java | 164 ++++++++++++ .../admin/cli/utils/topic-visualizer.css | 180 +++++++++++++ 4 files changed, 627 insertions(+), 1 deletion(-) create mode 100644 pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/TopicSubscriptionsVisualizer.java create mode 100644 pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/TopicSubscriptionsVisualizerHtmlUtil.java create mode 100644 pulsar-client-tools/src/main/resources/org/apache/pulsar/admin/cli/utils/topic-visualizer.css 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 31d9648babcff..fde380f03be29 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 @@ -1,4 +1,4 @@ -/* +/** * 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 @@ -32,6 +32,10 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufUtil; import io.netty.buffer.Unpooled; +import java.io.File; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -48,8 +52,10 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import lombok.Getter; +import lombok.SneakyThrows; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.admin.ListTopicsOptions; +import org.apache.pulsar.admin.cli.utils.TopicSubscriptionsVisualizer; import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.admin.OffloadProcessStatus; import org.apache.pulsar.client.admin.PulsarAdmin; @@ -77,6 +83,7 @@ import org.apache.pulsar.common.policies.data.PublishRate; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.SubscribeRate; +import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.util.DateFormatter; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.common.util.RelativeTimeUtil; @@ -109,6 +116,7 @@ public CmdTopics(Supplier admin) { jcommander.addCommand("stats", new GetStats()); jcommander.addCommand("stats-internal", new GetInternalStats()); jcommander.addCommand("info-internal", new GetInternalInfo()); + jcommander.addCommand("subscriptions-visual-stats", new GetSubscriptionsVisualStats()); jcommander.addCommand("partitioned-stats", new GetPartitionedStats()); jcommander.addCommand("partitioned-stats-internal", new GetPartitionedStatsInternal()); @@ -844,6 +852,33 @@ void run() throws PulsarAdminException { } } + @Parameters(commandDescription = "Get the internal metadata info for the topic") + private class GetSubscriptionsVisualStats extends CliCommand { + @Parameter(description = "persistent://tenant/namespace/topic", required = true) + private java.util.List params; + + @Parameter(names = { "-f", "--filename" }, description = "Output filename") + private String outputFilename; + + @Override + @SneakyThrows + void run() throws PulsarAdminException { + String topic = validateTopicName(params); + final PersistentTopicInternalStats internalStats = getTopics().getInternalStats(topic); + final TopicStats stats = getTopics().getStats(topic); + + final String result = TopicSubscriptionsVisualizer.createHtml(topic, internalStats, stats); + if (outputFilename != null) { + final Path path = new File(outputFilename).toPath(); + Files.write(path, result.getBytes(StandardCharsets.UTF_8)); + print("Written to " + path.toFile().getAbsolutePath()); + } else { + print(result); + } + + } + } + @Parameters(commandDescription = "Get the stats for the partitioned topic " + "and its connected producers and consumers. All the rates are computed over a 1 minute window " + "and are relative the last completed 1 minute period.") diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/TopicSubscriptionsVisualizer.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/TopicSubscriptionsVisualizer.java new file mode 100644 index 0000000000000..1f7b474ef5711 --- /dev/null +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/TopicSubscriptionsVisualizer.java @@ -0,0 +1,247 @@ +/** + * 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.utils; + +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import lombok.AllArgsConstructor; +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; +import org.apache.pulsar.common.policies.data.TopicStats; +import org.apache.pulsar.common.util.ObjectMapperFactory; + +/** + * Generate a static html file showing subscriptions cursor positions for a given topic. + * It helps analyze the number of entries needed by a subscription to be up-to-date. + * The visualization helps to understand which subscriptions are slows compared to the other ones for the same topic. + */ +public class TopicSubscriptionsVisualizer { + + static class EntryWidthMapping { + final Map map = new HashMap<>(); + + Integer put(long entry, int width) { + return map.put(entry, width); + } + } + + static class LedgerSegmentProgress { + final Map ledgersMapping = new HashMap<>(); + int currentWidth = 0; + } + + static class EntriesStatsByStatus { + long totalAcked; + long totalAhead; + long totalIndividuallyDeleted; + } + + + enum EntryStatus { + ACKED, + INDIVIDUALLLY_DELETED, + AHEAD + } + + @AllArgsConstructor + private static class Position { + public long ledgerId; + public long entryId; + + static Position fromString(String str) { + final String[] split = str.split(":"); + return new Position(Long.parseLong(split[0]), Long.parseLong(split[1])); + } + } + + public static String createHtml(String topic, ManagedLedgerInternalStats persistentTopicInternalStats, + TopicStats persistentTopicStats) throws Exception { + return new TopicSubscriptionsVisualizer().internalCreateHtml(topic, persistentTopicInternalStats, + persistentTopicStats); + } + + private TopicSubscriptionsVisualizer() { + } + + private String internalCreateHtml(String topic, ManagedLedgerInternalStats persistentTopicInternalStats, + TopicStats persistentTopicStats) throws Exception { + StringBuilder builder = new StringBuilder(); + + startBody(builder); + final Position lastConfirmedEntry = Position.fromString(persistentTopicInternalStats.lastConfirmedEntry); + long totalEntries = 0; + for (ManagedLedgerInternalStats.LedgerInfo ledger : persistentTopicInternalStats.ledgers) { + if (ledger.entries == 0 && ledger.ledgerId == lastConfirmedEntry.ledgerId) { + ledger.entries = lastConfirmedEntry.entryId + 1; + } + totalEntries += ledger.entries; + } + generateTopicLine(topic, persistentTopicInternalStats, builder, totalEntries); + generateSubscriptionLines(persistentTopicInternalStats, persistentTopicStats, builder, totalEntries); + endBody(builder); + return builder.toString(); + } + + private static void endBody(StringBuilder builder) { + builder.append(""); + } + + private static void startBody(StringBuilder builder) throws IOException { + builder.append(""); + } + + private static void generateSubscriptionLines(ManagedLedgerInternalStats persistentTopicInternalStats, + TopicStats persistentTopicStats, + StringBuilder builder, + long totalEntries) { + List> subscriptionsLines = new ArrayList<>(); + persistentTopicInternalStats.cursors.forEach((name, cursor) -> { + StringBuilder subBuilder = new StringBuilder(); + final EntriesStatsByStatus entriesStatsByStatus = + generateSubscriptionLine(subBuilder, persistentTopicInternalStats, + persistentTopicStats, totalEntries, name, cursor); + subscriptionsLines.add(Pair.of(entriesStatsByStatus, subBuilder)); + }); + subscriptionsLines.sort(Comparator + .comparing((Pair pair) -> pair.getLeft().totalAhead) + .reversed()); + + subscriptionsLines.forEach(pair -> builder.append(pair.getRight())); + } + + private static void generateTopicLine(String topic, + ManagedLedgerInternalStats persistentTopicInternalStats, + StringBuilder builder, + long totalEntries) { + builder.append("

"); + builder.append(topic); + builder.append("

"); + final LedgerSegmentProgress ledgerSegmentProgress = new LedgerSegmentProgress(); + boolean first = true; + for (ManagedLedgerInternalStats.LedgerInfo ledger : persistentTopicInternalStats.ledgers) { + builder.append(TopicSubscriptionsVisualizerHtmlUtil.genLedgerSegment(ledger, + totalEntries, ledgerSegmentProgress, first)); + first = false; + } + builder.append("
"); + } + + private static EntriesStatsByStatus generateSubscriptionLine( + StringBuilder builder, + ManagedLedgerInternalStats persistentTopicInternalStats, + TopicStats persistentTopicStats, + long totalEntries, + String name, + ManagedLedgerInternalStats.CursorStats cursor) { + Map> entryStatuses = new HashMap<>(); + final Position markDeletePos = Position.fromString(cursor.markDeletePosition); + final Position readPos = Position.fromString(cursor.readPosition); + + final ObjectMapper mapper = ObjectMapperFactory.getThreadLocal(); + + for (ManagedLedgerInternalStats.LedgerInfo ledger : persistentTopicInternalStats.ledgers) { + Map entryStatusesForLedger = new HashMap<>(); + entryStatuses.put(ledger.ledgerId, entryStatusesForLedger); + + if (markDeletePos.ledgerId > ledger.ledgerId) { + for (long i = 0; i < ledger.entries; i++) { + entryStatusesForLedger.put(i, EntryStatus.ACKED); + } + } else if (markDeletePos.ledgerId == ledger.ledgerId) { + for (long i = 0; i < markDeletePos.entryId + 1; i++) { + entryStatusesForLedger.put(i, EntryStatus.ACKED); + } + } else { + for (long i = 0; i < ledger.entries; i++) { + entryStatusesForLedger.put(i, EntryStatus.AHEAD); + } + } + + if (readPos.ledgerId > ledger.ledgerId) { + for (long i = 0; i < ledger.entries; i++) { + if (!entryStatusesForLedger.containsKey(i)) { + entryStatusesForLedger.put(i, EntryStatus.AHEAD); + } + } + } else if (readPos.ledgerId == ledger.ledgerId) { + for (long i = 0; i < readPos.entryId - 1; i++) { + if (!entryStatusesForLedger.containsKey(i)) { + entryStatusesForLedger.put(i, EntryStatus.AHEAD); + } + } + } + + } + parseRanges(cursor.individuallyDeletedMessages).forEach(range -> { + for (long i = range.getLeft().entryId; i < range.getRight().entryId; i++) { + final long ledgerId = range.getLeft().ledgerId; + final Map entryStatusMap = entryStatuses + .computeIfAbsent(ledgerId, l -> new HashMap<>()); + entryStatusMap.put(i, EntryStatus.INDIVIDUALLLY_DELETED); + } + }); + EntriesStatsByStatus entriesStatsByStatus = new EntriesStatsByStatus(); + + final String line = TopicSubscriptionsVisualizerHtmlUtil.genSubscriptionLine(entryStatuses, + persistentTopicInternalStats.ledgers, totalEntries, entriesStatsByStatus); + + Map> details = new LinkedHashMap<>(); + details.put("Subscription", mapper.convertValue(persistentTopicStats.getSubscriptions().get(name), Map.class)); + details.put("Cursor", mapper.convertValue(cursor, Map.class)); + final String description = TopicSubscriptionsVisualizerHtmlUtil.genSubscriptionDescription(name, + entriesStatsByStatus.totalAhead, details); + builder.append("
"); + builder.append(description); + builder.append(line); + builder.append("
"); + return entriesStatsByStatus; + } + + private static List> parseRanges(String str) { + List> result = new ArrayList<>(); + str = str + .replace("[", "") + .replace("]", "") + .replace("(", "") + .replace(")", ""); + if (str.isEmpty()) { + return result; + } + final String[] splitByComma = str.split(","); + for (String range : splitByComma) { + final String[] splitRange = range.split("\\.\\."); + result.add(Pair.of(Position.fromString(splitRange[0]), + Position.fromString(splitRange[1]))); + } + return result; + } +} diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/TopicSubscriptionsVisualizerHtmlUtil.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/TopicSubscriptionsVisualizerHtmlUtil.java new file mode 100644 index 0000000000000..f87c83267f0ed --- /dev/null +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/TopicSubscriptionsVisualizerHtmlUtil.java @@ -0,0 +1,164 @@ +/** + * 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.utils; + +import java.util.List; +import java.util.Map; +import org.apache.commons.lang.StringEscapeUtils; +import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; +import org.apache.pulsar.common.util.ObjectMapperFactory; + +class TopicSubscriptionsVisualizerHtmlUtil { + + static String genSubscriptionLine(Map> entryStatuses, + List ledgers, + long totalEntries, + TopicSubscriptionsVisualizer.EntriesStatsByStatus entriesStatsByStatus) { + + StringBuilder builder = new StringBuilder("
"); + final float singleEntryPercent = (float) 100 / totalEntries; + + TopicSubscriptionsVisualizer.EntryStatus currentStatus = null; + long currentCount = 0; + for (ManagedLedgerInternalStats.LedgerInfo ledger : ledgers) { + for (long i = 0; i < ledger.entries; i++) { + TopicSubscriptionsVisualizer.EntryStatus entryStatus = entryStatuses.get(ledger.ledgerId).get(i); + if (entryStatus == null) { + entryStatus = TopicSubscriptionsVisualizer.EntryStatus.AHEAD; + } + + if (currentStatus == null) { + currentStatus = entryStatus; + } + + if (entryStatus == currentStatus) { + currentCount++; + continue; + } + if (currentStatus == TopicSubscriptionsVisualizer.EntryStatus.ACKED + || currentStatus == TopicSubscriptionsVisualizer.EntryStatus.INDIVIDUALLLY_DELETED) { + String cssClass; + if (currentStatus == TopicSubscriptionsVisualizer.EntryStatus.ACKED) { + entriesStatsByStatus.totalAcked += currentCount; + cssClass = "acked"; + } else { + entriesStatsByStatus.totalIndividuallyDeleted += currentCount; + cssClass = "idel"; + } + builder.append("
"); + } else { + entriesStatsByStatus.totalAhead += currentCount; + builder.append("
"); + } + currentCount = 1; + currentStatus = entryStatus; + } + } + if (currentCount > 0) { + + if (currentStatus == TopicSubscriptionsVisualizer.EntryStatus.ACKED + || currentStatus == TopicSubscriptionsVisualizer.EntryStatus.INDIVIDUALLLY_DELETED) { + String cssClass; + if (currentStatus == TopicSubscriptionsVisualizer.EntryStatus.ACKED) { + entriesStatsByStatus.totalAcked += currentCount; + cssClass = "acked"; + } else { + entriesStatsByStatus.totalIndividuallyDeleted += currentCount; + cssClass = "idel"; + } + builder.append("
"); + } else { + entriesStatsByStatus.totalAhead += currentCount; + builder.append("
"); + } + + } + builder.append("
"); + return builder.toString(); + } + + static String genLedgerSegment(ManagedLedgerInternalStats.LedgerInfo ledger, + long totalEntries, + TopicSubscriptionsVisualizer.LedgerSegmentProgress ledgerSegmentProgress, + boolean addFirstClass) { + String tooltip = genLedgerInfoDescription(ledger); + double percentWidth = (double) 100 / totalEntries * ledger.entries; + ledgerSegmentProgress.ledgersMapping.put(ledger.ledgerId, new TopicSubscriptionsVisualizer.EntryWidthMapping()); + for (long i = 0; i < ledger.entries; i++) { + ledgerSegmentProgress.ledgersMapping.get(ledger.ledgerId) + .put(i, ledgerSegmentProgress.currentWidth++); + } + + return "
" + + "" + + tooltip + + "
"; + } + + + static String genLedgerInfoDescription(ManagedLedgerInternalStats.LedgerInfo ledger) { + return genDetailsDescription(ObjectMapperFactory.getThreadLocal() + .convertValue(ledger, Map.class)); + + } + + static String genDetailsDescription(Map details) { + StringBuilder builder = new StringBuilder("
"); + details.forEach((k, v) -> { + builder.append("
"); + builder.append(escapeHtml(k)); + builder.append(""); + builder.append(v == null ? "" : escapeHtml(v.toString())); + builder.append("
"); + }); + builder.append("
"); + return builder.toString(); + } + + static String genSubscriptionDescription(String name, long distance, Map> details) { + StringBuilder builder = new StringBuilder("
"); + builder.append(escapeHtml(name)); + builder.append("
"); + details.forEach((k, data) -> { + builder.append("
"); + builder.append(escapeHtml(k)); + builder.append(""); + builder.append(genDetailsDescription(data)); + builder.append("
"); + }); + builder.append("
" + (distance == 0 ? "0" : "-" + distance) + ""); + builder.append("
"); + return builder.toString(); + } + + private static String escapeHtml(String str) { + return StringEscapeUtils.escapeHtml(str); + } +} diff --git a/pulsar-client-tools/src/main/resources/org/apache/pulsar/admin/cli/utils/topic-visualizer.css b/pulsar-client-tools/src/main/resources/org/apache/pulsar/admin/cli/utils/topic-visualizer.css new file mode 100644 index 0000000000000..8c7520b98522a --- /dev/null +++ b/pulsar-client-tools/src/main/resources/org/apache/pulsar/admin/cli/utils/topic-visualizer.css @@ -0,0 +1,180 @@ +/** + * 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. + */ +:root { + --c-black: #03071e; + + --c-topic: #277DA1; + --c-topic-hover: #577590; + + --c-ahead: #F3722C; + --c-ahead-hover: #F94144; + + --c-acked: #90BE6D; + --c-acked-hover: #43AA8B; + + --c-idel: #b9a44c; + --c-idel-hover: #566e3d; + + + --c-tooltip: #F8961E; +} +body { + color: var(--c-black); + font-family: verdana; +} +span { + font-size: 14px +} +.line-container { + margin: 6px 0; + box-shadow: 0 2px 4px 0 rgba(0,0,0,0.2); + border-radius: 5px; + padding: 12px 16px; + display: flex; + flex-direction: column; + +} +.line-container:hover { + box-shadow: 0 4px 8px 0 rgba(0,0,0,0.2); +} +.line-container .subscription { + height: 20px; + display: flex; + margin-top: 5px; +} +.line-container .subscription div { + border-top: 1px var(--c-black) solid; + border-bottom: 1px var(--c-black) solid; +} +.line-container .subscription .segment-acked { + background-color: var(--c-acked); +} +.line-container .subscription .segment-acked:hover { + background-color: var(--c-acked-hover); +} +.line-container .subscription .segment-ahead { + background-color: var(--c-ahead); +} +.line-container .subscription .segment-ahead:hover { + background-color: var(--c-ahead-hover); +} +.line-container .subscription .segment-idel { + background-color: var(--c-idel); +} +.line-container .subscription .segment-idel:hover { + background-color: var(--c-idel-hover); +} + +.line-container .topic { + display: flex; + +} +.line-container .topic .ledger { + height: 15px; + background-color: var(--c-topic); + border-top: 1px white solid; + border-bottom: 1px white solid; + border-right: 1px white solid; +} +.line-container .topic .ledger:hover { + background-color: var(--c-topic-hover); +} +.line-container .topic .ledger.first { + border-left: 1px white solid; +} +.line-container .description { + font-size: 18px; + display: flex; + flex-direction: row; + justify-content: space-between; + align-items: flex-end; +} +.line-container .description .title { + cursor: pointer; + font-size: 20px; + font-weight: 800; + padding: 4px 0; +} + +details > summary { + list-style: none; +} +details > summary::-webkit-details-marker { + display: none; +} + +.details-sections-container { + display: flex; +} +.details-sections-container .details-section { + display: flex; + flex-direction: column; +} +.details-sections-container .details-section .title { + font-size: 14px; + font-weight: 800; +} +.details-section .description-details .detail-item span { + font-size: 10px; +} +.description-details { + display: flex; + flex-direction: column; +} +.description-details .detail-item { + display: flex; + +} +.description-details .detail-item span { + font-size: 14px; + padding: 2px; +} +.description-details .detail-item .detail-item-key { + padding-right: 4px; + font-weight: 400; +} +.description-details .detail-item .detail-item-value { + font-weight: 900; +} + +.tooltip { + position: relative; + display: inline-block; +} +.tooltip .tooltiptext { + visibility: hidden; + min-width: 30px; + background-color: var(--c-tooltip); + border-radius: 1px; + border: 2px var(--c-black) solid; + margin: 6px; + position: absolute; + z-index: 1; + + display: flex; + +} +.tooltip .tooltiptext .tooltip-section { + padding: 8px 4px; + display: flex; + flex-direction: column; +} +.tooltip:hover .tooltiptext { + visibility: visible; +} \ No newline at end of file From 456b8096c2388d3c325359e283aad1b7172106cc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Nicol=C3=B2=20Boschi?= Date: Tue, 20 Sep 2022 10:09:51 +0200 Subject: [PATCH 085/699] Move to the new datastax internal artifactory (cherry picked from commit 65de5a06edf054d514c8e8d2fb5deded8bb1cb91) Conflicts: buildtools/pom.xml pom.xml pulsar-sql/presto-distribution/pom.xml --- buildtools/pom.xml | 29 +++++++++++++++++ pom.xml | 44 ++++++++++++++++++++++++++ pulsar-sql/presto-distribution/pom.xml | 12 +++++++ 3 files changed, 85 insertions(+) diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 5359f274e81b4..c2f993387bd9a 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -265,4 +265,33 @@ + + + jdk11 + + [11,) + + + + + --add-opens java.base/jdk.internal.loader=ALL-UNNAMED + --add-opens java.base/java.lang=ALL-UNNAMED + + + + + + + + + datastax-releases + DataStax Local Releases + https://repo.aws.dsinternal.org/artifactory/datastax-public-releases-local/ + + + datastax-snapshots-local + DataStax Local Snapshots + https://repo.aws.dsinternal.org/artifactory/datastax-snapshots-local/ + + diff --git a/pom.xml b/pom.xml index eb23574114a01..8c8571597b84f 100644 --- a/pom.xml +++ b/pom.xml @@ -2686,5 +2686,49 @@ flexible messaging model and an intuitive client API. false + + datastax-releases + https://repo.aws.dsinternal.org/artifactory/datastax-releases-local + + false + + + true + + + + datastax-snapshots-local + https://repo.aws.dsinternal.org/artifactory/datastax-snapshots-local + + true + + + false + + + + public-datastax-releases + https://repo.datastax.com/datastax-public-releases-local + + false + + + true + + + + + + datastax-releases + DataStax Local Releases + https://repo.aws.dsinternal.org/artifactory/datastax-public-releases-local/ + + + datastax-snapshots-local + DataStax Local Snapshots + https://repo.aws.dsinternal.org/artifactory/datastax-snapshots-local/ + + + diff --git a/pulsar-sql/presto-distribution/pom.xml b/pulsar-sql/presto-distribution/pom.xml index 353445fd436ae..694eaf366058c 100644 --- a/pulsar-sql/presto-distribution/pom.xml +++ b/pulsar-sql/presto-distribution/pom.xml @@ -381,4 +381,16 @@ + + + datastax-releases + DataStax Local Releases + https://repo.aws.dsinternal.org/artifactory/datastax-public-releases-local/ + + + datastax-snapshots-local + DataStax Local Snapshots + https://repo.aws.dsinternal.org/artifactory/datastax-snapshots-local/ + + From d613c10ecb3caf7e76fee26f2db770f1f1d9ddf9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Nicol=C3=B2=20Boschi?= Date: Wed, 21 Sep 2022 09:00:49 +0200 Subject: [PATCH 086/699] Fix NPE on subscriptions-visual-stats command (cherry picked from commit b835048543255437d0b28c22df8cc638d58d1226) --- .../admin/cli/utils/TopicSubscriptionsVisualizer.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/TopicSubscriptionsVisualizer.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/TopicSubscriptionsVisualizer.java index 1f7b474ef5711..b82918b3399aa 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/TopicSubscriptionsVisualizer.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/TopicSubscriptionsVisualizer.java @@ -32,6 +32,7 @@ import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; +import org.apache.pulsar.common.policies.data.SubscriptionStats; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.util.ObjectMapperFactory; @@ -215,7 +216,10 @@ private static EntriesStatsByStatus generateSubscriptionLine( persistentTopicInternalStats.ledgers, totalEntries, entriesStatsByStatus); Map> details = new LinkedHashMap<>(); - details.put("Subscription", mapper.convertValue(persistentTopicStats.getSubscriptions().get(name), Map.class)); + final SubscriptionStats subscriptionStats = persistentTopicStats.getSubscriptions().get(name); + if (subscriptionStats != null) { + details.put("Subscription", mapper.convertValue(subscriptionStats, Map.class)); + } details.put("Cursor", mapper.convertValue(cursor, Map.class)); final String description = TopicSubscriptionsVisualizerHtmlUtil.genSubscriptionDescription(name, entriesStatsByStatus.totalAhead, details); From 3204ef9ad477f8e893858058711547974b4d69c2 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Thu, 7 Dec 2023 20:48:53 +0530 Subject: [PATCH 087/699] License Header update --- .../src/main/java/org/apache/pulsar/broker/PulsarService.java | 2 +- .../src/main/java/org/apache/pulsar/admin/cli/CmdTopics.java | 2 +- .../pulsar/admin/cli/utils/TopicSubscriptionsVisualizer.java | 2 +- .../admin/cli/utils/TopicSubscriptionsVisualizerHtmlUtil.java | 2 +- .../org/apache/pulsar/admin/cli/utils/topic-visualizer.css | 2 +- 5 files changed, 5 insertions(+), 5 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 620eb68fb2143..4ffb5b77d5424 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 @@ -1,4 +1,4 @@ -/** +/* * 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 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 fde380f03be29..411e6e6d84257 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 @@ -1,4 +1,4 @@ -/** +/* * 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 diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/TopicSubscriptionsVisualizer.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/TopicSubscriptionsVisualizer.java index b82918b3399aa..8aab0418339d9 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/TopicSubscriptionsVisualizer.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/TopicSubscriptionsVisualizer.java @@ -1,4 +1,4 @@ -/** +/* * 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 diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/TopicSubscriptionsVisualizerHtmlUtil.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/TopicSubscriptionsVisualizerHtmlUtil.java index f87c83267f0ed..d1104e8ff7c92 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/TopicSubscriptionsVisualizerHtmlUtil.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/utils/TopicSubscriptionsVisualizerHtmlUtil.java @@ -1,4 +1,4 @@ -/** +/* * 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 diff --git a/pulsar-client-tools/src/main/resources/org/apache/pulsar/admin/cli/utils/topic-visualizer.css b/pulsar-client-tools/src/main/resources/org/apache/pulsar/admin/cli/utils/topic-visualizer.css index 8c7520b98522a..0822b2a7b0423 100644 --- a/pulsar-client-tools/src/main/resources/org/apache/pulsar/admin/cli/utils/topic-visualizer.css +++ b/pulsar-client-tools/src/main/resources/org/apache/pulsar/admin/cli/utils/topic-visualizer.css @@ -1,4 +1,4 @@ -/** +/* * 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 From 1b461b56ad16343180d449f6855164342907d4f3 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Thu, 7 Dec 2023 21:48:26 +0530 Subject: [PATCH 088/699] Fix license header for topic-visualizer css --- .../org/apache/pulsar/admin/cli/utils/topic-visualizer.css | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-client-tools/src/main/resources/org/apache/pulsar/admin/cli/utils/topic-visualizer.css b/pulsar-client-tools/src/main/resources/org/apache/pulsar/admin/cli/utils/topic-visualizer.css index 0822b2a7b0423..8c7520b98522a 100644 --- a/pulsar-client-tools/src/main/resources/org/apache/pulsar/admin/cli/utils/topic-visualizer.css +++ b/pulsar-client-tools/src/main/resources/org/apache/pulsar/admin/cli/utils/topic-visualizer.css @@ -1,4 +1,4 @@ -/* +/** * 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 From 075be68b9bfa6ad15938d4e6a44aedbe038ab31b Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Fri, 8 Dec 2023 10:28:28 +0530 Subject: [PATCH 089/699] Fix import order of TopicSubscriptionsVisualizer --- .../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 411e6e6d84257..47d5326b002cf 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 @@ -54,8 +54,8 @@ import lombok.Getter; import lombok.SneakyThrows; import org.apache.commons.lang3.StringUtils; -import org.apache.pulsar.client.admin.ListTopicsOptions; import org.apache.pulsar.admin.cli.utils.TopicSubscriptionsVisualizer; +import org.apache.pulsar.client.admin.ListTopicsOptions; import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.admin.OffloadProcessStatus; import org.apache.pulsar.client.admin.PulsarAdmin; From 608c9e36314b618fb643209cf3e51512854b5c22 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Fri, 10 Mar 2023 17:15:16 +0100 Subject: [PATCH 090/699] [improve][offloaders] Automatically evict Offloaded Ledgers from memory (#168) (cherry picked from commit 335ab8abd570437250195e9405856b274f8556f4) Conflicts: tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java --- conf/broker.conf | 4 + .../mledger/ManagedLedgerConfig.java | 9 ++ .../mledger/OffloadedLedgerHandle.java | 29 +++++ .../mledger/impl/ManagedLedgerImpl.java | 56 +++++++++- .../impl/OffloadEvictUnusedLedgersTest.java | 103 ++++++++++++++++++ .../mledger/impl/OffloadPrefixReadTest.java | 20 +++- .../pulsar/broker/ServiceConfiguration.java | 8 ++ .../pulsar/broker/service/BrokerService.java | 4 + .../impl/BlobStoreBackedReadHandleImpl.java | 16 ++- 9 files changed, 242 insertions(+), 7 deletions(-) create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/OffloadedLedgerHandle.java create mode 100644 managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadEvictUnusedLedgersTest.java diff --git a/conf/broker.conf b/conf/broker.conf index 4ad8536fd8d68..86ad978affdd9 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1160,6 +1160,10 @@ managedLedgerMaxLedgerRolloverTimeMinutes=240 # Disable rollover with value 0 (Default value 0) managedLedgerInactiveLedgerRolloverTimeSeconds=0 +# Time to evict inactive offloaded ledger for inactive topic +# Disable eviction with value 0 +managedLedgerInactiveOffloadedLedgerEvictionTimeSeconds=600 + # Maximum ledger size before triggering a rollover for a topic (MB) managedLedgerMaxSizePerLedgerMbytes=2048 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 0c93a5b642cf6..f16866ac7923a 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 @@ -79,6 +79,7 @@ public class ManagedLedgerConfig { private ManagedLedgerInterceptor managedLedgerInterceptor; private Map properties; private int inactiveLedgerRollOverTimeMs = 0; + private int inactiveOffloadedLedgerEvictionTimeMs = 0; @Getter @Setter private boolean cacheEvictionByMarkDeletedPosition = false; @@ -691,6 +692,14 @@ public void setInactiveLedgerRollOverTime(int inactiveLedgerRollOverTimeMs, Time this.inactiveLedgerRollOverTimeMs = (int) unit.toMillis(inactiveLedgerRollOverTimeMs); } + public int getInactiveOffloadedLedgerEvictionTimeMs() { + return inactiveOffloadedLedgerEvictionTimeMs; + } + + public void setInactiveOffloadedLedgerEvictionTimeMs(int inactiveOffloadedLedgerEvictionTimeMs, TimeUnit unit) { + this.inactiveOffloadedLedgerEvictionTimeMs = (int) unit.toMillis(inactiveOffloadedLedgerEvictionTimeMs); + } + /** * Minimum cursors with backlog after which broker is allowed to cache read entries to reuse them for other cursors' * backlog reads. (Default = 0, broker will not cache backlog reads) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/OffloadedLedgerHandle.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/OffloadedLedgerHandle.java new file mode 100644 index 0000000000000..75d8cd88d64cd --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/OffloadedLedgerHandle.java @@ -0,0 +1,29 @@ +/** + * 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; + +/** + * This is a marked interface for ledger handle that represent offloaded data. + */ +public interface OffloadedLedgerHandle { + + default long lastAccessTimestamp() { + return -1; + } +} 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 fb9da6db60e7f..df653a1196b63 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 @@ -112,6 +112,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.NonRecoverableLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerException.TooManyRequestsException; import org.apache.bookkeeper.mledger.ManagedLedgerMXBean; +import org.apache.bookkeeper.mledger.OffloadedLedgerHandle; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.WaitingEntryCallBack; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.VoidCallback; @@ -1954,6 +1955,8 @@ CompletableFuture getLedgerHandle(long ledgerId) { // TODO: improve this to load ledger offloader by driver name recorded in metadata Map offloadDriverMetadata = OffloadUtils.getOffloadDriverMetadata(info); offloadDriverMetadata.put("ManagedLedgerName", name); + log.info("[{}] Opening ledger {} from offload driver {} with uid {}", name, ledgerId, + config.getLedgerOffloader().getOffloadDriverName(), uid); openFuture = config.getLedgerOffloader().readOffloaded(ledgerId, uid, offloadDriverMetadata); } else { @@ -1979,11 +1982,16 @@ CompletableFuture getLedgerHandle(long ledgerId) { void invalidateReadHandle(long ledgerId) { CompletableFuture rhf = ledgerCache.remove(ledgerId); if (rhf != null) { - rhf.thenAccept(ReadHandle::closeAsync) - .exceptionally(ex -> { - log.warn("[{}] Failed to close a Ledger ReadHandle:", name, ex); - return null; - }); + rhf.thenCompose(r -> { + if (r instanceof OffloadedLedgerHandle) { + log.info("[{}] Closing ledger {} from offload driver {}", name, ledgerId, + config.getLedgerOffloader().getOffloadDriverName()); + } + return r.closeAsync(); + }).exceptionally(ex -> { + log.warn("[{}] Failed to close Ledger ReadHandle {}:", name, ledgerId, ex); + return null; + }); } } @@ -2590,7 +2598,45 @@ void internalTrimConsumedLedgers(CompletableFuture promise) { internalTrimLedgers(false, promise); } + @VisibleForTesting + List internalEvictOffloadedLedgers() { + int inactiveOffloadedLedgerEvictionTimeMs = config.getInactiveOffloadedLedgerEvictionTimeMs(); + if (inactiveOffloadedLedgerEvictionTimeMs <= 0) { + return Collections.emptyList(); + } + List ledgersToRelease = new ArrayList<>(); + + long now = clock.millis(); + + ledgerCache.forEach((ledgerId, ledger) -> { + if (ledger.isDone() && !ledger.isCompletedExceptionally()) { + ReadHandle readHandle = ledger.join(); + if (readHandle instanceof OffloadedLedgerHandle) { + long lastAccessTimestamp = ((OffloadedLedgerHandle) readHandle).lastAccessTimestamp(); + if (lastAccessTimestamp >= 0) { + long delta = now - lastAccessTimestamp; + if (delta >= inactiveOffloadedLedgerEvictionTimeMs) { + log.info("[{}] Offloaded ledger {} can be released ({} ms elapsed since last access)", + name, ledgerId, delta); + ledgersToRelease.add(ledgerId); + } else if (log.isDebugEnabled()) { + log.debug("[{}] Offloaded ledger {} cannot be released ({} ms elapsed since last access)", + name, ledgerId, delta); + } + } + } + } + }); + for (Long ledgerId : ledgersToRelease) { + invalidateReadHandle(ledgerId); + } + return ledgersToRelease; + } + void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { + + internalEvictOffloadedLedgers(); + if (!factory.isMetadataServiceAvailable()) { // Defer trimming of ledger if we cannot connect to metadata service promise.completeExceptionally(new MetaStoreException("Metadata service is not available")); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadEvictUnusedLedgersTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadEvictUnusedLedgersTest.java new file mode 100644 index 0000000000000..cc40b1b01ed31 --- /dev/null +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadEvictUnusedLedgersTest.java @@ -0,0 +1,103 @@ +/** + * 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.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotEquals; +import static org.testng.Assert.assertTrue; + +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.test.MockedBookKeeperTestCase; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.Test; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; + +public class OffloadEvictUnusedLedgersTest extends MockedBookKeeperTestCase { + private static final Logger log = LoggerFactory.getLogger(OffloadEvictUnusedLedgersTest.class); + + @Test + public void testEvictUnusedLedgers() throws Exception { + OffloadPrefixReadTest.MockLedgerOffloader offloader = + new OffloadPrefixReadTest.MockLedgerOffloader(); + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setMaxEntriesPerLedger(10); + config.setMinimumRolloverTime(0, TimeUnit.SECONDS); + config.setRetentionTime(10, TimeUnit.MINUTES); + config.setRetentionSizeInMB(10); + int inactiveOffloadedLedgerEvictionTimeMs = 10000; + config.setInactiveOffloadedLedgerEvictionTimeMs(inactiveOffloadedLedgerEvictionTimeMs, TimeUnit.MILLISECONDS); + config.setLedgerOffloader(offloader); + ManagedLedgerImpl ledger = (ManagedLedgerImpl)factory.open("my_test_ledger_evict", config); + + // no evict when no offloaded ledgers + assertTrue(ledger.internalEvictOffloadedLedgers().isEmpty()); + + int i = 0; + for (; i < 25; i++) { + String content = "entry-" + i; + ledger.addEntry(content.getBytes()); + } + assertEquals(ledger.getLedgersInfoAsList().size(), 3); + + ledger.offloadPrefix(ledger.getLastConfirmedEntry()); + + assertEquals(ledger.getLedgersInfoAsList().size(), 3); + assertEquals(ledger.getLedgersInfoAsList().stream() + .filter(e -> e.getOffloadContext().getComplete()) + .map(e -> e.getLedgerId()).collect(Collectors.toSet()), + offloader.offloadedLedgers()); + + // ledgers should be marked as offloaded + ledger.getLedgersInfoAsList().stream().allMatch(l -> l.hasOffloadContext()); + + // no evict when no offloaded ledgers are marked as inactive + assertTrue(ledger.internalEvictOffloadedLedgers().isEmpty()); + + ManagedCursor cursor = ledger.newNonDurableCursor(PositionImpl.EARLIEST); + int j = 0; + for (Entry e : cursor.readEntries(25)) { + assertEquals(new String(e.getData()), "entry-" + j++); + } + cursor.close(); + + // set last access time to be 2x inactiveOffloadedLedgerEvictionTimeMs + AtomicLong first = new AtomicLong(-1); + assertTrue(!ledger.ledgerCache.isEmpty()); + ledger.ledgerCache.forEach((id, l) -> { + if (first.compareAndSet(-1, id)) { + OffloadPrefixReadTest.MockOffloadReadHandle handle = + (OffloadPrefixReadTest.MockOffloadReadHandle) l.join(); + handle.setLastAccessTimestamp(System.currentTimeMillis() - inactiveOffloadedLedgerEvictionTimeMs * 2); + } + }); + assertNotEquals(first.get(), -1L); + + List evicted = ledger.internalEvictOffloadedLedgers(); + assertEquals(evicted.size(), 1); + assertEquals(first.get(), evicted.get(0).longValue()); + + } + +} diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java index cd224e33e2734..00c50249b4ac2 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java @@ -36,11 +36,14 @@ import java.util.List; import java.util.Map; import java.util.NavigableMap; +import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + import lombok.SneakyThrows; import org.apache.bookkeeper.client.api.DigestType; import org.apache.bookkeeper.client.api.LastConfirmedAndEntry; @@ -54,6 +57,7 @@ import org.apache.bookkeeper.mledger.LedgerOffloader; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.mledger.OffloadedLedgerHandle; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.mledger.util.MockClock; import org.apache.bookkeeper.net.BookieId; @@ -212,6 +216,10 @@ static class MockLedgerOffloader implements LedgerOffloader { OffloadPoliciesImpl.DEFAULT_OFFLOAD_DELETION_LAG_IN_MILLIS, OffloadPoliciesImpl.DEFAULT_OFFLOADED_READ_PRIORITY); + Set offloadedLedgers() { + return offloads.values().stream().map(ReadHandle::getId).collect(Collectors.toSet()); + } + @Override public String getOffloadDriverName() { @@ -272,10 +280,11 @@ public CompletableFuture closeAsync() { } } - static class MockOffloadReadHandle implements ReadHandle { + static class MockOffloadReadHandle implements ReadHandle, OffloadedLedgerHandle { final long id; final List entries = new ArrayList(); final LedgerMetadata metadata; + long lastAccessTimestamp = System.currentTimeMillis(); MockOffloadReadHandle(ReadHandle toCopy) throws Exception { id = toCopy.getId(); @@ -353,6 +362,15 @@ private CompletableFuture unsupported() { future.completeExceptionally(new UnsupportedOperationException()); return future; } + + @Override + public long lastAccessTimestamp() { + return lastAccessTimestamp; + } + + public void setLastAccessTimestamp(long lastAccessTimestamp) { + this.lastAccessTimestamp = lastAccessTimestamp; + } } static class MockMetadata implements LedgerMetadata { 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 fb26775591345..b165e19b40211 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 @@ -3018,6 +3018,14 @@ The delayed message index time step(in seconds) in per bucket snapshot segment, ) private int managedLedgerInactiveLedgerRolloverTimeSeconds = 0; + @FieldContext( + dynamic = true, + category = CATEGORY_STORAGE_ML, + doc = "Time to evict inactive offloaded ledger for inactive topic. " + + "Disable eviction with value 0 (Default value 600)" + ) + private int managedLedgerInactiveOffloadedLedgerEvictionTimeSeconds = 600; + @FieldContext( category = CATEGORY_STORAGE_ML, doc = "Evicting cache data by the slowest markDeletedPosition or readPosition. " 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 2cf141ed329ab..1702a23ef3db6 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 @@ -1880,6 +1880,10 @@ public CompletableFuture getManagedLedgerConfig(TopicName t managedLedgerConfig.setLazyCursorRecovery(serviceConfig.isLazyCursorRecovery()); managedLedgerConfig.setInactiveLedgerRollOverTime( serviceConfig.getManagedLedgerInactiveLedgerRolloverTimeSeconds(), TimeUnit.SECONDS); + managedLedgerConfig.setInactiveOffloadedLedgerEvictionTimeMs( + serviceConfig.getManagedLedgerInactiveOffloadedLedgerEvictionTimeSeconds(), + TimeUnit.SECONDS); + managedLedgerConfig.setCacheEvictionByMarkDeletedPosition( serviceConfig.isCacheEvictionByMarkDeletedPosition()); managedLedgerConfig.setMinimumBacklogCursorsForCaching( diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java index 5a571bb208e34..46bfc8d5bdc1f 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java @@ -40,6 +40,7 @@ import org.apache.bookkeeper.client.impl.LedgerEntryImpl; import org.apache.bookkeeper.mledger.LedgerOffloaderStats; import org.apache.bookkeeper.mledger.ManagedLedgerException; +import org.apache.bookkeeper.mledger.OffloadedLedgerHandle; import org.apache.bookkeeper.mledger.offload.jcloud.BackedInputStream; import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlock; import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlockBuilder; @@ -51,7 +52,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class BlobStoreBackedReadHandleImpl implements ReadHandle { +public class BlobStoreBackedReadHandleImpl implements ReadHandle, OffloadedLedgerHandle { private static final Logger log = LoggerFactory.getLogger(BlobStoreBackedReadHandleImpl.class); private static final int CACHE_TTL_SECONDS = Integer.getInteger("pulsar.jclouds.readhandleimpl.offsetsscache.ttl.seconds", 30 * 60); @@ -74,6 +75,8 @@ enum State { private State state = null; + private long lastAccessTimestamp = System.currentTimeMillis(); + private BlobStoreBackedReadHandleImpl(long ledgerId, OffloadIndexBlock index, BackedInputStream inputStream, ExecutorService executor) { this.ledgerId = ledgerId; @@ -119,6 +122,7 @@ public CompletableFuture readAsync(long firstEntry, long lastEntr } CompletableFuture promise = new CompletableFuture<>(); executor.execute(() -> { + touch(); if (state == State.Closed) { log.warn("Reading a closed read handler. Ledger ID: {}, Read range: {}-{}", ledgerId, firstEntry, lastEntry); @@ -203,6 +207,7 @@ public CompletableFuture readAsync(long firstEntry, long lastEntr } private void seekToEntry(long nextExpectedId) throws IOException { + touch(); Long knownOffset = entryOffsets.getIfPresent(nextExpectedId); if (knownOffset != null) { inputStream.seek(knownOffset); @@ -301,4 +306,13 @@ public static ReadHandle open(ScheduledExecutorService executor, State getState() { return this.state; } + + @Override + public long lastAccessTimestamp() { + return lastAccessTimestamp; + } + + private void touch() { + lastAccessTimestamp = System.currentTimeMillis(); + } } From 0a1055e85a230a860698604089f4e6bfe81aaa4a Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Fri, 8 Dec 2023 17:23:57 +0530 Subject: [PATCH 091/699] Fix license headers and import order for OffloadedLedgerHandle.java and OffloadEvictUnusedLedgersTest.java --- .../bookkeeper/mledger/OffloadedLedgerHandle.java | 2 +- .../mledger/impl/OffloadEvictUnusedLedgersTest.java | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/OffloadedLedgerHandle.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/OffloadedLedgerHandle.java index 75d8cd88d64cd..f45d115090f73 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/OffloadedLedgerHandle.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/OffloadedLedgerHandle.java @@ -1,4 +1,4 @@ -/** +/* * 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 diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadEvictUnusedLedgersTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadEvictUnusedLedgersTest.java index cc40b1b01ed31..17742f5c374cc 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadEvictUnusedLedgersTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadEvictUnusedLedgersTest.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -22,6 +22,10 @@ import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertTrue; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; @@ -29,10 +33,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.annotations.Test; -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; public class OffloadEvictUnusedLedgersTest extends MockedBookKeeperTestCase { private static final Logger log = LoggerFactory.getLogger(OffloadEvictUnusedLedgersTest.class); From fbcdf517d320a11e2eb1466012b05571208dce4e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Nicol=C3=B2=20Boschi?= Date: Thu, 4 May 2023 15:25:50 +0200 Subject: [PATCH 092/699] [improve][io] Add ElasticSearch Sink bulk debug logs (#179) (cherry picked from commit e2e2c4ac097605ec2749f9b3df35109caca53630) --- .../elasticsearch/client/elastic/ElasticBulkProcessor.java | 7 +++++++ .../client/opensearch/OpenSearchHighLevelRestClient.java | 7 +++++++ 2 files changed, 14 insertions(+) diff --git a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/elastic/ElasticBulkProcessor.java b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/elastic/ElasticBulkProcessor.java index 270da84d94224..00a7703e14bf2 100644 --- a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/elastic/ElasticBulkProcessor.java +++ b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/elastic/ElasticBulkProcessor.java @@ -100,6 +100,10 @@ public void appendIndexRequest(BulkIndexRequest request) throws IOException { if (config.getBulkSizeInMb() > 0) { sourceLength = request.getDocumentSource().getBytes(StandardCharsets.UTF_8).length; } + if (log.isDebugEnabled()) { + log.debug("append index request id={}, type={}, source={}, length={}", + request.getDocumentId(), config.getTypeName(), mapped, sourceLength); + } add(BulkOperationWithPulsarRecord.indexOperation(indexOperation, request.getRecord(), sourceLength)); } @@ -109,6 +113,9 @@ public void appendDeleteRequest(BulkDeleteRequest request) { .index(request.getIndex()) .id(request.getDocumentId()) .build(); + if (log.isDebugEnabled()) { + log.debug("append delete request id={}, type={}", request.getDocumentId(), config.getTypeName()); + } add(BulkOperationWithPulsarRecord.deleteOperation(deleteOperation, request.getRecord())); } diff --git a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/opensearch/OpenSearchHighLevelRestClient.java b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/opensearch/OpenSearchHighLevelRestClient.java index 7b7041967026e..07aae7994f842 100644 --- a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/opensearch/OpenSearchHighLevelRestClient.java +++ b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/client/opensearch/OpenSearchHighLevelRestClient.java @@ -302,6 +302,10 @@ public void appendIndexRequest(BulkProcessor.BulkIndexRequest request) throws IO } indexRequest.type(config.getTypeName()); indexRequest.source(request.getDocumentSource(), XContentType.JSON); + if (log.isDebugEnabled()) { + log.debug("append index request id={}, type={}, source={}", request.getDocumentId(), config.getTypeName(), + request.getDocumentSource()); + } internalBulkProcessor.add(indexRequest); } @@ -310,6 +314,9 @@ public void appendDeleteRequest(BulkProcessor.BulkDeleteRequest request) throws DeleteRequest deleteRequest = new DeleteRequestWithPulsarRecord(request.getIndex(), request.getRecord()); deleteRequest.id(request.getDocumentId()); deleteRequest.type(config.getTypeName()); + if (log.isDebugEnabled()) { + log.debug("append delete request id={}, type={}", request.getDocumentId(), config.getTypeName()); + } internalBulkProcessor.add(deleteRequest); } From bd57e59f05669d2b2d4f3a2c472ceb99fe780362 Mon Sep 17 00:00:00 2001 From: ming luo Date: Mon, 5 Jun 2023 14:53:49 -0400 Subject: [PATCH 093/699] [improve][sink]add metrics to elastic search sink (cherry picked from commit c370cfece9944a1c6d6bc87d6bc47e2bbe0c262c) Conflicts: pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchSink.java --- .../io/elasticsearch/ElasticSearchClient.java | 24 ++++++++- .../elasticsearch/ElasticSearchMetrics.java | 52 +++++++++++++++++++ .../io/elasticsearch/ElasticSearchSink.java | 17 +++++- .../elasticsearch/ElasticSearchAuthTests.java | 25 +++++---- .../ElasticSearchClientSslTests.java | 6 ++- .../ElasticSearchClientTests.java | 52 ++++++++++++++----- .../opensearch/OpenSearchClientSslTests.java | 10 ++-- 7 files changed, 155 insertions(+), 31 deletions(-) create mode 100644 pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchMetrics.java diff --git a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClient.java b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClient.java index d6fb5bb705d86..16fd0571e1fcd 100644 --- a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClient.java +++ b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClient.java @@ -33,6 +33,7 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.api.schema.GenericObject; import org.apache.pulsar.functions.api.Record; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.elasticsearch.client.BulkProcessor; import org.apache.pulsar.io.elasticsearch.client.RestClient; import org.apache.pulsar.io.elasticsearch.client.RestClientFactory; @@ -47,7 +48,9 @@ public class ElasticSearchClient implements AutoCloseable { }; private ElasticSearchConfig config; + private ElasticSearchMetrics metrics; private RestClient client; + private SinkContext sinkContext; private final RandomExponentialRetry backoffRetry; final Set indexCache = new HashSet<>(); @@ -56,8 +59,9 @@ public class ElasticSearchClient implements AutoCloseable { final AtomicReference irrecoverableError = new AtomicReference<>(); private final IndexNameFormatter indexNameFormatter; - public ElasticSearchClient(ElasticSearchConfig elasticSearchConfig) { + public ElasticSearchClient(ElasticSearchConfig elasticSearchConfig, ElasticSearchMetrics metrics) { this.config = elasticSearchConfig; + this.metrics = metrics; if (this.config.getIndexName() != null) { this.indexNameFormatter = new IndexNameFormatter(this.config.getIndexName()); } else { @@ -79,6 +83,7 @@ public void afterBulk(long executionId, List checkForIrrecoverableError(record, result); } else { record.ack(); + metrics.incrementCounter(ElasticSearchMetrics.SUCCESS, 1); } } } @@ -89,6 +94,7 @@ public void afterBulk(long executionId, List for (BulkProcessor.BulkOperationRequest operation: bulkOperationList) { final Record record = operation.getPulsarRecord(); record.fail(); + metrics.incrementCounter(ElasticSearchMetrics.FAILURE, 1); } } }; @@ -115,10 +121,13 @@ void checkForIrrecoverableError(Record record, BulkProcessor.BulkOperationRes for (String error : MALFORMED_ERRORS) { if (errorCause.contains(error)) { isMalformed = true; + metrics.incrementCounter(ElasticSearchMetrics.FAILURE, 1); switch (config.getMalformedDocAction()) { case IGNORE: + metrics.incrementCounter(ElasticSearchMetrics.MALFORMED_IGNORE, 1); break; case WARN: + metrics.incrementCounter(ElasticSearchMetrics.WARN, 1); log.warn("Ignoring malformed document index={} id={}", result.getIndex(), result.getDocumentId(), @@ -137,7 +146,10 @@ void checkForIrrecoverableError(Record record, BulkProcessor.BulkOperationRes if (!isMalformed) { log.warn("Bulk request failed, message id=[{}] index={} error={}", record.getMessage() - .map(m -> m.getMessageId().toString()) + .map(m -> { + metrics.incrementCounter(ElasticSearchMetrics.FAILURE, 1); + return m.getMessageId().toString(); + }) .orElse(""), result.getIndex(), result.getError()); } @@ -160,6 +172,7 @@ public void bulkIndex(Record record, Pair idAndDoc) throws Excep client.getBulkProcessor().appendIndexRequest(bulkIndexRequest); } catch (Exception e) { log.debug("index failed id=" + idAndDoc.getLeft(), e); + metrics.incrementCounter(ElasticSearchMetrics.FAILURE, 1); record.fail(); throw e; } @@ -184,13 +197,16 @@ public boolean indexDocument(Record record, Pair final boolean createdOrUpdated = client.indexDocument(indexName, documentId, documentSource); if (createdOrUpdated) { record.ack(); + metrics.incrementCounter(ElasticSearchMetrics.SUCCESS, 1); } else { record.fail(); + metrics.incrementCounter(ElasticSearchMetrics.FAILURE, 1); } return createdOrUpdated; } catch (final Exception ex) { log.error("index failed id=" + idAndDoc.getLeft(), ex); record.fail(); + metrics.incrementCounter(ElasticSearchMetrics.FAILURE, 1); throw ex; } } @@ -211,6 +227,7 @@ public void bulkDelete(Record record, String id) throws Exception } catch (Exception e) { log.debug("delete failed id: {}", id, e); record.fail(); + metrics.incrementCounter(ElasticSearchMetrics.FAILURE, 1); throw e; } } @@ -230,13 +247,16 @@ public boolean deleteDocument(Record record, String id) throws Ex final boolean deleted = client.deleteDocument(indexName, id); if (deleted) { record.ack(); + metrics.incrementCounter(ElasticSearchMetrics.SUCCESS, 1); } else { record.fail(); + metrics.incrementCounter(ElasticSearchMetrics.FAILURE, 1); } return deleted; } catch (final Exception ex) { log.debug("index failed id: {}", id, ex); record.fail(); + metrics.incrementCounter(ElasticSearchMetrics.FAILURE, 1); throw ex; } } diff --git a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchMetrics.java b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchMetrics.java new file mode 100644 index 0000000000000..03b52324049c5 --- /dev/null +++ b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchMetrics.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.io.elasticsearch; + +import org.apache.pulsar.io.core.SinkContext; + +/* + * Metrics class for ElasticSearchSink + */ +public class ElasticSearchMetrics { + + private SinkContext sinkContext; + // sink metrics + public static final String INCOMING = "_elasticsearch_incoming"; + + // INCOMING = SUCCESS + FAILURE + SKIP + NULLVALUE_IGNORE + public static final String SUCCESS = "_elasticsearch_success"; + + // DELETE_ATTEMPT is an attempt to delete a document by id + // TODO: add delete success metrics, currently it's difficult to separate delete and index from the bulk operations + public static final String DELETE_ATTEMPT = "elasticsearch_delete_attempt"; + + public static final String FAILURE = "elasticsearch_failure"; + public static final String SKIP = "elasticsearch_skip"; + public static final String WARN = "elasticsearch_warn"; + public static final String MALFORMED_IGNORE = "elasticsearch_malformed_ignore"; + public static final String NULLVALUE_IGNORE = "elasticsearch_nullvalue_ignore"; + + public ElasticSearchMetrics(SinkContext sinkContext) { + this.sinkContext = sinkContext; + } + + public void incrementCounter(String counter, double value) { + this.sinkContext.recordMetric(counter, value); + } +} diff --git a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchSink.java b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchSink.java index f76f985f7215d..518de9e55cf30 100644 --- a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchSink.java +++ b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchSink.java @@ -66,6 +66,7 @@ public class ElasticSearchSink implements Sink { private ElasticSearchConfig elasticSearchConfig; private ElasticSearchClient elasticsearchClient; + private ElasticSearchMetrics metrics; private final ObjectMapper objectMapper = new ObjectMapper(); private ObjectMapper sortedObjectMapper; private List primaryFields = null; @@ -76,7 +77,8 @@ public class ElasticSearchSink implements Sink { public void open(Map config, SinkContext sinkContext) throws Exception { elasticSearchConfig = ElasticSearchConfig.load(config, sinkContext); elasticSearchConfig.validate(); - elasticsearchClient = new ElasticSearchClient(elasticSearchConfig); + metrics = new ElasticSearchMetrics(sinkContext); + elasticsearchClient = new ElasticSearchClient(elasticSearchConfig, metrics); if (!Strings.isNullOrEmpty(elasticSearchConfig.getPrimaryFields())) { primaryFields = Arrays.asList(elasticSearchConfig.getPrimaryFields().split(",")); } @@ -110,6 +112,7 @@ void setElasticsearchClient(ElasticSearchClient elasticsearchClient) { @Override public void write(Record record) throws Exception { + metrics.incrementCounter(ElasticSearchMetrics.INCOMING, 1); if (!elasticsearchClient.isFailed()) { Pair idAndDoc = extractIdAndDocument(record); try { @@ -120,16 +123,21 @@ public void write(Record record) throws Exception { switch (elasticSearchConfig.getNullValueAction()) { case DELETE: if (idAndDoc.getLeft() != null) { + metrics.incrementCounter(ElasticSearchMetrics.DELETE_ATTEMPT, 1); if (elasticSearchConfig.isBulkEnabled()) { elasticsearchClient.bulkDelete(record, idAndDoc.getLeft()); } else { elasticsearchClient.deleteDocument(record, idAndDoc.getLeft()); } + } else { + metrics.incrementCounter(ElasticSearchMetrics.SKIP, 1); } break; case IGNORE: + metrics.incrementCounter(ElasticSearchMetrics.NULLVALUE_IGNORE, 1); break; case FAIL: + metrics.incrementCounter(ElasticSearchMetrics.FAILURE, 1); elasticsearchClient.failed( new PulsarClientException.InvalidMessageException("Unexpected null message value")); throw elasticsearchClient.irrecoverableError.get(); @@ -142,10 +150,14 @@ public void write(Record record) throws Exception { } } } catch (JsonProcessingException jsonProcessingException) { + // this is from non-bulk action + // a generical failure counter should have been incremented switch (elasticSearchConfig.getMalformedDocAction()) { case IGNORE: + metrics.incrementCounter(ElasticSearchMetrics.MALFORMED_IGNORE, 1); break; case WARN: + metrics.incrementCounter(ElasticSearchMetrics.WARN, 1); log.warn("Ignoring malformed document messageId={}", record.getMessage().map(Message::getMessageId).orElse(null), jsonProcessingException); @@ -163,6 +175,7 @@ public void write(Record record) throws Exception { throw e; } } else { + metrics.incrementCounter(ElasticSearchMetrics.FAILURE, 1); throw new IllegalStateException("Elasticsearch client is in FAILED status"); } } @@ -282,7 +295,7 @@ public Pair extractIdAndDocument(Record record) t } doc = sanitizeValue(doc); return Pair.of(id, doc); - } else { + } else { Message message = record.getMessage().orElse(null); final String rawData; if (message != null) { diff --git a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchAuthTests.java b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchAuthTests.java index 7c56bfc23c96d..6aa7b82ecee4d 100644 --- a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchAuthTests.java +++ b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchAuthTests.java @@ -22,6 +22,8 @@ import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.functions.api.Record; +import org.apache.pulsar.io.core.SinkContext; +import org.mockito.Mockito; import org.testcontainers.elasticsearch.ElasticsearchContainer; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeMethod; @@ -78,8 +80,10 @@ public void testBasicAuth() throws Exception { config.setIndexName(indexName); config.setMaxRetries(1); config.setBulkEnabled(true); + SinkContext mockContext = Mockito.mock(SinkContext.class); + ElasticSearchMetrics metrics = new ElasticSearchMetrics(mockContext); // ensure auth is needed - try (ElasticSearchClient client = new ElasticSearchClient(config);) { + try (ElasticSearchClient client = new ElasticSearchClient(config, metrics);) { expectThrows(ElasticSearchConnectionException.class, () -> { client.createIndexIfNeeded(indexName); }); @@ -87,7 +91,7 @@ public void testBasicAuth() throws Exception { config.setPassword(ELASTICPWD); - try (ElasticSearchClient client = new ElasticSearchClient(config);) { + try (ElasticSearchClient client = new ElasticSearchClient(config, metrics);) { ensureCalls(client, indexName); } } @@ -103,10 +107,11 @@ public void testTokenAuth() throws Exception { config.setMaxRetries(1); config.setBulkEnabled(true); - + SinkContext mockContext = Mockito.mock(SinkContext.class); + ElasticSearchMetrics metrics = new ElasticSearchMetrics(mockContext); config.setPassword(ELASTICPWD); String token; - try (ElasticSearchClient client = new ElasticSearchClient(config);) { + try (ElasticSearchClient client = new ElasticSearchClient(config, metrics);) { token = createAuthToken(client, "elastic", ELASTICPWD); } @@ -114,14 +119,14 @@ public void testTokenAuth() throws Exception { config.setPassword(null); // ensure auth is needed - try (ElasticSearchClient client = new ElasticSearchClient(config);) { + try (ElasticSearchClient client = new ElasticSearchClient(config, metrics);) { expectThrows(ElasticSearchConnectionException.class, () -> { client.createIndexIfNeeded(indexName); }); } config.setToken(token); - try (ElasticSearchClient client = new ElasticSearchClient(config);) { + try (ElasticSearchClient client = new ElasticSearchClient(config, metrics);) { ensureCalls(client, indexName); } } @@ -137,9 +142,11 @@ public void testApiKey() throws Exception { config.setMaxRetries(1); config.setBulkEnabled(true); + SinkContext mockContext = Mockito.mock(SinkContext.class); + ElasticSearchMetrics metrics = new ElasticSearchMetrics(mockContext); config.setPassword(ELASTICPWD); String apiKey; - try (ElasticSearchClient client = new ElasticSearchClient(config);) { + try (ElasticSearchClient client = new ElasticSearchClient(config, metrics);) { apiKey = createApiKey(client); } @@ -147,14 +154,14 @@ public void testApiKey() throws Exception { config.setPassword(null); // ensure auth is needed - try (ElasticSearchClient client = new ElasticSearchClient(config);) { + try (ElasticSearchClient client = new ElasticSearchClient(config, metrics);) { expectThrows(ElasticSearchConnectionException.class, () -> { client.createIndexIfNeeded(indexName); }); } config.setApiKey(apiKey); - try (ElasticSearchClient client = new ElasticSearchClient(config);) { + try (ElasticSearchClient client = new ElasticSearchClient(config, metrics);) { ensureCalls(client, indexName); } } diff --git a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClientSslTests.java b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClientSslTests.java index 5598a88d410a9..86c8638a5a21f 100644 --- a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClientSslTests.java +++ b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClientSslTests.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.io.elasticsearch; +import org.apache.pulsar.io.core.SinkContext; +import org.mockito.Mockito; import org.testcontainers.containers.wait.strategy.Wait; import org.testcontainers.elasticsearch.ElasticsearchContainer; import org.testcontainers.utility.MountableFile; @@ -183,7 +185,9 @@ public void testSslDisableCertificateValidation() throws IOException { } private void testClientWithConfig(ElasticSearchConfig config) throws IOException { - try (ElasticSearchClient client = new ElasticSearchClient(config);) { + SinkContext mockContext = Mockito.mock(SinkContext.class); + ElasticSearchMetrics metrics = new ElasticSearchMetrics(mockContext); + try (ElasticSearchClient client = new ElasticSearchClient(config, metrics);) { testIndexExists(client); } } diff --git a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClientTests.java b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClientTests.java index 6d9928c042697..3876f3639dd0a 100644 --- a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClientTests.java +++ b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClientTests.java @@ -20,6 +20,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; @@ -37,6 +38,7 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.api.schema.GenericObject; import org.apache.pulsar.functions.api.Record; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.elasticsearch.client.elastic.ElasticSearchJavaRestClient; import org.apache.pulsar.io.elasticsearch.client.opensearch.OpenSearchHighLevelRestClient; import org.apache.pulsar.io.elasticsearch.testcontainers.ElasticToxiproxiContainer; @@ -107,9 +109,11 @@ public void fail() { @Test public void testClientInstance() throws Exception { + SinkContext mockContext = Mockito.mock(SinkContext.class); + ElasticSearchMetrics metrics = new ElasticSearchMetrics(mockContext); try (ElasticSearchClient client = new ElasticSearchClient(new ElasticSearchConfig() .setElasticSearchUrl("http://" + container.getHttpHostAddress()) - .setIndexName(INDEX))) { + .setIndexName(INDEX), metrics)) { if (elasticImageName.equals(OPENSEARCH) || elasticImageName.equals(ELASTICSEARCH_7)) { assertTrue(client.getRestClient() instanceof OpenSearchHighLevelRestClient); } else { @@ -123,21 +127,23 @@ public void testIndexName() throws Exception { String index = "myindex-" + UUID.randomUUID(); Record record = Mockito.mock(Record.class); String topicName = "topic-" + UUID.randomUUID(); + SinkContext mockContext = Mockito.mock(SinkContext.class); + ElasticSearchMetrics metrics = new ElasticSearchMetrics(mockContext); when(record.getTopicName()).thenReturn(Optional.of(topicName)); try (ElasticSearchClient client = new ElasticSearchClient(new ElasticSearchConfig() .setElasticSearchUrl("http://" + container.getHttpHostAddress()) - .setIndexName(index))) { + .setIndexName(index), metrics)) { assertEquals(client.indexName(record), index); } try (ElasticSearchClient client = new ElasticSearchClient(new ElasticSearchConfig() - .setElasticSearchUrl("http://" + container.getHttpHostAddress()))) { + .setElasticSearchUrl("http://" + container.getHttpHostAddress()), metrics)) { assertEquals(client.indexName(record), topicName); } String indexBase = "myindex-" + UUID.randomUUID(); index = indexBase + "-%{+yyyy-MM-dd}"; try (ElasticSearchClient client = new ElasticSearchClient(new ElasticSearchConfig() .setElasticSearchUrl("http://" + container.getHttpHostAddress()) - .setIndexName(index))) { + .setIndexName(index), metrics)) { assertThrows(IllegalStateException.class, () -> { client.indexName(record); }); @@ -145,7 +151,7 @@ public void testIndexName() throws Exception { when(record.getEventTime()).thenReturn(Optional.of(1645182000000L)); try (ElasticSearchClient client = new ElasticSearchClient(new ElasticSearchConfig() .setElasticSearchUrl("http://" + container.getHttpHostAddress()) - .setIndexName(index))) { + .setIndexName(index), metrics)) { assertEquals(client.indexName(record), indexBase + "-2022-02-18"); } } @@ -153,9 +159,11 @@ public void testIndexName() throws Exception { @Test public void testIndexDelete() throws Exception { String index = "myindex-" + UUID.randomUUID(); + SinkContext mockContext = Mockito.mock(SinkContext.class); + ElasticSearchMetrics metrics = new ElasticSearchMetrics(mockContext); try (ElasticSearchClient client = new ElasticSearchClient(new ElasticSearchConfig() .setElasticSearchUrl("http://" + container.getHttpHostAddress()) - .setIndexName(index));) { + .setIndexName(index), metrics);) { assertTrue(client.createIndexIfNeeded(index)); try { MockRecord mockRecord = new MockRecord<>(); @@ -177,9 +185,11 @@ public void testIndexDelete() throws Exception { @Test public void testIndexExists() throws IOException { String index = "mynewindex-" + UUID.randomUUID(); + SinkContext mockContext = Mockito.mock(SinkContext.class); + ElasticSearchMetrics metrics = new ElasticSearchMetrics(mockContext); try (ElasticSearchClient client = new ElasticSearchClient(new ElasticSearchConfig() .setElasticSearchUrl("http://" + container.getHttpHostAddress()) - .setIndexName(index));) { + .setIndexName(index), metrics);) { assertFalse(client.indexExists(index)); assertTrue(client.createIndexIfNeeded(index)); try { @@ -193,8 +203,10 @@ public void testIndexExists() throws IOException { @Test public void testTopicToIndexName() throws IOException { + SinkContext mockContext = Mockito.mock(SinkContext.class); + ElasticSearchMetrics metrics = new ElasticSearchMetrics(mockContext); try (ElasticSearchClient client = new ElasticSearchClient(new ElasticSearchConfig() - .setElasticSearchUrl("http://" + container.getHttpHostAddress()));) { + .setElasticSearchUrl("http://" + container.getHttpHostAddress()), metrics);) { assertEquals(client.topicToIndexName("data-ks1.table1"), "data-ks1.table1"); assertEquals(client.topicToIndexName("persistent://public/default/testesjson"), "testesjson"); assertEquals(client.topicToIndexName("default/testesjson"), "testesjson"); @@ -217,7 +229,9 @@ public void testMalformedDocFails() throws Exception { .setBulkEnabled(true) .setBulkFlushIntervalInMs(-1L) .setMalformedDocAction(ElasticSearchConfig.MalformedDocAction.FAIL); - try (ElasticSearchClient client = new ElasticSearchClient(config);) { + SinkContext mockContext = Mockito.mock(SinkContext.class); + ElasticSearchMetrics metrics = new ElasticSearchMetrics(mockContext); + try (ElasticSearchClient client = new ElasticSearchClient(config, metrics);) { MockRecord mockRecord = new MockRecord<>(); client.bulkIndex(mockRecord, Pair.of("1", "{\"a\":1}")); client.bulkIndex(mockRecord, Pair.of("2", "{\"a\":\"toto\"}")); @@ -241,7 +255,9 @@ public void testMalformedDocIgnore() throws Exception { .setBulkEnabled(true) .setBulkFlushIntervalInMs(-1) .setMalformedDocAction(ElasticSearchConfig.MalformedDocAction.IGNORE); - try (ElasticSearchClient client = new ElasticSearchClient(config);) { + SinkContext mockContext = Mockito.mock(SinkContext.class); + ElasticSearchMetrics metrics = new ElasticSearchMetrics(mockContext); + try (ElasticSearchClient client = new ElasticSearchClient(config, metrics);) { MockRecord mockRecord = new MockRecord<>(); client.bulkIndex(mockRecord, Pair.of("1", "{\"a\":1}")); client.bulkIndex(mockRecord, Pair.of("2", "{\"a\":\"toto\"}")); @@ -268,7 +284,9 @@ public void testBulkRetry() throws Exception { // disabled, we want to have full control over flush() method .setBulkFlushIntervalInMs(-1); - try (ElasticSearchClient client = new ElasticSearchClient(config);) { + SinkContext mockContext = Mockito.mock(SinkContext.class); + ElasticSearchMetrics metrics = new ElasticSearchMetrics(mockContext); + try (ElasticSearchClient client = new ElasticSearchClient(config, metrics);) { try { assertTrue(client.createIndexIfNeeded(index)); MockRecord mockRecord = new MockRecord<>(); @@ -314,7 +332,9 @@ public void testBulkBlocking() throws Exception { .setBulkConcurrentRequests(2) .setRetryBackoffInMs(100) .setBulkFlushIntervalInMs(10000); - try (ElasticSearchClient client = new ElasticSearchClient(config);) { + SinkContext mockContext = Mockito.mock(SinkContext.class); + ElasticSearchMetrics metrics = new ElasticSearchMetrics(mockContext); + try (ElasticSearchClient client = new ElasticSearchClient(config, metrics);) { assertTrue(client.createIndexIfNeeded(index)); try { @@ -373,7 +393,9 @@ public void testBulkIndexAndDelete() throws Exception { .setBulkActions(10) .setBulkFlushIntervalInMs(-1L); - try (ElasticSearchClient client = new ElasticSearchClient(config)) { + SinkContext mockContext = Mockito.mock(SinkContext.class); + ElasticSearchMetrics metrics = new ElasticSearchMetrics(mockContext); + try (ElasticSearchClient client = new ElasticSearchClient(config, metrics)) { assertTrue(client.createIndexIfNeeded(index)); MockRecord mockRecord = new MockRecord<>(); for (int i = 0; i < 5; i++) { @@ -396,8 +418,10 @@ public void testIndexKeepNulls() throws Exception { ElasticSearchConfig config = new ElasticSearchConfig() .setElasticSearchUrl("http://" + container.getHttpHostAddress()) .setIndexName(index); + SinkContext mockContext = Mockito.mock(SinkContext.class); + ElasticSearchMetrics metrics = new ElasticSearchMetrics(mockContext); - try (ElasticSearchClient client = new ElasticSearchClient(config)) { + try (ElasticSearchClient client = new ElasticSearchClient(config, metrics)) { MockRecord mockRecord = new MockRecord<>(); client.indexDocument(mockRecord, Pair.of("key0", "{\"a\":1,\"b\":null}")); final Map sourceAsMap; diff --git a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/opensearch/OpenSearchClientSslTests.java b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/opensearch/OpenSearchClientSslTests.java index de6946136855a..f93a873c465d9 100644 --- a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/opensearch/OpenSearchClientSslTests.java +++ b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/opensearch/OpenSearchClientSslTests.java @@ -20,6 +20,7 @@ import org.apache.pulsar.io.elasticsearch.ElasticSearchClient; import org.apache.pulsar.io.elasticsearch.ElasticSearchConfig; +import org.apache.pulsar.io.elasticsearch.ElasticSearchMetrics; import org.apache.pulsar.io.elasticsearch.ElasticSearchSslConfig; import org.apache.pulsar.io.elasticsearch.ElasticSearchTestBase; import org.testcontainers.containers.wait.strategy.Wait; @@ -81,7 +82,8 @@ public void testSslBasic() throws IOException { .setEnabled(true) .setTruststorePath(sslResourceDir + "/truststore.jks") .setTruststorePassword("changeit")); - ElasticSearchClient client = new ElasticSearchClient(config); + ElasticSearchMetrics metrics = new ElasticSearchMetrics(null); + ElasticSearchClient client = new ElasticSearchClient(config, metrics); testIndexExists(client); } } @@ -107,7 +109,8 @@ public void testSslWithHostnameVerification() throws IOException { .setHostnameVerification(true) .setTruststorePath(sslResourceDir + "/truststore.jks") .setTruststorePassword("changeit")); - ElasticSearchClient client = new ElasticSearchClient(config); + ElasticSearchMetrics metrics = new ElasticSearchMetrics(null); + ElasticSearchClient client = new ElasticSearchClient(config, metrics); testIndexExists(client); } } @@ -133,7 +136,8 @@ public void testSslWithClientAuth() throws IOException { .setTruststorePassword("changeit") .setKeystorePath(sslResourceDir + "/keystore.jks") .setKeystorePassword("changeit")); - ElasticSearchClient client = new ElasticSearchClient(config); + ElasticSearchMetrics metrics = new ElasticSearchMetrics(null); + ElasticSearchClient client = new ElasticSearchClient(config, metrics); testIndexExists(client); } } From 02570d8cefcef8847c4f21499c239dc77ee3ec17 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Nicol=C3=B2=20Boschi?= Date: Mon, 19 Jun 2023 15:47:32 +0200 Subject: [PATCH 094/699] [fix][monitor] topic/subscription with slash breaks the prometheus format (#187) (cherry picked from commit 7db320b10c6650f27dd709a861374851c2dc9ea5) Conflicts: pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/PrometheusMetricsTest.java --- .../PrometheusMetricsGeneratorUtils.java | 45 ++++++++++++++++--- .../PrometheusMetricsGeneratorUtilsTest.java | 22 +++++++++ .../prometheus/PrometheusMetricStreams.java | 5 +-- .../PrometheusMetricsGenerator.java | 7 ++- .../metrics/PrometheusTextFormatUtil.java | 1 + .../broker/stats/PrometheusMetricsTest.java | 11 +++-- .../elasticsearch/ElasticSearchMetrics.java | 2 +- .../ElasticSearchClientTests.java | 1 - 8 files changed, 78 insertions(+), 16 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorUtils.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorUtils.java index 828d9871bb3de..651abcd8dbbc2 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorUtils.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorUtils.java @@ -25,6 +25,7 @@ import java.io.OutputStream; import java.util.Enumeration; import java.util.List; +import java.util.regex.Pattern; import org.apache.commons.collections4.CollectionUtils; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.util.SimpleTextOutputStream; @@ -34,6 +35,7 @@ * Format specification can be found at {@link https://prometheus.io/docs/instrumenting/exposition_formats/} */ public class PrometheusMetricsGeneratorUtils { + private static final Pattern METRIC_LABEL_VALUE_SPECIAL_CHARACTERS = Pattern.compile("[\\\\\"\\n]"); public static void generate(String cluster, OutputStream out, List metricsProviders) @@ -68,17 +70,14 @@ public static void generateSystemMetrics(SimpleTextOutputStream stream, String c stream.write(sample.name); stream.write("{"); if (!sample.labelNames.contains("cluster")) { - stream.write("cluster=\"").write(cluster).write('"'); + stream.write("cluster=\"").write(writeEscapedLabelValue(cluster)).write('"'); // If label is empty, should not append ','. if (!CollectionUtils.isEmpty(sample.labelNames)){ stream.write(","); } } for (int j = 0; j < sample.labelNames.size(); j++) { - String labelValue = sample.labelValues.get(j); - if (labelValue != null) { - labelValue = labelValue.replace("\"", "\\\""); - } + String labelValue = writeEscapedLabelValue(sample.labelValues.get(j)); if (j > 0) { stream.write(","); } @@ -119,5 +118,41 @@ static String getTypeStr(Collector.Type type) { } } + + /** + * Write a label value to the writer, escaping backslashes, double quotes and newlines. + * See Promethues Exporter io.prometheus.client.exporter.common.TextFormat#writeEscapedLabelValue + */ + public static String writeEscapedLabelValue(String s) { + if (s == null) { + return null; + } + if (!labelValueNeedsEscape(s)) { + return s; + } + StringBuilder writer = new StringBuilder(); + for (int i = 0; i < s.length(); i++) { + char c = s.charAt(i); + switch (c) { + case '\\': + writer.append("\\\\"); + break; + case '\"': + writer.append("\\\""); + break; + case '\n': + writer.append("\\n"); + break; + default: + writer.append(c); + } + } + return writer.toString(); + } + + static boolean labelValueNeedsEscape(String s) { + return METRIC_LABEL_VALUE_SPECIAL_CHARACTERS.matcher(s).find(); + } + } diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorUtilsTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorUtilsTest.java index 0ac3e22197606..004542ef13f92 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorUtilsTest.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsGeneratorUtilsTest.java @@ -99,4 +99,26 @@ public void testGenerateSystemMetricsWithoutCustomizedLabel() throws Exception { private static String randomString(){ return UUID.randomUUID().toString().replaceAll("-", ""); } + + + @Test + public void testWriteEscapedLabelValue() throws Exception { + assertEquals(PrometheusMetricsGeneratorUtils.writeEscapedLabelValue(null), null); + assertEquals(PrometheusMetricsGeneratorUtils.writeEscapedLabelValue(""), ""); + assertEquals(PrometheusMetricsGeneratorUtils.writeEscapedLabelValue("ok"), "ok"); + assertEquals(PrometheusMetricsGeneratorUtils.writeEscapedLabelValue("ok_!234567890!£$%&/()"), + "ok_!234567890!£$%&/()"); + assertEquals(PrometheusMetricsGeneratorUtils.writeEscapedLabelValue("repl\"\\\n"), + "repl\\\"\\\\\\n"); + } + @Test + public void testWriteEscapedLabelValuePattern() throws Exception { + assertFalse(PrometheusMetricsGeneratorUtils.labelValueNeedsEscape("")); + assertFalse(PrometheusMetricsGeneratorUtils.labelValueNeedsEscape("ok")); + assertFalse(PrometheusMetricsGeneratorUtils.labelValueNeedsEscape("ok_!234567890!£$%&/()")); + assertTrue(PrometheusMetricsGeneratorUtils.labelValueNeedsEscape("repl\"\\\n")); + assertTrue(PrometheusMetricsGeneratorUtils.labelValueNeedsEscape("repl\"")); + assertTrue(PrometheusMetricsGeneratorUtils.labelValueNeedsEscape("repl\\")); + assertTrue(PrometheusMetricsGeneratorUtils.labelValueNeedsEscape("\nrepl")); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java index 93cbad4e19503..ca05fa40a2530 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricStreams.java @@ -41,10 +41,7 @@ void writeSample(String metricName, Number value, String... labelsAndValuesArray SimpleTextOutputStream stream = initGaugeType(metricName); stream.write(metricName).write('{'); for (int i = 0; i < labelsAndValuesArray.length; i += 2) { - String labelValue = labelsAndValuesArray[i + 1]; - if (labelValue != null) { - labelValue = labelValue.replace("\"", "\\\""); - } + String labelValue = PrometheusMetricsGeneratorUtils.writeEscapedLabelValue(labelsAndValuesArray[i + 1]); stream.write(labelsAndValuesArray[i]).write("=\"").write(labelValue).write('\"'); if (i + 2 != labelsAndValuesArray.length) { stream.write(','); 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 501bfbbb16331..293ac30ecd123 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 @@ -298,9 +298,12 @@ private static void parseMetricsToPrometheusMetrics(Collection metrics, if (metric.getKey().isEmpty() || "cluster".equals(metric.getKey())) { continue; } - stream.write(", ").write(metric.getKey()).write("=\"").write(metric.getValue()).write('"'); + final String metricValue = PrometheusMetricsGeneratorUtils + .writeEscapedLabelValue(metric.getValue()); + stream.write(", ").write(metric.getKey()).write("=\"").write(metricValue).write('"'); if (value != null && !value.isEmpty() && !appendedQuantile) { - stream.write(", ").write("quantile=\"").write(value).write('"'); + stream.write(", ").write("quantile=\"").write(PrometheusMetricsGeneratorUtils + .writeEscapedLabelValue(value)).write('"'); appendedQuantile = true; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusTextFormatUtil.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusTextFormatUtil.java index b6a5d3f46ec9d..ffd6db3f8b47d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusTextFormatUtil.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusTextFormatUtil.java @@ -135,4 +135,5 @@ private static void writeSum(Writer w, DataSketchesOpStatsLogger opStat, String .append(success.toString()).append("\"} ") .append(Double.toString(opStat.getSum(success))).append('\n'); } + } 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 31d468394ff17..455da11f4fdf0 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 @@ -1966,6 +1966,10 @@ public String toString() { } } + /** + * Test both subscription and topic name with special characters. + * @throws Exception + */ @Test public void testEscapeLabelValue() throws Exception { String ns1 = "prop/ns-abc1"; @@ -1975,7 +1979,7 @@ public void testEscapeLabelValue() throws Exception { @Cleanup final Consumer consumer = pulsarClient.newConsumer() - .subscriptionName("sub") + .subscriptionName("s\"ub\\") .topic(topic) .subscribe(); @Cleanup @@ -1984,12 +1988,13 @@ public void testEscapeLabelValue() throws Exception { false, statsOut); String metricsStr = statsOut.toString(); final List subCountLines = metricsStr.lines() - .filter(line -> line.startsWith("pulsar_subscriptions_count")) + .filter(line -> line.startsWith("pulsar_subscription_msg_drop_rate")) .collect(Collectors.toList()); System.out.println(subCountLines); assertEquals(subCountLines.size(), 1); assertEquals(subCountLines.get(0), - "pulsar_subscriptions_count{cluster=\"test\",namespace=\"prop/ns-abc1\",topic=\"persistent://prop/ns-abc1/\\\"mytopic\"} 1"); + "pulsar_subscription_msg_drop_rate{cluster=\"test\",namespace=\"prop/ns-abc1\"," + + "topic=\"persistent://prop/ns-abc1/\\\"mytopic\",subscription=\"s\\\"ub\\\\\"} 0.0"); } } diff --git a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchMetrics.java b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchMetrics.java index 03b52324049c5..a1b5943f58848 100644 --- a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchMetrics.java +++ b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchMetrics.java @@ -1,4 +1,4 @@ -/* +/** * 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 diff --git a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClientTests.java b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClientTests.java index 3876f3639dd0a..399cda8dc5b76 100644 --- a/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClientTests.java +++ b/pulsar-io/elastic-search/src/test/java/org/apache/pulsar/io/elasticsearch/ElasticSearchClientTests.java @@ -20,7 +20,6 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThanOrEqualTo; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; From 9f7f8d0739ef05800e0befc5ac5d5c53177feae1 Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Fri, 8 Dec 2023 20:02:59 +0530 Subject: [PATCH 095/699] Fix license header for ElasticSearchMetrics.java --- .../apache/pulsar/io/elasticsearch/ElasticSearchMetrics.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchMetrics.java b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchMetrics.java index a1b5943f58848..03b52324049c5 100644 --- a/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchMetrics.java +++ b/pulsar-io/elastic-search/src/main/java/org/apache/pulsar/io/elasticsearch/ElasticSearchMetrics.java @@ -1,4 +1,4 @@ -/** +/* * 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 From 9ad080f5763827de1319df9c153435845c148d09 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Nicol=C3=B2=20Boschi?= Date: Tue, 13 Dec 2022 15:08:21 +0100 Subject: [PATCH 096/699] Flag to block transactions in replicated namespaces (#156) (cherry picked from commit 7144b56dffeaf45b4a7fca069288e852cf69973c) Conflicts: conf/broker.conf pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Consumer.java pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Producer.java pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java --- conf/broker.conf | 4 + .../pulsar/broker/ServiceConfiguration.java | 6 ++ .../pulsar/broker/service/Consumer.java | 24 ++++- .../pulsar/broker/service/Producer.java | 20 +++- .../broker/transaction/TransactionTest.java | 95 ++++++++++++++++++- 5 files changed, 141 insertions(+), 8 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index 86ad978affdd9..94c89414504f9 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1743,6 +1743,10 @@ transactionPendingAckBatchedWriteMaxSize=4194304 # the first record in a batch. transactionPendingAckBatchedWriteMaxDelayInMillis=1 +# Block transactions if the namespace or the topic is replicated +# See https://github.com/apache/pulsar/issues/18888 +blockTransactionsIfReplicationEnabled=false + ### --- Packages management service configuration variables (begin) --- ### # Enable the packages management service or not 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 b165e19b40211..5b6b3acad3fac 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 @@ -3126,6 +3126,12 @@ The delayed message index time step(in seconds) in per bucket snapshot segment, ) private long transactionPendingAckLogIndexMinLag = 500L; + @FieldContext( + category = CATEGORY_TRANSACTION, + doc = "Block transactions if replication is enabled on the namespace/topic." + ) + private boolean blockTransactionsIfReplicationEnabled = false; + @FieldContext( category = CATEGORY_SERVER, dynamic = true, 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 ecf8682084803..e4acb313c19a4 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 @@ -1,4 +1,4 @@ -/* +/*pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java * 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 @@ -44,6 +44,7 @@ import org.apache.commons.lang3.mutable.MutableInt; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.authentication.AuthenticationDataSubscription; +import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.MessageId; @@ -138,6 +139,7 @@ public class Consumer { private final String clientAddress; // IP address only, no port number included private final MessageId startMessageId; private final boolean isAcknowledgmentAtBatchIndexLevelEnabled; + private final boolean blockTransactionsIfReplicationEnabled; @Getter @Setter @@ -199,10 +201,11 @@ public Consumer(Subscription subscription, SubType subType, String topicName, lo stats.setClientVersion(cnx.getClientVersion()); stats.metadata = this.metadata; + final ServiceConfiguration serviceConfiguration = subscription.getTopic().getBrokerService() + .getPulsar().getConfiguration(); if (Subscription.isIndividualAckMode(subType)) { this.pendingAcks = ConcurrentLongLongPairHashMap.newBuilder() - .autoShrink(subscription.getTopic().getBrokerService() - .getPulsar().getConfiguration().isAutoShrinkForConsumerPendingAcksMap()) + .autoShrink(serviceConfiguration.isAutoShrinkForConsumerPendingAcksMap()) .expectedItems(256) .concurrencyLevel(1) .build(); @@ -215,7 +218,7 @@ public Consumer(Subscription subscription, SubType subType, String topicName, lo this.consumerEpoch = consumerEpoch; this.isAcknowledgmentAtBatchIndexLevelEnabled = subscription.getTopic().getBrokerService() .getPulsar().getConfiguration().isAcknowledgmentAtBatchIndexLevelEnabled(); - + this.blockTransactionsIfReplicationEnabled = serviceConfiguration.isBlockTransactionsIfReplicationEnabled(); this.schemaType = schemaType; } @@ -244,6 +247,7 @@ public Consumer(Subscription subscription, SubType subType, String topicName, lo this.clientAddress = null; this.startMessageId = null; this.isAcknowledgmentAtBatchIndexLevelEnabled = false; + this.blockTransactionsIfReplicationEnabled = false; this.schemaType = null; MESSAGE_PERMITS_UPDATER.set(this, availablePermits); } @@ -434,6 +438,18 @@ public CompletableFuture messageAcked(CommandAck ack) { .collect(Collectors.toMap(KeyLongValue::getKey, KeyLongValue::getValue)); } + if (subscription instanceof PersistentSubscription + && ack.hasTxnidMostBits() + && ack.hasTxnidLeastBits() + && blockTransactionsIfReplicationEnabled + && subscription.getTopic().isReplicated()) { + final CompletableFuture failed = new CompletableFuture<>(); + failed.completeExceptionally(new BrokerServiceException.NotAllowedException( + "Transactions are not allowed in a namespace with replication enabled" + )); + return failed; + } + if (ack.getAckType() == AckType.Cumulative) { if (ack.getMessageIdsCount() != 1) { log.warn("[{}] [{}] Received multi-message ack", subscription, consumerId); 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 f7d2bb2dd2797..fec2d52a239cc 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 @@ -91,6 +91,7 @@ public class Producer { private final boolean isNonPersistentTopic; private final boolean isShadowTopic; private final boolean isEncrypted; + private final boolean blockTransactionsIfReplicationEnabled; private final ProducerAccessMode accessMode; private Optional topicEpoch; @@ -153,6 +154,7 @@ public Producer(Topic topic, TransportCnx cnx, long producerId, String producerN this.schemaVersion = schemaVersion; this.accessMode = accessMode; this.topicEpoch = topicEpoch; + this.blockTransactionsIfReplicationEnabled = serviceConf.isBlockTransactionsIfReplicationEnabled(); this.clientAddress = cnx.clientSourceAddress(); this.brokerInterceptor = cnx.getBrokerService().getInterceptor(); @@ -186,7 +188,7 @@ public boolean isSuccessorTo(Producer other) { } public void publishMessage(long producerId, long sequenceId, ByteBuf headersAndPayload, long batchSize, - boolean isChunked, boolean isMarker, Position position) { + boolean isChunked, boolean isMarker, Position position) { if (checkAndStartPublish(producerId, sequenceId, headersAndPayload, batchSize, position)) { publishMessageToTopic(headersAndPayload, sequenceId, batchSize, isChunked, isMarker, position); } @@ -266,6 +268,19 @@ public boolean checkAndStartPublish(long producerId, long sequenceId, ByteBuf he return true; } + private boolean checkCanProduceTxnOnTopic(long sequenceId, ByteBuf headersAndPayload) { + if (blockTransactionsIfReplicationEnabled && topic.isReplicated()) { + cnx.execute(() -> { + cnx.getCommandSender().sendSendError(producerId, + sequenceId, ServerError.NotAllowedError, + "Transactions are not allowed in a namespace with replication enabled"); + cnx.completedSendOperation(isNonPersistentTopic, headersAndPayload.readableBytes()); + }); + return false; + } + return true; + } + private void publishMessageToTopic(ByteBuf headersAndPayload, long sequenceId, long batchSize, boolean isChunked, boolean isMarker, Position position) { MessagePublishContext messagePublishContext = @@ -806,6 +821,9 @@ public void checkEncryption() { public void publishTxnMessage(TxnID txnID, long producerId, long sequenceId, long highSequenceId, ByteBuf headersAndPayload, long batchSize, boolean isChunked, boolean isMarker) { + if (!checkCanProduceTxnOnTopic(sequenceId, headersAndPayload)) { + return; + } if (!checkAndStartPublish(producerId, sequenceId, headersAndPayload, batchSize, null)) { return; } 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 cf389824794e5..ae0cc912935aa 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 @@ -44,10 +44,12 @@ import java.lang.reflect.Method; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.List; +import java.util.HashSet; import java.util.Map; +import java.util.List; import java.util.Optional; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -132,6 +134,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.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ManagedLedgerInternalStats; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.TopicPolicies; @@ -155,6 +158,7 @@ import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; /** @@ -1669,8 +1673,8 @@ public void testEncryptionRequired() throws Exception { Transaction txn = pulsarClient.newTransaction() .withTransactionTimeout(5, TimeUnit.SECONDS).build().get(); producer.newMessage(txn) - .value(UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8)) - .send(); + .value(UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8)) + .send(); txn.commit(); } @@ -1783,4 +1787,89 @@ private void getTopic(String topicName) { }); } + @DataProvider(name = "BlockTransactionsIfReplicationEnabledValues") + public static Object[][] packageNamesProvider() { + return new Object[][]{ + {false},{true} + }; + } + + @Test(dataProvider = "BlockTransactionsIfReplicationEnabledValues") + public void testBlockTransactionsIfReplicationEnabled(boolean block) throws Exception { + conf.setBlockTransactionsIfReplicationEnabled(block); + admin.clusters().createCluster("r1", ClusterData.builder() + .serviceUrl(getPulsarServiceList().get(0).getWebServiceAddress()) + .build() + ); + final String namespace = TENANT + "/txndisabledns"; + admin.namespaces().createNamespace(namespace); + String topic = "persistent://" + namespace + "/block-" + block; + admin.topics().createNonPartitionedTopic(topic); + getPulsarServiceList().get(0) + .getPulsarResources() + .getNamespaceResources() + .setPolicies(NamespaceName.get(namespace), p -> { + p.replication_clusters = new HashSet<>(Arrays.asList(CLUSTER_NAME, "r1")); + return p; + }); + getPulsarServiceList().get(0) + .getBrokerService() + .getTopic(topic, false) + .get() + .get() + .checkReplication() + .get(); + + @Cleanup + Consumer consumer = getConsumer(topic, "s1"); + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .producerName("testBlocked").sendTimeout(0, TimeUnit.SECONDS) + .topic(topic).enableBatching(true) + .create(); + + Transaction transaction = pulsarClient.newTransaction() + .withTransactionTimeout(10, TimeUnit.SECONDS).build().get(); + try { + producer.newMessage(transaction) + .value("test") + .send(); + if (block) { + fail(); + } + } catch (PulsarClientException.NotAllowedException notAllowedException) { + if (block) { + assertEquals(notAllowedException.getMessage(), "Transactions are not allowed " + + "in a namespace with replication enabled"); + } else { + fail("unexpected exception with block=false: " + notAllowedException.getMessage()); + } + } + + + final MessageId msgNoTxn = producer.newMessage() + .value("testnotxn") + .send(); + + try { + consumer.acknowledgeAsync(msgNoTxn, transaction).get(); + if (block) { + fail(); + } + } catch (ExecutionException ex) { + if (block) { + assertTrue(PulsarClientException.unwrap(ex.getCause()).getMessage() + .contains("Transactions are not allowed in a namespace with replication enabled")); + } else { + fail("unexpected exception with block=false: " + ex.getCause().getMessage()); + } + } finally { + getPulsarServiceList().get(0) + .getPulsarResources() + .getNamespaceResources().deletePolicies(NamespaceName.get(namespace)); + admin.clusters().deleteCluster("r1"); + } + consumer.acknowledgeAsync(msgNoTxn).get(); + } + } From cd3a05cf6115d9014263a0385111486d5e6c1dc7 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Mon, 11 Dec 2023 13:48:58 +0530 Subject: [PATCH 097/699] Fix license header in Consumer.java --- .../main/java/org/apache/pulsar/broker/service/Consumer.java | 2 +- 1 file changed, 1 insertion(+), 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 e4acb313c19a4..290df6946fddc 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 @@ -1,4 +1,4 @@ -/*pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +/* * 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 From ce58e4c99f4cdcb02435a0d91eabe2531cea7cb4 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Mon, 11 Dec 2023 14:05:52 +0530 Subject: [PATCH 098/699] Fixed import order in Consumer.java --- .../main/java/org/apache/pulsar/broker/service/Consumer.java | 2 +- 1 file changed, 1 insertion(+), 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 290df6946fddc..a9ef268e7eea9 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 @@ -43,8 +43,8 @@ import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.commons.lang3.tuple.MutablePair; -import org.apache.pulsar.broker.authentication.AuthenticationDataSubscription; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.authentication.AuthenticationDataSubscription; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.MessageId; From 5054351494eaf2653c3c9f47048c5ecfd71b17c8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Nicol=C3=B2=20Boschi?= Date: Thu, 5 May 2022 10:05:17 +0200 Subject: [PATCH 099/699] Fix Owasp suppressions (move suppressions from org.apache.pulsar to com.datastax.oss) (cherry picked from commit a05abe68b0326d7d86a51d7bf732d9f3c33b0709) Conflicts: src/owasp-dependency-check-false-positives.xml --- src/owasp-dependency-check-false-positives.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/owasp-dependency-check-false-positives.xml b/src/owasp-dependency-check-false-positives.xml index 345be8f4d2c06..0ddbd81a5ac81 100644 --- a/src/owasp-dependency-check-false-positives.xml +++ b/src/owasp-dependency-check-false-positives.xml @@ -35,7 +35,7 @@ pulsar-package-bookkeeper-storage gets mixed with bookkeeper. - org\.apache\.pulsar:.* + com\.datastax\.oss:.* cpe:/a:apache:bookkeeper From 9f850e33d9722b79898bf407e1db3945afa73a4b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Nicol=C3=B2=20Boschi?= Date: Tue, 19 Jul 2022 12:30:04 +0200 Subject: [PATCH 100/699] Fix owasp-dependency-check supression file (cherry picked from commit 1d5525181038fe8cb7b43da04c659fbf40bfb8ef) Conflicts: src/owasp-dependency-check-false-positives.xml --- src/owasp-dependency-check-false-positives.xml | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/owasp-dependency-check-false-positives.xml b/src/owasp-dependency-check-false-positives.xml index 0ddbd81a5ac81..45e24756d1a89 100644 --- a/src/owasp-dependency-check-false-positives.xml +++ b/src/owasp-dependency-check-false-positives.xml @@ -129,8 +129,6 @@ - 6560f966bcf1aa78d27bcfa75fb6c4463a72c6c5 - CVE-2021-34429 From 9b7f67982c7cecc0f18a64d9d1ef62da82f4ee63 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Nicol=C3=B2=20Boschi?= Date: Mon, 18 Jul 2022 14:32:18 +0200 Subject: [PATCH 101/699] Create -experimental docker image with all the builtin connectors --- docker/pom.xml | 1 + docker/pulsar-all/pom.xml | 1 + docker/pulsar-experimental/Dockerfile | 35 +++++ docker/pulsar-experimental/pom.xml | 179 ++++++++++++++++++++++++++ 4 files changed, 216 insertions(+) create mode 100644 docker/pulsar-experimental/Dockerfile create mode 100644 docker/pulsar-experimental/pom.xml diff --git a/docker/pom.xml b/docker/pom.xml index f94b9fb1078e5..a9fb7603863e9 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -59,6 +59,7 @@ pulsar pulsar-all + pulsar-experimental diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index a18b813344961..cfe6b2558efcd 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -174,6 +174,7 @@ + diff --git a/docker/pulsar-experimental/Dockerfile b/docker/pulsar-experimental/Dockerfile new file mode 100644 index 0000000000000..29a800a4e47a3 --- /dev/null +++ b/docker/pulsar-experimental/Dockerfile @@ -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. +# + +FROM busybox as pulsar-experimental + +ARG PULSAR_IO_DIR +ARG PULSAR_OFFLOADER_TARBALL + +ADD ${PULSAR_IO_DIR} /connectors +ADD ${PULSAR_OFFLOADER_TARBALL} / +RUN mv /apache-pulsar-offloaders-*/offloaders /offloaders + +FROM apachepulsar/pulsar:latest +COPY --from=pulsar-experimental /connectors /pulsar/connectors +COPY --from=pulsar-experimental /offloaders /pulsar/offloaders + +USER 0 +RUN rm -rf /pulsar/connectors/*cassandra*.nar && rm -rf /pulsar/connectors/*kafka-connect-adaptor*.nar +USER 10000 \ No newline at end of file diff --git a/docker/pulsar-experimental/pom.xml b/docker/pulsar-experimental/pom.xml new file mode 100644 index 0000000000000..76919c36cb7e3 --- /dev/null +++ b/docker/pulsar-experimental/pom.xml @@ -0,0 +1,179 @@ + + + + com.datastax.oss + docker-images + 2.10.1.1 + + 4.0.0 + pulsar-experimental-docker-image + Apache Pulsar :: Docker Images :: Pulsar Latest Version (Include Experimental Components) + pom + + + + ${project.groupId} + pulsar-io-distribution + ${project.parent.version} + pom + provided + + + * + * + + + + + io.perfmark + perfmark-api + runtime + + + ${project.groupId} + pulsar-offloader-distribution + ${project.parent.version} + bin + tar.gz + provided + + + * + * + + + + + + + + docker + + + + ${project.groupId} + pulsar-docker-image + ${project.version} + docker-info + + + + + + maven-resources-plugin + + + copy-resources + generate-resources + + copy-resources + + + ${basedir}/target/apache-pulsar-io-connectors-${project.version}-bin + + + ${basedir}/../../distribution/io/target/apache-pulsar-io-connectors-${project.version}-bin + false + + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-offloader-tarball + + copy-dependencies + + generate-resources + + ${project.build.directory}/ + pulsar-offloader-distribution + true + + + + + + com.spotify + dockerfile-maven-plugin + ${dockerfile-maven.version} + + + default-experimental + + build + + + + add-no-repo-and-version-experimental + + tag + + + pulsar-experimental + ${project.version} + + + + add-no-repo-and-latest-experimental + + tag + + + pulsar-experimental + latest + + + + tag-and-push-latest-experimental + + tag + push + + + ${docker.organization}/pulsar-experimental + latest + + + + + ${docker.organization}/pulsar-experimental + false + ${project.version} + + target/apache-pulsar-io-connectors-${project.version}-bin + target/pulsar-offloader-distribution-${project.version}-bin.tar.gz + + + + + + + + + From 9ed0a30b8781efb6b049a4eba1a5aabd41035bc6 Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Tue, 19 Dec 2023 11:56:10 +0530 Subject: [PATCH 102/699] Updated pulsar-experimental/pom.xml for version 3.1 --- docker/pulsar-experimental/pom.xml | 342 ++++++++++++++++------------- 1 file changed, 189 insertions(+), 153 deletions(-) diff --git a/docker/pulsar-experimental/pom.xml b/docker/pulsar-experimental/pom.xml index 76919c36cb7e3..89344e610a37d 100644 --- a/docker/pulsar-experimental/pom.xml +++ b/docker/pulsar-experimental/pom.xml @@ -20,160 +20,196 @@ --> - - com.datastax.oss - docker-images - 2.10.1.1 - - 4.0.0 - pulsar-experimental-docker-image - Apache Pulsar :: Docker Images :: Pulsar Latest Version (Include Experimental Components) - pom + + com.datastax.oss + docker-images + 3.1.1 + + 4.0.0 + pulsar-experimental-docker-image + Apache Pulsar :: Docker Images :: Pulsar Latest Version (Include Experimental Components) + pom - - - ${project.groupId} - pulsar-io-distribution - ${project.parent.version} - pom - provided - - - * - * - - - - - io.perfmark - perfmark-api - runtime - - - ${project.groupId} - pulsar-offloader-distribution - ${project.parent.version} - bin - tar.gz - provided - - - * - * - - - - - - - - docker - - + + + ${project.groupId} + pulsar-io-distribution + ${project.parent.version} + pom + provided + + + * + * + + + - ${project.groupId} - pulsar-docker-image - ${project.version} - docker-info + io.perfmark + perfmark-api + runtime - - - - - maven-resources-plugin - - - copy-resources - generate-resources - - copy-resources - - - ${basedir}/target/apache-pulsar-io-connectors-${project.version}-bin - - - ${basedir}/../../distribution/io/target/apache-pulsar-io-connectors-${project.version}-bin - false - - - - - - - - org.apache.maven.plugins - maven-dependency-plugin - - - copy-offloader-tarball - - copy-dependencies - - generate-resources - - ${project.build.directory}/ - pulsar-offloader-distribution - true - - - - - - com.spotify - dockerfile-maven-plugin - ${dockerfile-maven.version} - - - default-experimental - - build - - - - add-no-repo-and-version-experimental - - tag - - - pulsar-experimental - ${project.version} - - - - add-no-repo-and-latest-experimental - - tag - - - pulsar-experimental - latest - - - - tag-and-push-latest-experimental - - tag - push - - - ${docker.organization}/pulsar-experimental - latest - - - - - ${docker.organization}/pulsar-experimental - false - ${project.version} - - target/apache-pulsar-io-connectors-${project.version}-bin - target/pulsar-offloader-distribution-${project.version}-bin.tar.gz - - - + + ${project.groupId} + pulsar-offloader-distribution + ${project.parent.version} + bin + tar.gz + provided + + + * + * + + + + + + + + git-commit-id-no-git + + + ${basedir}/../../.git/index + + + + no-git + + + + docker + + + + ${project.groupId} + pulsar-docker-image + ${project.version} + pom + provided + + + * + * + + + + + + + + maven-resources-plugin + + + copy-resources + generate-resources + + copy-resources + + + + ${basedir}/target/apache-pulsar-io-connectors-${project.version}-bin + + + + + ${basedir}/../../distribution/io/target/apache-pulsar-io-connectors-${project.version}-bin + + false + + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-offloader-tarball + + copy-dependencies + + generate-resources + + ${project.build.directory}/ + pulsar-offloader-distribution + true + + + + + + io.fabric8 + docker-maven-plugin + + + default-experimental + package + + build + tag + + + + + ${docker.organization}/pulsar-experimental + + ${project.basedir} + + latest + ${project.version}-${git.commit.id.abbrev} + + + + target/apache-pulsar-io-connectors-${project.version}-bin + + + target/pulsar-offloader-distribution-${project.version}-bin.tar.gz + + + ${docker.organization}/pulsar:${project.version}-${git.commit.id.abbrev} + + + + + ${docker.platforms} + + + + + + + + + + + + + + + docker-push + + + + io.fabric8 + docker-maven-plugin + + + default-experimental + package + + build + tag + push + + + + - - - - - + + + + + \ No newline at end of file From 526c681a14dcfa5fa71b60f9be14aaa54b2ce8f2 Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Wed, 20 Dec 2023 13:04:47 +0530 Subject: [PATCH 103/699] Excluded com.datastax.oss:pulsar-experimental-docker-image --- build/run_unit_group.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build/run_unit_group.sh b/build/run_unit_group.sh index 216b239dbf1b7..f6c212dcdd28f 100755 --- a/build/run_unit_group.sh +++ b/build/run_unit_group.sh @@ -151,7 +151,7 @@ function test_group_proxy() { function test_group_other() { mvn_test --clean --install \ - -pl '!com.datastax.oss:distribution,!com.datastax.oss:pulsar-offloader-distribution,!com.datastax.oss:pulsar-server-distribution,!com.datastax.oss:pulsar-io-distribution,!com.datastax.oss:pulsar-all-docker-image' \ + -pl '!com.datastax.oss:distribution,!com.datastax.oss:pulsar-offloader-distribution,!com.datastax.oss:pulsar-server-distribution,!com.datastax.oss:pulsar-io-distribution,!com.datastax.oss:pulsar-all-docker-image,!com.datastax.oss:pulsar-experimental-docker-image' \ -PskipTestsForUnitGroupOther -DdisableIoMainProfile=true -DdisableSqlMainProfile=true -DskipIntegrationTests \ -Dexclude='**/ManagedLedgerTest.java, **/OffloadersCacheTest.java From f39032d2c6d60e1c29e169bb01c101641d22e35f Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 30 Sep 2023 11:27:36 +0300 Subject: [PATCH 104/699] [fix][sec] Add OWASP Dependency Check suppressions (#21281) (cherry picked from commit 1bf7371b6d33c4e015d006e547b393b97686ff20) --- src/owasp-dependency-check-suppressions.xml | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/src/owasp-dependency-check-suppressions.xml b/src/owasp-dependency-check-suppressions.xml index d5ddc28e884cb..b5bb58c3d0eaf 100644 --- a/src/owasp-dependency-check-suppressions.xml +++ b/src/owasp-dependency-check-suppressions.xml @@ -457,4 +457,16 @@ ]]> CVE-2023-35116 + + + CVE-2023-37475 + + + + CVE-2023-4586 + From 38216da692d1298c5a0684870528994432455cbc Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 30 Sep 2023 12:14:11 +0300 Subject: [PATCH 105/699] [fix][build] Upgrade Lombok to 1.18.30 to support compiling with JDK21 (#21278) (cherry picked from commit 682eb36a76746fd47ce57dd8915e3d4603aa039e) # Conflicts: # pom.xml --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 8c8571597b84f..fc003333e2e82 100644 --- a/pom.xml +++ b/pom.xml @@ -223,7 +223,7 @@ flexible messaging model and an intuitive client API. 0.9.1 2.1.0 3.24.2 - 1.18.26 + 1.18.30 1.3.2 2.3.1 1.2.0 From ed6107c08f27daaabaf373c57c6e469ef4e9539a Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 7 Oct 2023 05:26:48 +0300 Subject: [PATCH 106/699] [fix][ml] Fix thread safe issue with RangeCache.put and RangeCache.clear (#21302) (cherry picked from commit 70d086f8f35d36800059d0d68e13d0ca017bf233) --- .../bookkeeper/mledger/util/RangeCache.java | 21 +++++++++++-------- 1 file changed, 12 insertions(+), 9 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 7599e2cc1874f..d34857e5e5177 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 @@ -27,7 +27,6 @@ import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicLong; -import org.apache.commons.lang3.mutable.MutableBoolean; import org.apache.commons.lang3.tuple.Pair; /** @@ -74,13 +73,18 @@ public RangeCache(Weighter weighter, TimestampExtractor timestampE * @return whether the entry was inserted in the cache */ public boolean put(Key key, Value value) { - MutableBoolean flag = new MutableBoolean(); - entries.computeIfAbsent(key, (k) -> { - size.addAndGet(weighter.getSize(value)); - flag.setValue(true); - return value; - }); - return flag.booleanValue(); + // retain value so that it's not released before we put it in the cache and calculate the weight + value.retain(); + try { + if (entries.putIfAbsent(key, value) == null) { + size.addAndGet(weighter.getSize(value)); + return true; + } else { + return false; + } + } finally { + value.release(); + } } public boolean exists(Key key) { @@ -242,7 +246,6 @@ public synchronized Pair clear() { value.release(); } - entries.clear(); size.getAndAdd(-removedSize); return Pair.of(removedCount, removedSize); } From cdc9cf15a5cb2e64eef594ce683a88d14bf4b299 Mon Sep 17 00:00:00 2001 From: Guangning E Date: Fri, 11 Aug 2023 23:21:32 +0800 Subject: [PATCH 107/699] [fix][broker]Check that the super user role is in the MultiRolesTokenAuthorizationProvider plugin (#20939) Fixed https://github.com/apache/pulsar/issues/20938 --- .../MultiRolesTokenAuthorizationProvider.java | 13 +++++--- ...tiRolesTokenAuthorizationProviderTest.java | 32 +++++++++++++++++++ 2 files changed, 40 insertions(+), 5 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java index fa613245cfa27..db5f4f18e8cc3 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java @@ -89,15 +89,18 @@ public void initialize(ServiceConfiguration conf, PulsarResources pulsarResource @Override public CompletableFuture isSuperUser(String role, AuthenticationDataSource authenticationData, ServiceConfiguration serviceConfiguration) { - Set roles = getRoles(authenticationData); - if (roles.isEmpty()) { - return CompletableFuture.completedFuture(false); - } + // if superUser role contains in config, return true. Set superUserRoles = serviceConfiguration.getSuperUserRoles(); if (superUserRoles.isEmpty()) { return CompletableFuture.completedFuture(false); } - + if (role != null && superUserRoles.contains(role)) { + return CompletableFuture.completedFuture(true); + } + Set roles = getRoles(authenticationData); + if (roles.isEmpty()) { + return CompletableFuture.completedFuture(false); + } return CompletableFuture.completedFuture(roles.stream().anyMatch(superUserRoles::contains)); } diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java index 7e329d14307c7..f0a857bdd695d 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java @@ -31,6 +31,7 @@ import org.testng.annotations.Test; import javax.crypto.SecretKey; +import java.util.Set; import java.util.concurrent.CompletableFuture; public class MultiRolesTokenAuthorizationProviderTest { @@ -198,4 +199,35 @@ public String getHttpHeader(String name) { return CompletableFuture.completedFuture(false); }).get()); } + + @Test + public void testMultiRolesAuthzWithSuperUser() throws Exception { + SecretKey secretKey = AuthTokenUtils.createSecretKey(SignatureAlgorithm.HS256); + String testAdminRole = "admin"; + String token = Jwts.builder().claim("sub", testAdminRole).signWith(secretKey).compact(); + + ServiceConfiguration conf = new ServiceConfiguration(); + conf.setSuperUserRoles(Set.of(testAdminRole)); + + MultiRolesTokenAuthorizationProvider provider = new MultiRolesTokenAuthorizationProvider(); + provider.initialize(conf, mock(PulsarResources.class)); + + AuthenticationDataSource ads = new AuthenticationDataSource() { + @Override + public boolean hasDataFromHttp() { + return true; + } + + @Override + public String getHttpHeader(String name) { + if (name.equals("Authorization")) { + return "Bearer " + token; + } else { + throw new IllegalArgumentException("Wrong HTTP header"); + } + } + }; + + assertTrue(provider.isSuperUser(testAdminRole, ads, conf).get()); + } } From e981fea3416761baf705e91c87d2712962b66695 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 10 Oct 2023 23:45:10 +0800 Subject: [PATCH 108/699] [fix][sec] Fix MultiRoles token provider when using anonymous clients (#21338) Co-authored-by: Lari Hotari --- .../MultiRolesTokenAuthorizationProvider.java | 45 +++++++++++-------- ...tiRolesTokenAuthorizationProviderTest.java | 31 ++++++++++--- 2 files changed, 50 insertions(+), 26 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java index db5f4f18e8cc3..6376b60217fef 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java @@ -97,7 +97,7 @@ public CompletableFuture isSuperUser(String role, AuthenticationDataSou if (role != null && superUserRoles.contains(role)) { return CompletableFuture.completedFuture(true); } - Set roles = getRoles(authenticationData); + Set roles = getRoles(role, authenticationData); if (roles.isEmpty()) { return CompletableFuture.completedFuture(false); } @@ -112,7 +112,7 @@ public CompletableFuture validateTenantAdminAccess(String tenantName, S if (isSuperUser) { return CompletableFuture.completedFuture(true); } - Set roles = getRoles(authData); + Set roles = getRoles(role, authData); if (roles.isEmpty()) { return CompletableFuture.completedFuture(false); } @@ -143,7 +143,11 @@ public CompletableFuture validateTenantAdminAccess(String tenantName, S }); } - private Set getRoles(AuthenticationDataSource authData) { + private Set getRoles(String role, AuthenticationDataSource authData) { + if (authData == null) { + return Collections.singleton(role); + } + String token = null; if (authData.hasDataFromCommand()) { @@ -192,9 +196,9 @@ private Set getRoles(AuthenticationDataSource authData) { return Collections.emptySet(); } - public CompletableFuture authorize(AuthenticationDataSource authenticationData, Function> authorizeFunc) { - Set roles = getRoles(authenticationData); + public CompletableFuture authorize(String role, AuthenticationDataSource authenticationData, + Function> authorizeFunc) { + Set roles = getRoles(role, authenticationData); if (roles.isEmpty()) { return CompletableFuture.completedFuture(false); } @@ -212,7 +216,7 @@ public CompletableFuture authorize(AuthenticationDataSource authenticat @Override public CompletableFuture canProduceAsync(TopicName topicName, String role, AuthenticationDataSource authenticationData) { - return authorize(authenticationData, r -> super.canProduceAsync(topicName, r, authenticationData)); + return authorize(role, authenticationData, r -> super.canProduceAsync(topicName, r, authenticationData)); } /** @@ -227,7 +231,7 @@ public CompletableFuture canProduceAsync(TopicName topicName, String ro public CompletableFuture canConsumeAsync(TopicName topicName, String role, AuthenticationDataSource authenticationData, String subscription) { - return authorize(authenticationData, r -> super.canConsumeAsync(topicName, r, authenticationData, + return authorize(role, authenticationData, r -> super.canConsumeAsync(topicName, r, authenticationData, subscription)); } @@ -244,25 +248,27 @@ public CompletableFuture canConsumeAsync(TopicName topicName, String ro @Override public CompletableFuture canLookupAsync(TopicName topicName, String role, AuthenticationDataSource authenticationData) { - return authorize(authenticationData, r -> super.canLookupAsync(topicName, r, authenticationData)); + return authorize(role, authenticationData, r -> super.canLookupAsync(topicName, r, authenticationData)); } @Override public CompletableFuture allowFunctionOpsAsync(NamespaceName namespaceName, String role, AuthenticationDataSource authenticationData) { - return authorize(authenticationData, r -> super.allowFunctionOpsAsync(namespaceName, r, authenticationData)); + return authorize(role, authenticationData, + r -> super.allowFunctionOpsAsync(namespaceName, r, authenticationData)); } @Override public CompletableFuture allowSourceOpsAsync(NamespaceName namespaceName, String role, AuthenticationDataSource authenticationData) { - return authorize(authenticationData, r -> super.allowSourceOpsAsync(namespaceName, r, authenticationData)); + return authorize(role, authenticationData, + r -> super.allowSourceOpsAsync(namespaceName, r, authenticationData)); } @Override public CompletableFuture allowSinkOpsAsync(NamespaceName namespaceName, String role, AuthenticationDataSource authenticationData) { - return authorize(authenticationData, r -> super.allowSinkOpsAsync(namespaceName, r, authenticationData)); + return authorize(role, authenticationData, r -> super.allowSinkOpsAsync(namespaceName, r, authenticationData)); } @Override @@ -270,7 +276,7 @@ public CompletableFuture allowTenantOperationAsync(String tenantName, String role, TenantOperation operation, AuthenticationDataSource authData) { - return authorize(authData, r -> super.allowTenantOperationAsync(tenantName, r, operation, authData)); + return authorize(role, authData, r -> super.allowTenantOperationAsync(tenantName, r, operation, authData)); } @Override @@ -278,7 +284,8 @@ public CompletableFuture allowNamespaceOperationAsync(NamespaceName nam String role, NamespaceOperation operation, AuthenticationDataSource authData) { - return authorize(authData, r -> super.allowNamespaceOperationAsync(namespaceName, r, operation, authData)); + return authorize(role, authData, + r -> super.allowNamespaceOperationAsync(namespaceName, r, operation, authData)); } @Override @@ -287,8 +294,8 @@ public CompletableFuture allowNamespacePolicyOperationAsync(NamespaceNa PolicyOperation operation, String role, AuthenticationDataSource authData) { - return authorize(authData, r -> super.allowNamespacePolicyOperationAsync(namespaceName, policy, operation, r, - authData)); + return authorize(role, authData, + r -> super.allowNamespacePolicyOperationAsync(namespaceName, policy, operation, r, authData)); } @Override @@ -296,7 +303,7 @@ public CompletableFuture allowTopicOperationAsync(TopicName topicName, String role, TopicOperation operation, AuthenticationDataSource authData) { - return authorize(authData, r -> super.allowTopicOperationAsync(topicName, r, operation, authData)); + return authorize(role, authData, r -> super.allowTopicOperationAsync(topicName, r, operation, authData)); } @Override @@ -305,7 +312,7 @@ public CompletableFuture allowTopicPolicyOperationAsync(TopicName topic PolicyName policyName, PolicyOperation policyOperation, AuthenticationDataSource authData) { - return authorize(authData, r -> super.allowTopicPolicyOperationAsync(topicName, r, policyName, policyOperation, - authData)); + return authorize(role, authData, + r -> super.allowTopicPolicyOperationAsync(topicName, r, policyName, policyOperation, authData)); } } diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java index f0a857bdd695d..4b67f52075ca9 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java @@ -24,6 +24,8 @@ import io.jsonwebtoken.Jwts; import io.jsonwebtoken.SignatureAlgorithm; import java.util.Properties; +import java.util.function.Function; +import lombok.Cleanup; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils; @@ -61,18 +63,18 @@ public String getHttpHeader(String name) { } }; - assertTrue(provider.authorize(ads, role -> { + assertTrue(provider.authorize("test", ads, role -> { if (role.equals(userB)) { return CompletableFuture.completedFuture(true); // only userB has permission } return CompletableFuture.completedFuture(false); }).get()); - assertTrue(provider.authorize(ads, role -> { + assertTrue(provider.authorize("test", ads, role -> { return CompletableFuture.completedFuture(true); // all users has permission }).get()); - assertFalse(provider.authorize(ads, role -> { + assertFalse(provider.authorize("test", ads, role -> { return CompletableFuture.completedFuture(false); // all users has no permission }).get()); } @@ -100,7 +102,7 @@ public String getHttpHeader(String name) { } }; - assertFalse(provider.authorize(ads, role -> CompletableFuture.completedFuture(false)).get()); + assertFalse(provider.authorize("test", ads, role -> CompletableFuture.completedFuture(false)).get()); } @Test @@ -127,7 +129,7 @@ public String getHttpHeader(String name) { } }; - assertTrue(provider.authorize(ads, role -> { + assertTrue(provider.authorize("test", ads, role -> { if (role.equals(testRole)) { return CompletableFuture.completedFuture(true); } @@ -135,6 +137,21 @@ public String getHttpHeader(String name) { }).get()); } + @Test + public void testMultiRolesAuthzWithAnonymousUser() throws Exception { + @Cleanup + MultiRolesTokenAuthorizationProvider provider = new MultiRolesTokenAuthorizationProvider(); + + Function> authorizeFunc = (String role) -> { + if (role.equals("test-role")) { + return CompletableFuture.completedFuture(true); + } + return CompletableFuture.completedFuture(false); + }; + assertTrue(provider.authorize("test-role", null, authorizeFunc).get()); + assertFalse(provider.authorize("test-role-x", null, authorizeFunc).get()); + } + @Test public void testMultiRolesNotFailNonJWT() throws Exception { String token = "a-non-jwt-token"; @@ -157,7 +174,7 @@ public String getHttpHeader(String name) { } }; - assertFalse(provider.authorize(ads, role -> CompletableFuture.completedFuture(false)).get()); + assertFalse(provider.authorize("test", ads, role -> CompletableFuture.completedFuture(false)).get()); } @Test @@ -192,7 +209,7 @@ public String getHttpHeader(String name) { } }; - assertTrue(provider.authorize(ads, role -> { + assertTrue(provider.authorize("test", ads, role -> { if (role.equals(testRole)) { return CompletableFuture.completedFuture(true); } From e290abbc3c9a87b57d02444d2b04ff9bbb1c8801 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Sat, 23 Sep 2023 22:15:23 +0800 Subject: [PATCH 109/699] [fix] [broker] fix flaky test PatternTopicsConsumerImplTest (#21222) (cherry picked from commit be4bcac11ae76cdf3d4c4b0639735f3309919e4c) --- .../impl/PatternTopicsConsumerImplTest.java | 55 +++++++++++++++++++ 1 file changed, 55 insertions(+) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java index c8f7b721cce6a..451f93067b2ca 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java @@ -48,6 +48,7 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.annotations.AfterMethod; @@ -67,6 +68,7 @@ public void setup() throws Exception { isTcpLookup = true; // enabled transaction, to test pattern consumers not subscribe to transaction system topic. conf.setTransactionCoordinatorEnabled(true); + conf.setSubscriptionPatternMaxLength(10000); super.internalSetup(); super.producerBaseSetup(); } @@ -210,6 +212,12 @@ public void testBinaryProtoToGetTopicsOfNamespacePersistent() throws Exception { .subscribe(); assertTrue(consumer.getTopic().startsWith(PatternMultiTopicsConsumerImpl.DUMMY_TOPIC_NAME_PREFIX)); + // Wait topic list watcher creation. + Awaitility.await().untilAsserted(() -> { + CompletableFuture completableFuture = WhiteboxImpl.getInternalState(consumer, "watcherFuture"); + assertTrue(completableFuture.isDone() && !completableFuture.isCompletedExceptionally()); + }); + // 4. verify consumer get methods, to get right number of partitions and topics. assertSame(pattern, ((PatternMultiTopicsConsumerImpl) consumer).getPattern()); List topics = ((PatternMultiTopicsConsumerImpl) consumer).getPartitions(); @@ -287,6 +295,12 @@ public void testBinaryProtoSubscribeAllTopicOfNamespace() throws Exception { .subscribe(); assertTrue(consumer.getTopic().startsWith(PatternMultiTopicsConsumerImpl.DUMMY_TOPIC_NAME_PREFIX)); + // Wait topic list watcher creation. + Awaitility.await().untilAsserted(() -> { + CompletableFuture completableFuture = WhiteboxImpl.getInternalState(consumer, "watcherFuture"); + assertTrue(completableFuture.isDone() && !completableFuture.isCompletedExceptionally()); + }); + // 4. verify consumer get methods, to get right number of partitions and topics. assertSame(pattern, ((PatternMultiTopicsConsumerImpl) consumer).getPattern()); List topics = ((PatternMultiTopicsConsumerImpl) consumer).getPartitions(); @@ -364,6 +378,12 @@ public void testBinaryProtoToGetTopicsOfNamespaceNonPersistent() throws Exceptio .subscriptionTopicsMode(RegexSubscriptionMode.NonPersistentOnly) .subscribe(); + // Wait topic list watcher creation. + Awaitility.await().untilAsserted(() -> { + CompletableFuture completableFuture = WhiteboxImpl.getInternalState(consumer, "watcherFuture"); + assertTrue(completableFuture.isDone() && !completableFuture.isCompletedExceptionally()); + }); + // 4. verify consumer get methods, to get right number of partitions and topics. assertSame(pattern, ((PatternMultiTopicsConsumerImpl) consumer).getPattern()); List topics = ((PatternMultiTopicsConsumerImpl) consumer).getPartitions(); @@ -455,6 +475,12 @@ public void testBinaryProtoToGetTopicsOfNamespaceAll() throws Exception { .ackTimeout(ackTimeOutMillis, TimeUnit.MILLISECONDS) .subscribe(); + // Wait topic list watcher creation. + Awaitility.await().untilAsserted(() -> { + CompletableFuture completableFuture = WhiteboxImpl.getInternalState(consumer, "watcherFuture"); + assertTrue(completableFuture.isDone() && !completableFuture.isCompletedExceptionally()); + }); + // 4. verify consumer get methods, to get right number of partitions and topics. assertSame(pattern, ((PatternMultiTopicsConsumerImpl) consumer).getPattern()); List topics = ((PatternMultiTopicsConsumerImpl) consumer).getPartitions(); @@ -525,6 +551,11 @@ public void testStartEmptyPatternConsumer() throws Exception { .ackTimeout(ackTimeOutMillis, TimeUnit.MILLISECONDS) .receiverQueueSize(4) .subscribe(); + // Wait topic list watcher creation. + Awaitility.await().untilAsserted(() -> { + CompletableFuture completableFuture = WhiteboxImpl.getInternalState(consumer, "watcherFuture"); + assertTrue(completableFuture.isDone() && !completableFuture.isCompletedExceptionally()); + }); // 3. verify consumer get methods, to get 5 number of partitions and topics. assertSame(pattern, ((PatternMultiTopicsConsumerImpl) consumer).getPattern()); @@ -605,6 +636,12 @@ public void testAutoSubscribePatterConsumerFromBrokerWatcher() throws Exception .receiverQueueSize(4) .subscribe(); + // Wait topic list watcher creation. + Awaitility.await().untilAsserted(() -> { + CompletableFuture completableFuture = WhiteboxImpl.getInternalState(consumer, "watcherFuture"); + assertTrue(completableFuture.isDone() && !completableFuture.isCompletedExceptionally()); + }); + // 1. create partition String topicName = "persistent://my-property/my-ns/pattern-topic-1-" + key; TenantInfoImpl tenantInfo = createDefaultTenantInfo(); @@ -665,6 +702,12 @@ public void testAutoSubscribePatternConsumer() throws Exception { .receiverQueueSize(4) .subscribe(); + // Wait topic list watcher creation. + Awaitility.await().untilAsserted(() -> { + CompletableFuture completableFuture = WhiteboxImpl.getInternalState(consumer, "watcherFuture"); + assertTrue(completableFuture.isDone() && !completableFuture.isCompletedExceptionally()); + }); + assertTrue(consumer instanceof PatternMultiTopicsConsumerImpl); // 4. verify consumer get methods, to get 6 number of partitions and topics: 6=1+2+3 @@ -775,6 +818,12 @@ public void testAutoUnsubscribePatternConsumer() throws Exception { .receiverQueueSize(4) .subscribe(); + // Wait topic list watcher creation. + Awaitility.await().untilAsserted(() -> { + CompletableFuture completableFuture = WhiteboxImpl.getInternalState(consumer, "watcherFuture"); + assertTrue(completableFuture.isDone() && !completableFuture.isCompletedExceptionally()); + }); + assertTrue(consumer instanceof PatternMultiTopicsConsumerImpl); // 4. verify consumer get methods, to get 0 number of partitions and topics: 6=1+2+3 @@ -861,6 +910,12 @@ public void testTopicDeletion() throws Exception { .subscriptionName("sub") .subscribe(); + // Wait topic list watcher creation. + Awaitility.await().untilAsserted(() -> { + CompletableFuture completableFuture = WhiteboxImpl.getInternalState(consumer, "watcherFuture"); + assertTrue(completableFuture.isDone() && !completableFuture.isCompletedExceptionally()); + }); + assertTrue(consumer instanceof PatternMultiTopicsConsumerImpl); PatternMultiTopicsConsumerImpl consumerImpl = (PatternMultiTopicsConsumerImpl) consumer; From 8811d5b702efed232f7891a8b94fa20e5558ff4f Mon Sep 17 00:00:00 2001 From: Yan Zhao Date: Wed, 11 Oct 2023 11:53:35 +0800 Subject: [PATCH 110/699] [improve] [auto-recovery] [branch-3.1] Migrate the replication testing from BookKeeper to Pulsar. (#21340) There is no testing for AutoRecovery replication in Pulsar's current test suite, and we need to cover it. So migrate the replication testing from BookKeeper to Pulsar. --- .../bookkeeper/PulsarLayoutManager.java | 22 +- .../PulsarLedgerAuditorManager.java | 4 +- .../bookkeeper/PulsarLedgerManager.java | 2 +- .../AuditorBookieCheckTaskTest.java | 153 ++ .../replication/AuditorBookieTest.java | 299 ++++ .../AuditorCheckAllLedgersTaskTest.java | 136 ++ .../replication/AuditorLedgerCheckerTest.java | 778 +++++++++- .../AuditorPeriodicBookieCheckTest.java | 117 ++ .../replication/AuditorPeriodicCheckTest.java | 852 +++++++++++ .../AuditorPlacementPolicyCheckTaskTest.java | 124 ++ .../AuditorPlacementPolicyCheckTest.java | 861 ++++++++++++ .../AuditorReplicasCheckTaskTest.java | 119 ++ .../replication/AuditorReplicasCheckTest.java | 937 +++++++++++++ .../AuditorRollingRestartTest.java | 112 ++ .../replication/AuthAutoRecoveryTest.java | 119 ++ .../replication/AutoRecoveryMainTest.java | 55 +- .../replication/BookieAutoRecoveryTest.java | 652 +++++++++ .../replication/BookieLedgerIndexTest.java | 248 ++++ .../replication/ReplicationTestUtil.java | 61 + ...estAutoRecoveryAlongWithBookieServers.java | 117 ++ .../replication/TestReplicationWorker.java | 1249 +++++++++++++++++ 21 files changed, 6982 insertions(+), 35 deletions(-) create mode 100644 pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorBookieCheckTaskTest.java create mode 100644 pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java create mode 100644 pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorCheckAllLedgersTaskTest.java create mode 100644 pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java create mode 100644 pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTaskTest.java create mode 100644 pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java create mode 100644 pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTaskTest.java create mode 100644 pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java create mode 100644 pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorRollingRestartTest.java create mode 100644 pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuthAutoRecoveryTest.java create mode 100644 pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java create mode 100644 pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookieLedgerIndexTest.java create mode 100644 pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/ReplicationTestUtil.java create mode 100644 pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java create mode 100644 pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLayoutManager.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLayoutManager.java index 4444295b6169e..ee06930b3c880 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLayoutManager.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLayoutManager.java @@ -18,12 +18,9 @@ */ package org.apache.pulsar.metadata.bookkeeper; -import static java.util.concurrent.TimeUnit.MILLISECONDS; -import static org.apache.pulsar.metadata.bookkeeper.AbstractMetadataDriver.BLOCKING_CALL_TIMEOUT; import java.io.IOException; import java.util.Optional; import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeoutException; import lombok.AccessLevel; import lombok.Getter; import org.apache.bookkeeper.bookie.BookieException; @@ -33,8 +30,7 @@ import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; - -class PulsarLayoutManager implements LayoutManager { +public class PulsarLayoutManager implements LayoutManager { @Getter(AccessLevel.PACKAGE) private final MetadataStoreExtended store; @@ -44,7 +40,7 @@ class PulsarLayoutManager implements LayoutManager { private final String layoutPath; - PulsarLayoutManager(MetadataStoreExtended store, String ledgersRootPath) { + public PulsarLayoutManager(MetadataStoreExtended store, String ledgersRootPath) { this.ledgersRootPath = ledgersRootPath; this.store = store; this.layoutPath = ledgersRootPath + "/" + BookKeeperConstants.LAYOUT_ZNODE; @@ -53,14 +49,14 @@ class PulsarLayoutManager implements LayoutManager { @Override public LedgerLayout readLedgerLayout() throws IOException { try { - byte[] layoutData = store.get(layoutPath).get(BLOCKING_CALL_TIMEOUT, MILLISECONDS) + byte[] layoutData = store.get(layoutPath).get() .orElseThrow(() -> new BookieException.MetadataStoreException("Layout node not found")) .getValue(); return LedgerLayout.parseLayout(layoutData); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new IOException(e); - } catch (BookieException | ExecutionException | TimeoutException e) { + } catch (BookieException | ExecutionException e) { throw new IOException(e); } } @@ -70,13 +66,10 @@ public void storeLedgerLayout(LedgerLayout ledgerLayout) throws IOException { try { byte[] layoutData = ledgerLayout.serialize(); - store.put(layoutPath, layoutData, Optional.of(-1L)) - .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); + store.put(layoutPath, layoutData, Optional.of(-1L)).get(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new IOException(e); - } catch (TimeoutException e) { - throw new IOException(e); } catch (ExecutionException e) { if (e.getCause() instanceof MetadataStoreException.BadVersionException) { throw new LedgerLayoutExistsException(e); @@ -89,12 +82,11 @@ public void storeLedgerLayout(LedgerLayout ledgerLayout) throws IOException { @Override public void deleteLedgerLayout() throws IOException { try { - store.delete(layoutPath, Optional.empty()) - .get(BLOCKING_CALL_TIMEOUT, MILLISECONDS); + store.delete(layoutPath, Optional.empty()).get(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new IOException(e); - } catch (ExecutionException | TimeoutException e) { + } catch (ExecutionException e) { throw new IOException(e); } } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerAuditorManager.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerAuditorManager.java index d664ecdcd2016..44870ed47f05b 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerAuditorManager.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerAuditorManager.java @@ -31,9 +31,9 @@ import org.apache.pulsar.metadata.coordination.impl.CoordinationServiceImpl; @Slf4j -class PulsarLedgerAuditorManager implements LedgerAuditorManager { +public class PulsarLedgerAuditorManager implements LedgerAuditorManager { - private static final String ELECTION_PATH = "leader"; + public static final String ELECTION_PATH = "leader"; private final CoordinationService coordinationService; private final LeaderElection leaderElection; diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerManager.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerManager.java index 59452a3d54db3..b003c656353c0 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerManager.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/PulsarLedgerManager.java @@ -377,7 +377,7 @@ public void close() throws IOException { } } - private String getLedgerPath(long ledgerId) { + public String getLedgerPath(long ledgerId) { return this.ledgerRootPath + StringUtils.getHybridHierarchicalLedgerPath(ledgerId); } diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorBookieCheckTaskTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorBookieCheckTaskTest.java new file mode 100644 index 0000000000000..d30af4de6a803 --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorBookieCheckTaskTest.java @@ -0,0 +1,153 @@ +/* + * 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.replication; + +import static org.apache.bookkeeper.replication.ReplicationStats.AUDITOR_SCOPE; +import static org.mockito.ArgumentMatchers.anyCollection; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.testng.AssertJUnit.assertTrue; +import com.beust.jcommander.internal.Lists; +import com.beust.jcommander.internal.Sets; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.client.api.LedgerMetadata; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.stats.OpStatsLogger; +import org.apache.bookkeeper.test.TestStatsProvider; +import org.apache.bookkeeper.versioning.LongVersion; +import org.apache.bookkeeper.versioning.Versioned; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +/** + * Unit test {@link AuditorBookieCheckTask}. + */ +public class AuditorBookieCheckTaskTest { + + private AuditorStats auditorStats; + private BookKeeperAdmin admin; + private LedgerManager ledgerManager; + private LedgerUnderreplicationManager underreplicationManager; + private BookieLedgerIndexer ledgerIndexer; + private AuditorBookieCheckTask bookieCheckTask; + private final AtomicBoolean shutdownCompleted = new AtomicBoolean(false); + private final AuditorTask.ShutdownTaskHandler shutdownTaskHandler = () -> shutdownCompleted.set(true); + private long startLedgerId = 0; + + @BeforeMethod + public void setup() { + ServerConfiguration conf = mock(ServerConfiguration.class); + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsProvider.TestStatsLogger statsLogger = statsProvider.getStatsLogger(AUDITOR_SCOPE); + final AuditorStats auditorStats = new AuditorStats(statsLogger); + this.auditorStats = spy(auditorStats); + admin = mock(BookKeeperAdmin.class); + ledgerManager = mock(LedgerManager.class); + underreplicationManager = mock(LedgerUnderreplicationManager.class); + ledgerIndexer = mock(BookieLedgerIndexer.class); + AuditorBookieCheckTask bookieCheckTask1 = new AuditorBookieCheckTask( + conf, this.auditorStats, admin, ledgerManager, underreplicationManager, + shutdownTaskHandler, ledgerIndexer, null, null); + bookieCheckTask = spy(bookieCheckTask1); + } + + @Test + public void testShutdownAuditBookiesException() + throws BKException, ReplicationException.BKAuditException, InterruptedException { + doThrow(new ReplicationException.BKAuditException("test failed")) + .when(bookieCheckTask) + .auditBookies(); + bookieCheckTask.startAudit(true); + + assertTrue("shutdownTaskHandler should be execute.", shutdownCompleted.get()); + } + + @Test + public void testAuditBookies() + throws ReplicationException.UnavailableException, ReplicationException.BKAuditException, BKException { + final String bookieId1 = "127.0.0.1:1000"; + final String bookieId2 = "127.0.0.1:1001"; + final long bookie1LedgersCount = 10; + final long bookie2LedgersCount = 20; + + final Map> bookiesAndLedgers = new HashMap<>(); + bookiesAndLedgers.put(bookieId1, getLedgers(bookie1LedgersCount)); + bookiesAndLedgers.put(bookieId2, getLedgers(bookie2LedgersCount)); + when(ledgerIndexer.getBookieToLedgerIndex()).thenReturn(bookiesAndLedgers); + when(underreplicationManager.isLedgerReplicationEnabled()).thenReturn(true); + + CompletableFuture> metaPromise = new CompletableFuture<>(); + final LongVersion version = mock(LongVersion.class); + final LedgerMetadata metadata = mock(LedgerMetadata.class); + metaPromise.complete(new Versioned<>(metadata, version)); + when(ledgerManager.readLedgerMetadata(anyLong())).thenReturn(metaPromise); + + CompletableFuture markPromise = new CompletableFuture<>(); + markPromise.complete(null); + when(underreplicationManager.markLedgerUnderreplicatedAsync(anyLong(), anyCollection())) + .thenReturn(markPromise); + + OpStatsLogger numUnderReplicatedLedgerStats = mock(OpStatsLogger.class); + when(auditorStats.getNumUnderReplicatedLedger()).thenReturn(numUnderReplicatedLedgerStats); + + final List availableBookies = Lists.newArrayList(); + final List readOnlyBookies = Lists.newArrayList(); + // test bookie1 lost + availableBookies.add(BookieId.parse(bookieId2)); + when(admin.getAvailableBookies()).thenReturn(availableBookies); + when(admin.getReadOnlyBookies()).thenReturn(readOnlyBookies); + bookieCheckTask.startAudit(true); + verify(numUnderReplicatedLedgerStats, times(1)) + .registerSuccessfulValue(eq(bookie1LedgersCount)); + + // test bookie2 lost + numUnderReplicatedLedgerStats = mock(OpStatsLogger.class); + when(auditorStats.getNumUnderReplicatedLedger()).thenReturn(numUnderReplicatedLedgerStats); + availableBookies.clear(); + availableBookies.add(BookieId.parse(bookieId1)); + bookieCheckTask.startAudit(true); + verify(numUnderReplicatedLedgerStats, times(1)) + .registerSuccessfulValue(eq(bookie2LedgersCount)); + + } + + private Set getLedgers(long count) { + final Set ledgers = Sets.newHashSet(); + for (int i = 0; i < count; i++) { + ledgers.add(i + startLedgerId++); + } + return ledgers; + } +} diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java new file mode 100644 index 0000000000000..9750fb52d41a3 --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java @@ -0,0 +1,299 @@ +/* + * 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.replication; + +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertFalse; +import static org.testng.AssertJUnit.assertNotNull; +import static org.testng.AssertJUnit.assertNotSame; +import static org.testng.AssertJUnit.assertSame; +import static org.testng.AssertJUnit.assertTrue; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.proto.BookieServer; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.pulsar.metadata.bookkeeper.PulsarLedgerAuditorManager; +import org.apache.zookeeper.ZooKeeper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +/** + * This test verifies the auditor bookie scenarios which will be monitoring the + * bookie failures. + */ +public class AuditorBookieTest extends BookKeeperClusterTestCase { + // Depending on the taste, select the amount of logging + // by decommenting one of the two lines below + // private static final Logger LOG = Logger.getRootLogger(); + private static final Logger LOG = LoggerFactory + .getLogger(AuditorBookieTest.class); + private String electionPath; + private HashMap auditorElectors = new HashMap(); + private List zkClients = new LinkedList(); + + public AuditorBookieTest() throws Exception { + super(6); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataClientDriver"); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataBookieDriver"); + + } + + @BeforeMethod + @Override + public void setUp() throws Exception { + super.setUp(); + electionPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(baseConf) + + "/underreplication/" + PulsarLedgerAuditorManager.ELECTION_PATH; + baseConf.setMetadataServiceUri( + zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + startAuditorElectors(); + } + + @AfterMethod + @Override + public void tearDown() throws Exception { + stopAuditorElectors(); + for (ZooKeeper zk : zkClients) { + zk.close(); + } + zkClients.clear(); + super.tearDown(); + } + + /** + * Test should ensure only one should act as Auditor. Starting/shutdown + * other than auditor bookie shouldn't initiate re-election and multiple + * auditors. + */ + @Test + public void testEnsureOnlySingleAuditor() throws Exception { + BookieServer auditor = verifyAuditor(); + + // shutdown bookie which is not an auditor + int indexOf = indexOfServer(auditor); + int bkIndexDownBookie; + if (indexOf < lastBookieIndex()) { + bkIndexDownBookie = indexOf + 1; + } else { + bkIndexDownBookie = indexOf - 1; + } + shutdownBookie(serverByIndex(bkIndexDownBookie)); + + startNewBookie(); + startNewBookie(); + // grace period for the auditor re-election if any + BookieServer newAuditor = waitForNewAuditor(auditor); + assertSame( + "Auditor re-election is not happened for auditor failure!", + auditor, newAuditor); + } + + /** + * Test Auditor crashes should trigger re-election and another bookie should + * take over the auditor ship. + */ + @Test + public void testSuccessiveAuditorCrashes() throws Exception { + BookieServer auditor = verifyAuditor(); + shutdownBookie(auditor); + + BookieServer newAuditor1 = waitForNewAuditor(auditor); + shutdownBookie(newAuditor1); + BookieServer newAuditor2 = waitForNewAuditor(newAuditor1); + assertNotSame( + "Auditor re-election is not happened for auditor failure!", + auditor, newAuditor2); + } + + /** + * Test restarting the entire bookie cluster. It shouldn't create multiple + * bookie auditors. + */ + @Test + public void testBookieClusterRestart() throws Exception { + BookieServer auditor = verifyAuditor(); + for (AuditorElector auditorElector : auditorElectors.values()) { + assertTrue("Auditor elector is not running!", auditorElector + .isRunning()); + } + stopBKCluster(); + stopAuditorElectors(); + + startBKCluster(zkUtil.getMetadataServiceUri()); + //startBKCluster(zkUtil.getMetadataServiceUri()) override the base conf metadataServiceUri + baseConf.setMetadataServiceUri( + zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + startAuditorElectors(); + BookieServer newAuditor = waitForNewAuditor(auditor); + assertNotSame( + "Auditor re-election is not happened for auditor failure!", + auditor, newAuditor); + } + + /** + * Test the vote is deleting from the ZooKeeper during shutdown. + */ + @Test + public void testShutdown() throws Exception { + BookieServer auditor = verifyAuditor(); + shutdownBookie(auditor); + + // waiting for new auditor + BookieServer newAuditor = waitForNewAuditor(auditor); + assertNotSame( + "Auditor re-election is not happened for auditor failure!", + auditor, newAuditor); + + List children = zkc.getChildren(electionPath, false); + for (String child : children) { + byte[] data = zkc.getData(electionPath + '/' + child, false, null); + String bookieIP = new String(data); + String addr = auditor.getBookieId().toString(); + assertFalse("AuditorElection cleanup fails", bookieIP + .contains(addr)); + } + } + + /** + * Test restart of the previous Auditor bookie shouldn't initiate + * re-election and should create new vote after restarting. + */ + @Test + public void testRestartAuditorBookieAfterCrashing() throws Exception { + BookieServer auditor = verifyAuditor(); + + String addr = auditor.getBookieId().toString(); + + // restarting Bookie with same configurations. + ServerConfiguration serverConfiguration = shutdownBookie(auditor); + + auditorElectors.remove(addr); + startBookie(serverConfiguration); + // starting corresponding auditor elector + + if (LOG.isDebugEnabled()) { + LOG.debug("Performing Auditor Election:" + addr); + } + startAuditorElector(addr); + + // waiting for new auditor to come + BookieServer newAuditor = waitForNewAuditor(auditor); + assertNotSame( + "Auditor re-election is not happened for auditor failure!", + auditor, newAuditor); + assertFalse("No relection after old auditor rejoins", auditor + .getBookieId().equals(newAuditor.getBookieId())); + } + + private void startAuditorElector(String addr) throws Exception { + AuditorElector auditorElector = new AuditorElector(addr, + baseConf); + auditorElectors.put(addr, auditorElector); + auditorElector.start(); + if (LOG.isDebugEnabled()) { + LOG.debug("Starting Auditor Elector"); + } + } + + private void startAuditorElectors() throws Exception { + for (BookieId addr : bookieAddresses()) { + startAuditorElector(addr.toString()); + } + } + + private void stopAuditorElectors() throws Exception { + for (AuditorElector auditorElector : auditorElectors.values()) { + auditorElector.shutdown(); + if (LOG.isDebugEnabled()) { + LOG.debug("Stopping Auditor Elector!"); + } + } + } + + private BookieServer verifyAuditor() throws Exception { + List auditors = getAuditorBookie(); + assertEquals("Multiple Bookies acting as Auditor!", 1, auditors + .size()); + if (LOG.isDebugEnabled()) { + LOG.debug("Bookie running as Auditor:" + auditors.get(0)); + } + return auditors.get(0); + } + + private List getAuditorBookie() throws Exception { + List auditors = new LinkedList(); + byte[] data = zkc.getData(electionPath, false, null); + assertNotNull("Auditor election failed", data); + for (int i = 0; i < bookieCount(); i++) { + BookieServer bks = serverByIndex(i); + if (new String(data).contains(bks.getBookieId() + "")) { + auditors.add(bks); + } + } + return auditors; + } + + private ServerConfiguration shutdownBookie(BookieServer bkServer) throws Exception { + int index = indexOfServer(bkServer); + String addr = addressByIndex(index).toString(); + if (LOG.isDebugEnabled()) { + LOG.debug("Shutting down bookie:" + addr); + } + + // shutdown bookie which is an auditor + ServerConfiguration conf = killBookie(index); + + // stopping corresponding auditor elector + auditorElectors.get(addr).shutdown(); + return conf; + } + + private BookieServer waitForNewAuditor(BookieServer auditor) + throws Exception { + BookieServer newAuditor = null; + int retryCount = 8; + while (retryCount > 0) { + try { + List auditors = getAuditorBookie(); + if (auditors.size() > 0) { + newAuditor = auditors.get(0); + if (auditor != newAuditor) { + break; + } + } + } catch (Exception ignore) { + } + + Thread.sleep(500); + retryCount--; + } + assertNotNull( + "New Auditor is not reelected after auditor crashes", + newAuditor); + verifyAuditor(); + return newAuditor; + } +} diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorCheckAllLedgersTaskTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorCheckAllLedgersTaskTest.java new file mode 100644 index 0000000000000..5c0a3f39325e5 --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorCheckAllLedgersTaskTest.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.bookkeeper.replication; + +import static org.apache.bookkeeper.replication.ReplicationStats.AUDITOR_SCOPE; +import static org.testng.AssertJUnit.assertEquals; +import java.util.LinkedList; +import java.util.List; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.meta.LayoutManager; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.bookkeeper.test.TestStatsProvider; +import org.apache.pulsar.metadata.api.MetadataStoreConfig; +import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; +import org.apache.pulsar.metadata.bookkeeper.PulsarLayoutManager; +import org.apache.pulsar.metadata.bookkeeper.PulsarLedgerManagerFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +/** + * Unit test {@link AuditorCheckAllLedgersTask}. + */ +public class AuditorCheckAllLedgersTaskTest extends BookKeeperClusterTestCase { + private static final Logger LOG = LoggerFactory + .getLogger(AuditorCheckAllLedgersTaskTest.class); + + private static final int maxNumberOfConcurrentOpenLedgerOperations = 500; + private static final int acquireConcurrentOpenLedgerOperationsTimeoutMSec = 120000; + + private BookKeeperAdmin admin; + private LedgerManager ledgerManager; + private LedgerUnderreplicationManager ledgerUnderreplicationManager; + + public AuditorCheckAllLedgersTaskTest() { + super(3); + baseConf.setPageLimit(1); + baseConf.setAutoRecoveryDaemonEnabled(false); + } + + @BeforeMethod + @Override + public void setUp() throws Exception { + super.setUp(); + final BookKeeper bookKeeper = new BookKeeper(baseClientConf); + admin = new BookKeeperAdmin(bookKeeper, NullStatsLogger.INSTANCE, new ClientConfiguration(baseClientConf)); + + String ledgersRoot = "/ledgers"; + String storeUri = metadataServiceUri.replaceAll("zk://", "").replaceAll("/ledgers", ""); + MetadataStoreExtended store = MetadataStoreExtended.create(storeUri, + MetadataStoreConfig.builder().fsyncEnable(false).build()); + LayoutManager layoutManager = new PulsarLayoutManager(store, ledgersRoot); + PulsarLedgerManagerFactory ledgerManagerFactory = new PulsarLedgerManagerFactory(); + + ClientConfiguration conf = new ClientConfiguration(); + conf.setZkLedgersRootPath(ledgersRoot); + ledgerManagerFactory.initialize(conf, layoutManager, 1); + ledgerUnderreplicationManager = ledgerManagerFactory.newLedgerUnderreplicationManager(); + ledgerManager = ledgerManagerFactory.newLedgerManager(); + + baseConf.setAuditorMaxNumberOfConcurrentOpenLedgerOperations(maxNumberOfConcurrentOpenLedgerOperations); + baseConf.setAuditorAcquireConcurrentOpenLedgerOperationsTimeoutMSec( + acquireConcurrentOpenLedgerOperationsTimeoutMSec); + } + + @AfterMethod + @Override + public void tearDown() throws Exception { + if (ledgerManager != null) { + ledgerManager.close(); + } + if (ledgerUnderreplicationManager != null) { + ledgerUnderreplicationManager.close(); + } + if (admin != null) { + admin.close(); + } + super.tearDown(); + } + + @Test + public void testCheckAllLedgers() throws Exception { + // 1. create ledgers + final int numLedgers = 10; + List ids = new LinkedList(); + for (int i = 0; i < numLedgers; i++) { + LedgerHandle lh = bkc.createLedger(3, 3, BookKeeper.DigestType.CRC32, "passwd".getBytes()); + ids.add(lh.getId()); + for (int j = 0; j < 2; j++) { + lh.addEntry("testdata".getBytes()); + } + lh.close(); + } + // 2. init CheckAllLedgersTask + final TestStatsProvider statsProvider = new TestStatsProvider(); + final TestStatsProvider.TestStatsLogger statsLogger = statsProvider.getStatsLogger(AUDITOR_SCOPE); + final AuditorStats auditorStats = new AuditorStats(statsLogger); + + AuditorCheckAllLedgersTask auditorCheckAllLedgersTask = new AuditorCheckAllLedgersTask( + baseConf, auditorStats, admin, ledgerManager, + ledgerUnderreplicationManager, null, (flag, throwable) -> flag.set(false)); + + // 3. checkAllLedgers + auditorCheckAllLedgersTask.runTask(); + + // 4. verify + assertEquals("CHECK_ALL_LEDGERS_TIME", 1, ((TestStatsProvider.TestOpStatsLogger) statsLogger + .getOpStatsLogger(ReplicationStats.CHECK_ALL_LEDGERS_TIME)).getSuccessCount()); + assertEquals("NUM_LEDGERS_CHECKED", numLedgers, + (long) statsLogger.getCounter(ReplicationStats.NUM_LEDGERS_CHECKED).get()); + } +} diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java index d394e4ae7d1d5..ffd71f9311f42 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java @@ -23,36 +23,63 @@ import static org.testng.AssertJUnit.assertNotNull; import static org.testng.AssertJUnit.assertNotSame; import static org.testng.AssertJUnit.assertTrue; -import java.io.IOException; +import static org.testng.AssertJUnit.fail; +import java.net.URI; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Random; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; +import lombok.Cleanup; +import org.apache.bookkeeper.bookie.BookieImpl; import org.apache.bookkeeper.client.AsyncCallback.AddCallback; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper.DigestType; import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.client.LedgerMetadataBuilder; +import org.apache.bookkeeper.client.api.LedgerMetadata; +import org.apache.bookkeeper.common.util.OrderedScheduler; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.LayoutManager; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.meta.MetadataClientDriver; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.meta.UnderreplicatedLedger; import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.net.BookieSocketAddress; import org.apache.bookkeeper.proto.BookieServer; -import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; +import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; +import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.pulsar.metadata.api.MetadataStoreConfig; +import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; +import org.apache.pulsar.metadata.bookkeeper.PulsarLayoutManager; +import org.apache.pulsar.metadata.bookkeeper.PulsarLedgerAuditorManager; +import org.apache.pulsar.metadata.bookkeeper.PulsarLedgerManagerFactory; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -75,26 +102,26 @@ public class AuditorLedgerCheckerTest extends BookKeeperClusterTestCase { private String underreplicatedPath; private Map auditorElectors = new ConcurrentHashMap<>(); - private ZkLedgerUnderreplicationManager urLedgerMgr; + private LedgerUnderreplicationManager urLedgerMgr; + private Set urLedgerList; private String electionPath; private List ledgerList; public AuditorLedgerCheckerTest() - throws IOException, KeeperException, InterruptedException, - CompatibilityException { - this("org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory"); + throws Exception { + this("org.apache.pulsar.metadata.bookkeeper.PulsarLedgerManagerFactory"); } AuditorLedgerCheckerTest(String ledgerManagerFactoryClass) - throws IOException, KeeperException, InterruptedException, - CompatibilityException { + throws Exception { super(3); LOG.info("Running test case using ledger manager : " + ledgerManagerFactoryClass); this.digestType = DigestType.CRC32; - // set ledger manager name + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataClientDriver"); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataBookieDriver"); // set ledger manager name baseConf.setLedgerManagerFactoryClassName(ledgerManagerFactoryClass); baseClientConf .setLedgerManagerFactoryClassName(ledgerManagerFactoryClass); @@ -106,18 +133,31 @@ public void setUp() throws Exception { underreplicatedPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(baseClientConf) + "/underreplication/ledgers"; electionPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(baseConf) - + "/underreplication/auditorelection"; + + "/underreplication/" + PulsarLedgerAuditorManager.ELECTION_PATH; - urLedgerMgr = new ZkLedgerUnderreplicationManager(baseClientConf, zkc); + String ledgersRoot = "/ledgers"; + String storeUri = metadataServiceUri.replaceAll("zk://", "").replaceAll("/ledgers", ""); + MetadataStoreExtended store = MetadataStoreExtended.create(storeUri, + MetadataStoreConfig.builder().fsyncEnable(false).build()); + LayoutManager layoutManager = new PulsarLayoutManager(store, ledgersRoot); + PulsarLedgerManagerFactory ledgerManagerFactory = new PulsarLedgerManagerFactory(); + ClientConfiguration conf = new ClientConfiguration(); + conf.setZkLedgersRootPath(ledgersRoot); + ledgerManagerFactory.initialize(conf, layoutManager, 1); + urLedgerMgr = ledgerManagerFactory.newLedgerUnderreplicationManager(); urLedgerMgr.setCheckAllLedgersCTime(System.currentTimeMillis()); + + baseClientConf.setMetadataServiceUri( + zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + baseConf.setMetadataServiceUri( + zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); startAuditorElectors(); rng = new Random(System.currentTimeMillis()); // Initialize the Random urLedgerList = new HashSet(); ledgerList = new ArrayList(2); - baseClientConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri()); - baseConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri()); } + @AfterMethod @Override public void tearDown() throws Exception { stopAuditorElectors(); @@ -145,6 +185,515 @@ private void stopAuditorElectors() throws Exception { } } + /** + * Test publishing of under replicated ledgers by the auditor bookie. + */ + @Test + public void testSimpleLedger() throws Exception { + LedgerHandle lh1 = createAndAddEntriesToLedger(); + Long ledgerId = lh1.getId(); + if (LOG.isDebugEnabled()) { + LOG.debug("Created ledger : " + ledgerId); + } + ledgerList.add(ledgerId); + lh1.close(); + + final CountDownLatch underReplicaLatch = registerUrLedgerWatcher(ledgerList + .size()); + + int bkShutdownIndex = lastBookieIndex(); + String shutdownBookie = shutdownBookie(bkShutdownIndex); + + // grace period for publishing the bk-ledger + if (LOG.isDebugEnabled()) { + LOG.debug("Waiting for ledgers to be marked as under replicated"); + } + waitForAuditToComplete(); + underReplicaLatch.await(5, TimeUnit.SECONDS); + Map urLedgerData = getUrLedgerData(urLedgerList); + assertEquals("Missed identifying under replicated ledgers", 1, + urLedgerList.size()); + + /* + * Sample data format present in the under replicated ledger path + * + * {4=replica: "10.18.89.153:5002"} + */ + assertTrue("Ledger is not marked as underreplicated:" + ledgerId, + urLedgerList.contains(ledgerId)); + String data = urLedgerData.get(ledgerId); + assertTrue("Bookie " + shutdownBookie + + "is not listed in the ledger as missing replica :" + data, + data.contains(shutdownBookie)); + } + + /** + * Test once published under replicated ledger should exists even after + * restarting respective bookie. + */ + @Test + public void testRestartBookie() throws Exception { + LedgerHandle lh1 = createAndAddEntriesToLedger(); + LedgerHandle lh2 = createAndAddEntriesToLedger(); + + if (LOG.isDebugEnabled()) { + LOG.debug("Created following ledgers : {}, {}", lh1, lh2); + } + + int bkShutdownIndex = lastBookieIndex(); + ServerConfiguration bookieConf1 = confByIndex(bkShutdownIndex); + String shutdownBookie = shutdownBookie(bkShutdownIndex); + + // restart the failed bookie + startAndAddBookie(bookieConf1); + + waitForLedgerMissingReplicas(lh1.getId(), 10, shutdownBookie); + waitForLedgerMissingReplicas(lh2.getId(), 10, shutdownBookie); + } + + /** + * Test publishing of under replicated ledgers when multiple bookie failures + * one after another. + */ + @Test + public void testMultipleBookieFailures() throws Exception { + LedgerHandle lh1 = createAndAddEntriesToLedger(); + + // failing first bookie + shutdownBookie(lastBookieIndex()); + + // simulate re-replication + doLedgerRereplication(lh1.getId()); + + // failing another bookie + String shutdownBookie = shutdownBookie(lastBookieIndex()); + + // grace period for publishing the bk-ledger + if (LOG.isDebugEnabled()) { + LOG.debug("Waiting for ledgers to be marked as under replicated"); + } + assertTrue("Ledger should be missing second replica", + waitForLedgerMissingReplicas(lh1.getId(), 10, shutdownBookie)); + } + + @Test + public void testToggleLedgerReplication() throws Exception { + LedgerHandle lh1 = createAndAddEntriesToLedger(); + ledgerList.add(lh1.getId()); + if (LOG.isDebugEnabled()) { + LOG.debug("Created following ledgers : " + ledgerList); + } + + // failing another bookie + CountDownLatch urReplicaLatch = registerUrLedgerWatcher(ledgerList + .size()); + + // disabling ledger replication + urLedgerMgr.disableLedgerReplication(); + ArrayList shutdownBookieList = new ArrayList(); + shutdownBookieList.add(shutdownBookie(lastBookieIndex())); + shutdownBookieList.add(shutdownBookie(lastBookieIndex())); + + assertFalse("Ledger replication is not disabled!", urReplicaLatch + .await(1, TimeUnit.SECONDS)); + + // enabling ledger replication + urLedgerMgr.enableLedgerReplication(); + assertTrue("Ledger replication is not enabled!", urReplicaLatch.await( + 5, TimeUnit.SECONDS)); + } + + @Test + public void testDuplicateEnDisableAutoRecovery() throws Exception { + urLedgerMgr.disableLedgerReplication(); + try { + urLedgerMgr.disableLedgerReplication(); + fail("Must throw exception, since AutoRecovery is already disabled"); + } catch (UnavailableException e) { + assertTrue("AutoRecovery is not disabled previously!", + e.getCause().getCause() instanceof MetadataStoreException.BadVersionException); + } + urLedgerMgr.enableLedgerReplication(); + try { + urLedgerMgr.enableLedgerReplication(); + fail("Must throw exception, since AutoRecovery is already enabled"); + } catch (UnavailableException e) { + assertTrue("AutoRecovery is not enabled previously!", + e.getCause().getCause() instanceof MetadataStoreException.NotFoundException); + } + } + + /** + * Test Auditor should consider Readonly bookie as available bookie. Should not publish ur ledgers for + * readonly bookies. + */ + @Test + public void testReadOnlyBookieExclusionFromURLedgersCheck() throws Exception { + LedgerHandle lh = createAndAddEntriesToLedger(); + ledgerList.add(lh.getId()); + if (LOG.isDebugEnabled()) { + LOG.debug("Created following ledgers : " + ledgerList); + } + + int count = ledgerList.size(); + final CountDownLatch underReplicaLatch = registerUrLedgerWatcher(count); + + final int bkIndex = 2; + ServerConfiguration bookieConf = confByIndex(bkIndex); + BookieServer bk = serverByIndex(bkIndex); + bookieConf.setReadOnlyModeEnabled(true); + + ((BookieImpl) bk.getBookie()).getStateManager().doTransitionToReadOnlyMode(); + bkc.waitForReadOnlyBookie(BookieImpl.getBookieId(confByIndex(bkIndex))) + .get(30, TimeUnit.SECONDS); + + // grace period for publishing the bk-ledger + if (LOG.isDebugEnabled()) { + LOG.debug("Waiting for Auditor to finish ledger check."); + } + waitForAuditToComplete(); + assertFalse("latch should not have completed", underReplicaLatch.await(5, TimeUnit.SECONDS)); + } + + /** + * Test Auditor should consider Readonly bookie fail and publish ur ledgers for readonly bookies. + */ + @Test + public void testReadOnlyBookieShutdown() throws Exception { + LedgerHandle lh = createAndAddEntriesToLedger(); + long ledgerId = lh.getId(); + ledgerList.add(ledgerId); + if (LOG.isDebugEnabled()) { + LOG.debug("Created following ledgers : " + ledgerList); + } + + int count = ledgerList.size(); + final CountDownLatch underReplicaLatch = registerUrLedgerWatcher(count); + + int bkIndex = lastBookieIndex(); + if (LOG.isDebugEnabled()) { + LOG.debug("Moving bookie {} {} to read only...", bkIndex, serverByIndex(bkIndex)); + } + ServerConfiguration bookieConf = confByIndex(bkIndex); + BookieServer bk = serverByIndex(bkIndex); + bookieConf.setReadOnlyModeEnabled(true); + + ((BookieImpl) bk.getBookie()).getStateManager().doTransitionToReadOnlyMode(); + bkc.waitForReadOnlyBookie(BookieImpl.getBookieId(confByIndex(bkIndex))) + .get(30, TimeUnit.SECONDS); + + // grace period for publishing the bk-ledger + if (LOG.isDebugEnabled()) { + LOG.debug("Waiting for Auditor to finish ledger check."); + } + waitForAuditToComplete(); + assertFalse("latch should not have completed", underReplicaLatch.await(1, TimeUnit.SECONDS)); + + String shutdownBookie = shutdownBookie(bkIndex); + + // grace period for publishing the bk-ledger + if (LOG.isDebugEnabled()) { + LOG.debug("Waiting for ledgers to be marked as under replicated"); + } + waitForAuditToComplete(); + underReplicaLatch.await(5, TimeUnit.SECONDS); + Map urLedgerData = getUrLedgerData(urLedgerList); + assertEquals("Missed identifying under replicated ledgers", 1, urLedgerList.size()); + + /* + * Sample data format present in the under replicated ledger path + * + * {4=replica: "10.18.89.153:5002"} + */ + assertTrue("Ledger is not marked as underreplicated:" + ledgerId, urLedgerList.contains(ledgerId)); + String data = urLedgerData.get(ledgerId); + assertTrue("Bookie " + shutdownBookie + "is not listed in the ledger as missing replica :" + data, + data.contains(shutdownBookie)); + } + + public void testInnerDelayedAuditOfLostBookies() throws Exception { + LedgerHandle lh1 = createAndAddEntriesToLedger(); + Long ledgerId = lh1.getId(); + if (LOG.isDebugEnabled()) { + LOG.debug("Created ledger : " + ledgerId); + } + ledgerList.add(ledgerId); + lh1.close(); + + final CountDownLatch underReplicaLatch = registerUrLedgerWatcher(ledgerList + .size()); + + // wait for 5 seconds before starting the recovery work when a bookie fails + urLedgerMgr.setLostBookieRecoveryDelay(5); + + // shutdown a non auditor bookie; choosing non-auditor to avoid another election + String shutdownBookie = shutDownNonAuditorBookie(); + + if (LOG.isDebugEnabled()) { + LOG.debug("Waiting for ledgers to be marked as under replicated"); + } + assertFalse("audit of lost bookie isn't delayed", underReplicaLatch.await(4, TimeUnit.SECONDS)); + assertEquals("under replicated ledgers identified when it was not expected", 0, + urLedgerList.size()); + + // wait for another 5 seconds for the ledger to get reported as under replicated + assertTrue("audit of lost bookie isn't delayed", underReplicaLatch.await(2, TimeUnit.SECONDS)); + + assertTrue("Ledger is not marked as underreplicated:" + ledgerId, + urLedgerList.contains(ledgerId)); + Map urLedgerData = getUrLedgerData(urLedgerList); + String data = urLedgerData.get(ledgerId); + assertTrue("Bookie " + shutdownBookie + + "is not listed in the ledger as missing replica :" + data, + data.contains(shutdownBookie)); + } + + /** + * Test publishing of under replicated ledgers by the auditor + * bookie is delayed if LostBookieRecoveryDelay option is set. + */ + @Test + public void testDelayedAuditOfLostBookies() throws Exception { + // wait for a second so that the initial periodic check finishes + Thread.sleep(1000); + + testInnerDelayedAuditOfLostBookies(); + } + + /** + * Test publishing of under replicated ledgers by the auditor + * bookie is delayed if LostBookieRecoveryDelay option is set + * and it continues to be delayed even when periodic bookie check + * is set to run every 2 secs. I.e. periodic bookie check doesn't + * override the delay + */ + @Test + public void testDelayedAuditWithPeriodicBookieCheck() throws Exception { + // enable periodic bookie check on a cadence of every 2 seconds. + // this requires us to stop the auditor/auditorElectors, set the + // periodic check interval and restart the auditorElectors + stopAuditorElectors(); + baseConf.setAuditorPeriodicBookieCheckInterval(2); + startAuditorElectors(); + + // wait for a second so that the initial periodic check finishes + Thread.sleep(1000); + + // the delaying of audit should just work despite the fact + // we have enabled periodic bookie check + testInnerDelayedAuditOfLostBookies(); + } + + @Test + public void testRescheduleOfDelayedAuditOfLostBookiesToStartImmediately() throws Exception { + // wait for a second so that the initial periodic check finishes + Thread.sleep(1000); + + LedgerHandle lh1 = createAndAddEntriesToLedger(); + Long ledgerId = lh1.getId(); + if (LOG.isDebugEnabled()) { + LOG.debug("Created ledger : " + ledgerId); + } + ledgerList.add(ledgerId); + lh1.close(); + + final CountDownLatch underReplicaLatch = registerUrLedgerWatcher(ledgerList + .size()); + + // wait for 50 seconds before starting the recovery work when a bookie fails + urLedgerMgr.setLostBookieRecoveryDelay(50); + + // shutdown a non auditor bookie; choosing non-auditor to avoid another election + String shutdownBookie = shutDownNonAuditorBookie(); + + if (LOG.isDebugEnabled()) { + LOG.debug("Waiting for ledgers to be marked as under replicated"); + } + assertFalse("audit of lost bookie isn't delayed", underReplicaLatch.await(4, TimeUnit.SECONDS)); + assertEquals("under replicated ledgers identified when it was not expected", 0, + urLedgerList.size()); + + // set lostBookieRecoveryDelay to 0, so that it triggers AuditTask immediately + urLedgerMgr.setLostBookieRecoveryDelay(0); + + // wait for 1 second for the ledger to get reported as under replicated + assertTrue("audit of lost bookie isn't delayed", underReplicaLatch.await(1, TimeUnit.SECONDS)); + + assertTrue("Ledger is not marked as underreplicated:" + ledgerId, + urLedgerList.contains(ledgerId)); + Map urLedgerData = getUrLedgerData(urLedgerList); + String data = urLedgerData.get(ledgerId); + assertTrue("Bookie " + shutdownBookie + + "is not listed in the ledger as missing replica :" + data, + data.contains(shutdownBookie)); + } + + @Test + public void testRescheduleOfDelayedAuditOfLostBookiesToStartLater() throws Exception { + // wait for a second so that the initial periodic check finishes + Thread.sleep(1000); + + LedgerHandle lh1 = createAndAddEntriesToLedger(); + Long ledgerId = lh1.getId(); + if (LOG.isDebugEnabled()) { + LOG.debug("Created ledger : " + ledgerId); + } + ledgerList.add(ledgerId); + lh1.close(); + + final CountDownLatch underReplicaLatch = registerUrLedgerWatcher(ledgerList + .size()); + + // wait for 3 seconds before starting the recovery work when a bookie fails + urLedgerMgr.setLostBookieRecoveryDelay(3); + + // shutdown a non auditor bookie; choosing non-auditor to avoid another election + String shutdownBookie = shutDownNonAuditorBookie(); + + if (LOG.isDebugEnabled()) { + LOG.debug("Waiting for ledgers to be marked as under replicated"); + } + assertFalse("audit of lost bookie isn't delayed", underReplicaLatch.await(2, TimeUnit.SECONDS)); + assertEquals("under replicated ledgers identified when it was not expected", 0, + urLedgerList.size()); + + // set lostBookieRecoveryDelay to 4, so the pending AuditTask is resheduled + urLedgerMgr.setLostBookieRecoveryDelay(4); + + // since we changed the BookieRecoveryDelay period to 4, the audittask shouldn't have been executed + if (LOG.isDebugEnabled()) { + LOG.debug("Waiting for ledgers to be marked as under replicated"); + } + assertFalse("audit of lost bookie isn't delayed", underReplicaLatch.await(2, TimeUnit.SECONDS)); + assertEquals("under replicated ledgers identified when it was not expected", 0, + urLedgerList.size()); + + // wait for 3 seconds (since we already waited for 2 secs) for the ledger to get reported as under replicated + assertTrue("audit of lost bookie isn't delayed", underReplicaLatch.await(3, TimeUnit.SECONDS)); + assertTrue("Ledger is not marked as underreplicated:" + ledgerId, + urLedgerList.contains(ledgerId)); + Map urLedgerData = getUrLedgerData(urLedgerList); + String data = urLedgerData.get(ledgerId); + assertTrue("Bookie " + shutdownBookie + + "is not listed in the ledger as missing replica :" + data, + data.contains(shutdownBookie)); + } + + @Test + public void testTriggerAuditorWithNoPendingAuditTask() throws Exception { + // wait for a second so that the initial periodic check finishes + Thread.sleep(1000); + int lostBookieRecoveryDelayConfValue = baseConf.getLostBookieRecoveryDelay(); + Auditor auditorBookiesAuditor = getAuditorBookiesAuditor(); + Future auditTask = auditorBookiesAuditor.getAuditTask(); + int lostBookieRecoveryDelayBeforeChange = auditorBookiesAuditor.getLostBookieRecoveryDelayBeforeChange(); + assertEquals("auditTask is supposed to be null", null, auditTask); + assertEquals( + "lostBookieRecoveryDelayBeforeChange of Auditor should be equal to BaseConf's lostBookieRecoveryDelay", + lostBookieRecoveryDelayConfValue, lostBookieRecoveryDelayBeforeChange); + + @Cleanup("shutdown") OrderedScheduler scheduler = OrderedScheduler.newSchedulerBuilder() + .name("test-scheduler") + .numThreads(1) + .build(); + @Cleanup MetadataClientDriver driver = + MetadataDrivers.getClientDriver(URI.create(baseClientConf.getMetadataServiceUri())); + driver.initialize(baseClientConf, scheduler, NullStatsLogger.INSTANCE, Optional.of(zkc)); + + // there is no easy way to validate if the Auditor has executed Audit process (Auditor.startAudit), + // without shuttingdown Bookie. To test if by resetting LostBookieRecoveryDelay it does Auditing + // even when there is no pending AuditTask, following approach is needed. + + // Here we are creating few ledgers ledgermetadata with non-existing bookies as its ensemble. + // When Auditor does audit it recognizes these ledgers as underreplicated and mark them as + // under-replicated, since these bookies are not available. + int numofledgers = 5; + Random rand = new Random(); + for (int i = 0; i < numofledgers; i++) { + ArrayList ensemble = new ArrayList(); + ensemble.add(new BookieSocketAddress("99.99.99.99:9999").toBookieId()); + ensemble.add(new BookieSocketAddress("11.11.11.11:1111").toBookieId()); + ensemble.add(new BookieSocketAddress("88.88.88.88:8888").toBookieId()); + + long ledgerId = (Math.abs(rand.nextLong())) % 100000000; + + LedgerMetadata metadata = LedgerMetadataBuilder.create() + .withId(ledgerId) + .withEnsembleSize(3).withWriteQuorumSize(2).withAckQuorumSize(2) + .withPassword("passwd".getBytes()) + .withDigestType(DigestType.CRC32.toApiDigestType()) + .newEnsembleEntry(0L, ensemble).build(); + + try (LedgerManager lm = driver.getLedgerManagerFactory().newLedgerManager()) { + lm.createLedgerMetadata(ledgerId, metadata).get(2000, TimeUnit.MILLISECONDS); + } + ledgerList.add(ledgerId); + } + + final CountDownLatch underReplicaLatch = registerUrLedgerWatcher(ledgerList.size()); + urLedgerMgr.setLostBookieRecoveryDelay(lostBookieRecoveryDelayBeforeChange); + assertTrue("Audit should be triggered and created ledgers should be marked as underreplicated", + underReplicaLatch.await(2, TimeUnit.SECONDS)); + assertEquals("All the ledgers should be marked as underreplicated", ledgerList.size(), urLedgerList.size()); + + auditTask = auditorBookiesAuditor.getAuditTask(); + assertEquals("auditTask is supposed to be null", null, auditTask); + assertEquals( + "lostBookieRecoveryDelayBeforeChange of Auditor should be equal to BaseConf's lostBookieRecoveryDelay", + lostBookieRecoveryDelayBeforeChange, auditorBookiesAuditor.getLostBookieRecoveryDelayBeforeChange()); + } + + @Test + public void testTriggerAuditorWithPendingAuditTask() throws Exception { + // wait for a second so that the initial periodic check finishes + Thread.sleep(1000); + + Auditor auditorBookiesAuditor = getAuditorBookiesAuditor(); + LedgerHandle lh1 = createAndAddEntriesToLedger(); + Long ledgerId = lh1.getId(); + if (LOG.isDebugEnabled()) { + LOG.debug("Created ledger : " + ledgerId); + } + ledgerList.add(ledgerId); + lh1.close(); + + final CountDownLatch underReplicaLatch = registerUrLedgerWatcher(ledgerList + .size()); + + int lostBookieRecoveryDelay = 5; + // wait for 5 seconds before starting the recovery work when a bookie fails + urLedgerMgr.setLostBookieRecoveryDelay(lostBookieRecoveryDelay); + + // shutdown a non auditor bookie; choosing non-auditor to avoid another election + String shutdownBookie = shutDownNonAuditorBookie(); + + if (LOG.isDebugEnabled()) { + LOG.debug("Waiting for ledgers to be marked as under replicated"); + } + assertFalse("audit of lost bookie isn't delayed", underReplicaLatch.await(2, TimeUnit.SECONDS)); + assertEquals("under replicated ledgers identified when it was not expected", 0, + urLedgerList.size()); + + Future auditTask = auditorBookiesAuditor.getAuditTask(); + assertNotSame("auditTask is not supposed to be null", null, auditTask); + assertEquals( + "lostBookieRecoveryDelayBeforeChange of Auditor should be equal to what we set", + lostBookieRecoveryDelay, auditorBookiesAuditor.getLostBookieRecoveryDelayBeforeChange()); + + // set lostBookieRecoveryDelay to 5 (previous value), so that Auditor is triggered immediately + urLedgerMgr.setLostBookieRecoveryDelay(lostBookieRecoveryDelay); + assertTrue("audit of lost bookie shouldn't be delayed", underReplicaLatch.await(2, TimeUnit.SECONDS)); + assertEquals("all under replicated ledgers should be identified", ledgerList.size(), + urLedgerList.size()); + + Thread.sleep(100); + auditTask = auditorBookiesAuditor.getAuditTask(); + assertEquals("auditTask is supposed to be null", null, auditTask); + assertEquals( + "lostBookieRecoveryDelayBeforeChange of Auditor should be equal to previously set value", + lostBookieRecoveryDelay, auditorBookiesAuditor.getLostBookieRecoveryDelayBeforeChange()); + } + @Test public void testTriggerAuditorBySettingDelayToZeroWithPendingAuditTask() throws Exception { // wait for a second so that the initial periodic check finishes @@ -196,6 +745,163 @@ public void testTriggerAuditorBySettingDelayToZeroWithPendingAuditTask() throws 0, auditorBookiesAuditor.getLostBookieRecoveryDelayBeforeChange()); } + /** + * Test audit of bookies is delayed when one bookie is down. But when + * another one goes down, the audit is started immediately. + */ + @Test + public void testDelayedAuditWithMultipleBookieFailures() throws Exception { + // wait for the periodic bookie check to finish + Thread.sleep(1000); + + // create a ledger with a bunch of entries + LedgerHandle lh1 = createAndAddEntriesToLedger(); + Long ledgerId = lh1.getId(); + if (LOG.isDebugEnabled()) { + LOG.debug("Created ledger : " + ledgerId); + } + ledgerList.add(ledgerId); + lh1.close(); + + CountDownLatch underReplicaLatch = registerUrLedgerWatcher(ledgerList.size()); + + // wait for 10 seconds before starting the recovery work when a bookie fails + urLedgerMgr.setLostBookieRecoveryDelay(10); + + // shutdown a non auditor bookie to avoid an election + String shutdownBookie1 = shutDownNonAuditorBookie(); + + // wait for 3 seconds and there shouldn't be any under replicated ledgers + // because we have delayed the start of audit by 10 seconds + assertFalse("audit of lost bookie isn't delayed", underReplicaLatch.await(3, TimeUnit.SECONDS)); + assertEquals("under replicated ledgers identified when it was not expected", 0, + urLedgerList.size()); + + // Now shutdown the second non auditor bookie; We want to make sure that + // the history about having delayed recovery remains. Hence we make sure + // we bring down a non auditor bookie. This should cause the audit to take + // place immediately and not wait for the remaining 7 seconds to elapse + String shutdownBookie2 = shutDownNonAuditorBookie(); + + // 2 second grace period for the ledgers to get reported as under replicated + Thread.sleep(2000); + + // If the following checks pass, it means that audit happened + // within 2 seconds of second bookie going down and it didn't + // wait for 7 more seconds. Hence the second bookie failure doesn't + // delay the audit + assertTrue("Ledger is not marked as underreplicated:" + ledgerId, + urLedgerList.contains(ledgerId)); + Map urLedgerData = getUrLedgerData(urLedgerList); + String data = urLedgerData.get(ledgerId); + assertTrue("Bookie " + shutdownBookie1 + shutdownBookie2 + + " are not listed in the ledger as missing replicas :" + data, + data.contains(shutdownBookie1) && data.contains(shutdownBookie2)); + } + + /** + * Test audit of bookies is delayed during rolling upgrade scenario: + * a bookies goes down and comes up, the next bookie go down and up and so on. + * At any time only one bookie is down. + */ + @Test + public void testDelayedAuditWithRollingUpgrade() throws Exception { + // wait for the periodic bookie check to finish + Thread.sleep(1000); + + // create a ledger with a bunch of entries + LedgerHandle lh1 = createAndAddEntriesToLedger(); + Long ledgerId = lh1.getId(); + if (LOG.isDebugEnabled()) { + LOG.debug("Created ledger : " + ledgerId); + } + ledgerList.add(ledgerId); + lh1.close(); + + CountDownLatch underReplicaLatch = registerUrLedgerWatcher(ledgerList.size()); + + // wait for 5 seconds before starting the recovery work when a bookie fails + urLedgerMgr.setLostBookieRecoveryDelay(5); + + // shutdown a non auditor bookie to avoid an election + int idx1 = getShutDownNonAuditorBookieIdx(""); + ServerConfiguration conf1 = confByIndex(idx1); + String shutdownBookie1 = shutdownBookie(idx1); + + // wait for 2 seconds and there shouldn't be any under replicated ledgers + // because we have delayed the start of audit by 5 seconds + assertFalse("audit of lost bookie isn't delayed", underReplicaLatch.await(2, TimeUnit.SECONDS)); + assertEquals("under replicated ledgers identified when it was not expected", 0, + urLedgerList.size()); + + // restart the bookie we shut down above + startAndAddBookie(conf1); + + // Now to simulate the rolling upgrade, bring down a bookie different from + // the one we brought down/up above. + String shutdownBookie2 = shutDownNonAuditorBookie(shutdownBookie1); + + // since the first bookie that was brought down/up has come up, there is only + // one bookie down at this time. Hence the lost bookie check shouldn't start + // immediately; it will start 5 seconds after the second bookie went down + assertFalse("audit of lost bookie isn't delayed", underReplicaLatch.await(2, TimeUnit.SECONDS)); + assertEquals("under replicated ledgers identified when it was not expected", 0, + urLedgerList.size()); + + // wait for a total of 6 seconds(2+4) for the ledgers to get reported as under replicated + Thread.sleep(4000); + + // If the following checks pass, it means that auditing happened + // after lostBookieRecoveryDelay during rolling upgrade as expected + assertTrue("Ledger is not marked as underreplicated:" + ledgerId, + urLedgerList.contains(ledgerId)); + Map urLedgerData = getUrLedgerData(urLedgerList); + String data = urLedgerData.get(ledgerId); + assertTrue("Bookie " + shutdownBookie1 + "wrongly listed as missing the ledger: " + data, + !data.contains(shutdownBookie1)); + assertTrue("Bookie " + shutdownBookie2 + + " is not listed in the ledger as missing replicas :" + data, + data.contains(shutdownBookie2)); + LOG.info("*****************Test Complete"); + } + + private void waitForAuditToComplete() throws Exception { + long endTime = System.currentTimeMillis() + 5_000; + while (System.currentTimeMillis() < endTime) { + Auditor auditor = getAuditorBookiesAuditor(); + if (auditor != null) { + Future task = auditor.submitAuditTask(); + task.get(5, TimeUnit.SECONDS); + return; + } + Thread.sleep(100); + } + throw new TimeoutException("Could not find an audit within 5 seconds"); + } + + /** + * Wait for ledger to be underreplicated, and to be missing all replicas specified. + */ + private boolean waitForLedgerMissingReplicas(Long ledgerId, long secondsToWait, String... replicas) + throws Exception { + for (int i = 0; i < secondsToWait; i++) { + try { + UnderreplicatedLedger data = urLedgerMgr.getLedgerUnreplicationInfo(ledgerId); + boolean all = true; + for (String r : replicas) { + all = all && data.getReplicaList().contains(r); + } + if (all) { + return true; + } + } catch (Exception e) { + // may not find node + } + Thread.sleep(1000); + } + return false; + } + private CountDownLatch registerUrLedgerWatcher(int count) throws KeeperException, InterruptedException { final CountDownLatch underReplicaLatch = new CountDownLatch(count); @@ -208,11 +914,21 @@ private CountDownLatch registerUrLedgerWatcher(int count) return underReplicaLatch; } + private void doLedgerRereplication(Long... ledgerIds) + throws UnavailableException { + for (int i = 0; i < ledgerIds.length; i++) { + long lid = urLedgerMgr.getLedgerToRereplicate(); + assertTrue("Received unexpected ledgerid", Arrays.asList(ledgerIds).contains(lid)); + urLedgerMgr.markLedgerReplicated(lid); + urLedgerMgr.releaseUnderreplicatedLedger(lid); + } + } + private String shutdownBookie(int bkShutdownIndex) throws Exception { BookieServer bkServer = serverByIndex(bkShutdownIndex); String bookieAddr = bkServer.getBookieId().toString(); - if (LOG.isDebugEnabled()) { - LOG.debug("Shutting down bookie:" + bookieAddr); + if (LOG.isInfoEnabled()) { + LOG.info("Shutting down bookie:" + bookieAddr); } killBookie(bkShutdownIndex); auditorElectors.get(bookieAddr).shutdown(); @@ -253,6 +969,18 @@ public void addComplete(int rc2, LedgerHandle lh, long entryId, Object ctx) { } + private Map getUrLedgerData(Set urLedgerList) + throws KeeperException, InterruptedException { + Map urLedgerData = new HashMap(); + for (Long ledgerId : urLedgerList) { + String znode = ZkLedgerUnderreplicationManager.getUrLedgerZnode(underreplicatedPath, + ledgerId); + byte[] data = zkc.getData(znode, false, null); + urLedgerData.put(ledgerId, new String(data)); + } + return urLedgerData; + } + private class ChildWatcher implements Watcher { private final CountDownLatch underReplicaLatch; @@ -298,7 +1026,7 @@ private Auditor getAuditorBookiesAuditor() throws Exception { return auditorElectors.get(bookieAddr).auditor; } - private String shutDownNonAuditorBookie() throws Exception { + private String shutDownNonAuditorBookie() throws Exception { // shutdown bookie which is not an auditor int indexOf = indexOfServer(getAuditorBookie()); int bkIndexDownBookie; @@ -309,4 +1037,22 @@ private String shutDownNonAuditorBookie() throws Exception { } return shutdownBookie(bkIndexDownBookie); } + + private int getShutDownNonAuditorBookieIdx(String exclude) throws Exception { + // shutdown bookie which is not an auditor + int indexOf = indexOfServer(getAuditorBookie()); + int bkIndexDownBookie = 0; + for (int i = 0; i <= lastBookieIndex(); i++) { + if (i == indexOf || addressByIndex(i).toString().equals(exclude)) { + continue; + } + bkIndexDownBookie = i; + break; + } + return bkIndexDownBookie; + } + + private String shutDownNonAuditorBookie(String exclude) throws Exception { + return shutdownBookie(getShutDownNonAuditorBookieIdx(exclude)); + } } diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java new file mode 100644 index 0000000000000..4acb207570a2d --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.bookkeeper.replication; + +import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithLedgerManagerFactory; +import static org.testng.AssertJUnit.assertEquals; +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.UncheckedExecutionException; +import lombok.Cleanup; +import org.apache.bookkeeper.client.ClientUtil; +import org.apache.bookkeeper.client.LedgerMetadataBuilder; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.conf.TestBKConfiguration; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +/** + * This test verifies that the period check on the auditor + * will pick up on missing data in the client. + */ +public class AuditorPeriodicBookieCheckTest extends BookKeeperClusterTestCase { + private static final Logger LOG = LoggerFactory + .getLogger(AuditorPeriodicBookieCheckTest.class); + + private AuditorElector auditorElector = null; + + private static final int CHECK_INTERVAL = 1; // run every second + + public AuditorPeriodicBookieCheckTest() throws Exception { + super(3); + baseConf.setPageLimit(1); // to make it easy to push ledger out of cache + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataClientDriver"); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataBookieDriver"); + } + + @BeforeMethod + @Override + public void setUp() throws Exception { + super.setUp(); + + ServerConfiguration conf = TestBKConfiguration.newServerConfiguration(); + conf.setAuditorPeriodicBookieCheckInterval(CHECK_INTERVAL); + + conf.setMetadataServiceUri( + metadataServiceUri.replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + conf.setProperty("clientConnectTimeoutMillis", 500); + String addr = addressByIndex(0).toString(); + + auditorElector = new AuditorElector(addr, conf); + auditorElector.start(); + } + + @AfterMethod + @Override + public void tearDown() throws Exception { + auditorElector.shutdown(); + super.tearDown(); + } + + /** + * Test that the periodic bookie checker works. + */ + @Test + public void testPeriodicBookieCheckInterval() throws Exception { + confByIndex(0).setMetadataServiceUri( + zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + runFunctionWithLedgerManagerFactory(confByIndex(0), mFactory -> { + try (LedgerManager ledgerManager = mFactory.newLedgerManager()) { + @Cleanup final LedgerUnderreplicationManager underReplicationManager = + mFactory.newLedgerUnderreplicationManager(); + long ledgerId = 12345L; + ClientUtil.setupLedger(bkc.getLedgerManager(), ledgerId, + LedgerMetadataBuilder.create().withEnsembleSize(3) + .withWriteQuorumSize(3).withAckQuorumSize(3) + .newEnsembleEntry(0L, Lists.newArrayList( + new BookieSocketAddress("192.0.2.1", 1000).toBookieId(), + getBookie(0), + getBookie(1)))); + long underReplicatedLedger = -1; + for (int i = 0; i < 10; i++) { + underReplicatedLedger = underReplicationManager.pollLedgerToRereplicate(); + if (underReplicatedLedger != -1) { + break; + } + Thread.sleep(CHECK_INTERVAL * 1000); + } + assertEquals("Ledger should be under replicated", ledgerId, underReplicatedLedger); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + return null; + }); + } +} diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java index 901361dd3a277..8eb04ce7185d7 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java @@ -18,20 +18,42 @@ */ package org.apache.bookkeeper.replication; +import static org.apache.bookkeeper.replication.ReplicationStats.AUDITOR_SCOPE; import static org.testng.AssertJUnit.assertEquals; import static org.testng.AssertJUnit.assertFalse; import static org.testng.AssertJUnit.assertNotSame; +import static org.testng.AssertJUnit.assertTrue; +import static org.testng.AssertJUnit.fail; import io.netty.buffer.ByteBuf; +import java.io.File; +import java.io.FileOutputStream; +import java.io.FilenameFilter; import java.io.IOException; import java.net.URI; +import java.net.UnknownHostException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; +import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.concurrent.CountDownLatch; +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.concurrent.atomic.AtomicReference; import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.BookieAccessor; import org.apache.bookkeeper.bookie.BookieException; +import org.apache.bookkeeper.bookie.BookieImpl; +import org.apache.bookkeeper.bookie.IndexPersistenceMgr; import org.apache.bookkeeper.bookie.TestBookieImpl; +import org.apache.bookkeeper.client.AsyncCallback.AddCallback; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.BookKeeper.DigestType; +import org.apache.bookkeeper.client.BookKeeperAdmin; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManagerFactory; @@ -40,8 +62,15 @@ import org.apache.bookkeeper.meta.MetadataDrivers; import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback; +import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; +import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.bookkeeper.test.TestStatsProvider; +import org.apache.bookkeeper.test.TestStatsProvider.TestOpStatsLogger; +import org.apache.bookkeeper.test.TestStatsProvider.TestStatsLogger; +import org.awaitility.Awaitility; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.annotations.AfterMethod; @@ -112,6 +141,829 @@ public void tearDown() throws Exception { super.tearDown(); } + /** + * test that the periodic checking will detect corruptions in + * the bookie entry log. + */ + @Test + public void testEntryLogCorruption() throws Exception { + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerUnderreplicationManager underReplicationManager = mFactory.newLedgerUnderreplicationManager(); + underReplicationManager.disableLedgerReplication(); + + LedgerHandle lh = bkc.createLedger(3, 3, DigestType.CRC32, "passwd".getBytes()); + long ledgerId = lh.getId(); + for (int i = 0; i < 100; i++) { + lh.addEntry("testdata".getBytes()); + } + lh.close(); + + BookieAccessor.forceFlush((BookieImpl) serverByIndex(0).getBookie()); + + + File ledgerDir = confByIndex(0).getLedgerDirs()[0]; + ledgerDir = BookieImpl.getCurrentDirectory(ledgerDir); + // corrupt of entryLogs + File[] entryLogs = ledgerDir.listFiles(new FilenameFilter() { + public boolean accept(File dir, String name) { + return name.endsWith(".log"); + } + }); + ByteBuffer junk = ByteBuffer.allocate(1024 * 1024); + for (File f : entryLogs) { + FileOutputStream out = new FileOutputStream(f); + out.getChannel().write(junk); + out.close(); + } + restartBookies(); // restart to clear read buffers + + underReplicationManager.enableLedgerReplication(); + long underReplicatedLedger = -1; + for (int i = 0; i < 10; i++) { + underReplicatedLedger = underReplicationManager.pollLedgerToRereplicate(); + if (underReplicatedLedger != -1) { + break; + } + Thread.sleep(CHECK_INTERVAL * 1000); + } + assertEquals("Ledger should be under replicated", ledgerId, underReplicatedLedger); + underReplicationManager.close(); + } + + /** + * test that the period checker will detect corruptions in + * the bookie index files. + */ + @Test + public void testIndexCorruption() throws Exception { + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + + LedgerUnderreplicationManager underReplicationManager = mFactory.newLedgerUnderreplicationManager(); + + LedgerHandle lh = bkc.createLedger(3, 3, DigestType.CRC32, "passwd".getBytes()); + long ledgerToCorrupt = lh.getId(); + for (int i = 0; i < 100; i++) { + lh.addEntry("testdata".getBytes()); + } + lh.close(); + + // push ledgerToCorrupt out of page cache (bookie is configured to only use 1 page) + lh = bkc.createLedger(3, 3, DigestType.CRC32, "passwd".getBytes()); + for (int i = 0; i < 100; i++) { + lh.addEntry("testdata".getBytes()); + } + lh.close(); + + BookieAccessor.forceFlush((BookieImpl) serverByIndex(0).getBookie()); + + File ledgerDir = confByIndex(0).getLedgerDirs()[0]; + ledgerDir = BookieImpl.getCurrentDirectory(ledgerDir); + + // corrupt of entryLogs + File index = new File(ledgerDir, IndexPersistenceMgr.getLedgerName(ledgerToCorrupt)); + LOG.info("file to corrupt{}", index); + ByteBuffer junk = ByteBuffer.allocate(1024 * 1024); + FileOutputStream out = new FileOutputStream(index); + out.getChannel().write(junk); + out.close(); + + long underReplicatedLedger = -1; + for (int i = 0; i < 15; i++) { + underReplicatedLedger = underReplicationManager.pollLedgerToRereplicate(); + if (underReplicatedLedger != -1) { + break; + } + Thread.sleep(CHECK_INTERVAL * 1000); + } + assertEquals("Ledger should be under replicated", ledgerToCorrupt, underReplicatedLedger); + underReplicationManager.close(); + } + + /** + * Test that the period checker will not run when auto replication has been disabled. + */ + @Test + public void testPeriodicCheckWhenDisabled() throws Exception { + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + final LedgerUnderreplicationManager underReplicationManager = mFactory.newLedgerUnderreplicationManager(); + final int numLedgers = 10; + final int numMsgs = 2; + final CountDownLatch completeLatch = new CountDownLatch(numMsgs * numLedgers); + final AtomicInteger rc = new AtomicInteger(BKException.Code.OK); + + List lhs = new ArrayList(); + for (int i = 0; i < numLedgers; i++) { + LedgerHandle lh = bkc.createLedger(3, 3, DigestType.CRC32, "passwd".getBytes()); + lhs.add(lh); + for (int j = 0; j < 2; j++) { + lh.asyncAddEntry("testdata".getBytes(), new AddCallback() { + public void addComplete(int rc2, LedgerHandle lh, long entryId, Object ctx) { + if (rc.compareAndSet(BKException.Code.OK, rc2)) { + LOG.info("Failed to add entry : {}", BKException.getMessage(rc2)); + } + completeLatch.countDown(); + } + }, null); + } + } + completeLatch.await(); + if (rc.get() != BKException.Code.OK) { + throw BKException.create(rc.get()); + } + + for (LedgerHandle lh : lhs) { + lh.close(); + } + + underReplicationManager.disableLedgerReplication(); + + final AtomicInteger numReads = new AtomicInteger(0); + ServerConfiguration conf = killBookie(0); + + Bookie deadBookie = new TestBookieImpl(conf) { + @Override + public ByteBuf readEntry(long ledgerId, long entryId) + throws IOException, NoLedgerException { + // we want to disable during checking + numReads.incrementAndGet(); + throw new IOException("Fake I/O exception"); + } + }; + startAndAddBookie(conf, deadBookie); + + Thread.sleep(CHECK_INTERVAL * 2000); + assertEquals("Nothing should have tried to read", 0, numReads.get()); + underReplicationManager.enableLedgerReplication(); + Thread.sleep(CHECK_INTERVAL * 2000); // give it time to run + + underReplicationManager.disableLedgerReplication(); + // give it time to stop, from this point nothing new should be marked + Thread.sleep(CHECK_INTERVAL * 2000); + + int numUnderreplicated = 0; + long underReplicatedLedger = -1; + do { + underReplicatedLedger = underReplicationManager.pollLedgerToRereplicate(); + if (underReplicatedLedger == -1) { + break; + } + numUnderreplicated++; + + underReplicationManager.markLedgerReplicated(underReplicatedLedger); + } while (underReplicatedLedger != -1); + + Thread.sleep(CHECK_INTERVAL * 2000); // give a chance to run again (it shouldn't, it's disabled) + + // ensure that nothing is marked as underreplicated + underReplicatedLedger = underReplicationManager.pollLedgerToRereplicate(); + assertEquals("There should be no underreplicated ledgers", -1, underReplicatedLedger); + + LOG.info("{} of {} ledgers underreplicated", numUnderreplicated, numUnderreplicated); + assertTrue("All should be underreplicated", + numUnderreplicated <= numLedgers && numUnderreplicated > 0); + } + + /** + * Test that the period check will succeed if a ledger is deleted midway. + */ + @Test + public void testPeriodicCheckWhenLedgerDeleted() throws Exception { + for (AuditorElector e : auditorElectors.values()) { + e.shutdown(); + } + + final int numLedgers = 10; + List ids = new LinkedList(); + for (int i = 0; i < numLedgers; i++) { + LedgerHandle lh = bkc.createLedger(3, 3, DigestType.CRC32, "passwd".getBytes()); + ids.add(lh.getId()); + for (int j = 0; j < 2; j++) { + lh.addEntry("testdata".getBytes()); + } + lh.close(); + } + + try (final Auditor auditor = new Auditor( + BookieImpl.getBookieId(confByIndex(0)).toString(), + confByIndex(0), NullStatsLogger.INSTANCE)) { + final AtomicBoolean exceptionCaught = new AtomicBoolean(false); + final CountDownLatch latch = new CountDownLatch(1); + Thread t = new Thread() { + public void run() { + try { + latch.countDown(); + for (int i = 0; i < numLedgers; i++) { + ((AuditorCheckAllLedgersTask) auditor.auditorCheckAllLedgersTask).checkAllLedgers(); + } + } catch (Exception e) { + LOG.error("Caught exception while checking all ledgers", e); + exceptionCaught.set(true); + } + } + }; + t.start(); + latch.await(); + for (Long id : ids) { + bkc.deleteLedger(id); + } + t.join(); + assertFalse("Shouldn't have thrown exception", exceptionCaught.get()); + } + } + + @Test + public void testGetLedgerFromZookeeperThrottled() throws Exception { + final int numberLedgers = 30; + + // write ledgers into bookkeeper cluster + try { + for (AuditorElector e : auditorElectors.values()) { + e.shutdown(); + } + + for (int i = 0; i < numberLedgers; ++i) { + LedgerHandle lh = bkc.createLedger(3, 3, DigestType.CRC32, "passwd".getBytes()); + for (int j = 0; j < 5; j++) { + lh.addEntry("testdata".getBytes()); + } + lh.close(); + } + } catch (InterruptedException | BKException e) { + LOG.error("Failed to shutdown auditor elector or write data to ledgers ", e); + fail(); + } + + // create auditor and call `checkAllLedgers` + ServerConfiguration configuration = confByIndex(0); + configuration.setAuditorMaxNumberOfConcurrentOpenLedgerOperations(10); + + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(AUDITOR_SCOPE); + Counter numLedgersChecked = statsLogger + .getCounter(ReplicationStats.NUM_LEDGERS_CHECKED); + Auditor auditor = new Auditor(BookieImpl.getBookieId(configuration).toString(), + configuration, statsLogger); + + try { + ((AuditorCheckAllLedgersTask) auditor.auditorCheckAllLedgersTask).checkAllLedgers(); + assertEquals("NUM_LEDGERS_CHECKED", numberLedgers, (long) numLedgersChecked.get()); + } catch (Exception e) { + LOG.error("Caught exception while checking all ledgers ", e); + fail(); + } + } + + @Test + public void testInitialDelayOfCheckAllLedgers() throws Exception { + for (AuditorElector e : auditorElectors.values()) { + e.shutdown(); + } + + final int numLedgers = 10; + List ids = new LinkedList(); + for (int i = 0; i < numLedgers; i++) { + LedgerHandle lh = bkc.createLedger(3, 3, DigestType.CRC32, "passwd".getBytes()); + ids.add(lh.getId()); + for (int j = 0; j < 2; j++) { + lh.addEntry("testdata".getBytes()); + } + lh.close(); + } + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerUnderreplicationManager urm = mFactory.newLedgerUnderreplicationManager(); + + ServerConfiguration servConf = new ServerConfiguration(confByIndex(0)); + validateInitialDelayOfCheckAllLedgers(urm, -1, 1000, servConf, bkc); + validateInitialDelayOfCheckAllLedgers(urm, 999, 1000, servConf, bkc); + validateInitialDelayOfCheckAllLedgers(urm, 1001, 1000, servConf, bkc); + } + + void validateInitialDelayOfCheckAllLedgers(LedgerUnderreplicationManager urm, long timeSinceLastExecutedInSecs, + long auditorPeriodicCheckInterval, ServerConfiguration servConf, + BookKeeper bkc) + throws UnavailableException, UnknownHostException, InterruptedException { + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(AUDITOR_SCOPE); + TestOpStatsLogger checkAllLedgersStatsLogger = (TestOpStatsLogger) statsLogger + .getOpStatsLogger(ReplicationStats.CHECK_ALL_LEDGERS_TIME); + servConf.setAuditorPeriodicCheckInterval(auditorPeriodicCheckInterval); + servConf.setAuditorPeriodicPlacementPolicyCheckInterval(0); + servConf.setAuditorPeriodicBookieCheckInterval(0); + + final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf, bkc, false, + statsLogger, null); + CountDownLatch latch = auditor.getLatch(); + assertEquals("CHECK_ALL_LEDGERS_TIME SuccessCount", 0, checkAllLedgersStatsLogger.getSuccessCount()); + long curTimeBeforeStart = System.currentTimeMillis(); + long checkAllLedgersCTime = -1; + long initialDelayInMsecs = -1; + long nextExpectedCheckAllLedgersExecutionTime = -1; + long bufferTimeInMsecs = 12000L; + if (timeSinceLastExecutedInSecs == -1) { + /* + * if we are setting checkAllLedgersCTime to -1, it means that + * checkAllLedgers hasn't run before. So initialDelay for + * checkAllLedgers should be 0. + */ + checkAllLedgersCTime = -1; + initialDelayInMsecs = 0; + } else { + checkAllLedgersCTime = curTimeBeforeStart - timeSinceLastExecutedInSecs * 1000L; + initialDelayInMsecs = timeSinceLastExecutedInSecs > auditorPeriodicCheckInterval ? 0 + : (auditorPeriodicCheckInterval - timeSinceLastExecutedInSecs) * 1000L; + } + /* + * next checkAllLedgers should happen atleast after + * nextExpectedCheckAllLedgersExecutionTime. + */ + nextExpectedCheckAllLedgersExecutionTime = curTimeBeforeStart + initialDelayInMsecs; + + urm.setCheckAllLedgersCTime(checkAllLedgersCTime); + auditor.start(); + /* + * since auditorPeriodicCheckInterval are higher values (in the order of + * 100s of seconds), its ok bufferTimeInMsecs to be ` 10 secs. + */ + assertTrue("checkAllLedgers should have executed with initialDelay " + initialDelayInMsecs, + latch.await(initialDelayInMsecs + bufferTimeInMsecs, TimeUnit.MILLISECONDS)); + for (int i = 0; i < 10; i++) { + Thread.sleep(100); + if (checkAllLedgersStatsLogger.getSuccessCount() >= 1) { + break; + } + } + assertEquals("CHECK_ALL_LEDGERS_TIME SuccessCount", 1, checkAllLedgersStatsLogger.getSuccessCount()); + long currentCheckAllLedgersCTime = urm.getCheckAllLedgersCTime(); + assertTrue( + "currentCheckAllLedgersCTime: " + currentCheckAllLedgersCTime + + " should be greater than nextExpectedCheckAllLedgersExecutionTime: " + + nextExpectedCheckAllLedgersExecutionTime, + currentCheckAllLedgersCTime > nextExpectedCheckAllLedgersExecutionTime); + assertTrue( + "currentCheckAllLedgersCTime: " + currentCheckAllLedgersCTime + + " should be lesser than nextExpectedCheckAllLedgersExecutionTime+bufferTimeInMsecs: " + + (nextExpectedCheckAllLedgersExecutionTime + bufferTimeInMsecs), + currentCheckAllLedgersCTime < (nextExpectedCheckAllLedgersExecutionTime + bufferTimeInMsecs)); + auditor.close(); + } + + @Test + public void testInitialDelayOfPlacementPolicyCheck() throws Exception { + for (AuditorElector e : auditorElectors.values()) { + e.shutdown(); + } + + final int numLedgers = 10; + List ids = new LinkedList(); + for (int i = 0; i < numLedgers; i++) { + LedgerHandle lh = bkc.createLedger(3, 3, DigestType.CRC32, "passwd".getBytes()); + ids.add(lh.getId()); + for (int j = 0; j < 2; j++) { + lh.addEntry("testdata".getBytes()); + } + lh.close(); + } + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerUnderreplicationManager urm = mFactory.newLedgerUnderreplicationManager(); + + ServerConfiguration servConf = new ServerConfiguration(confByIndex(0)); + validateInitialDelayOfPlacementPolicyCheck(urm, -1, 1000, servConf, bkc); + validateInitialDelayOfPlacementPolicyCheck(urm, 999, 1000, servConf, bkc); + validateInitialDelayOfPlacementPolicyCheck(urm, 1001, 1000, servConf, bkc); + } + + void validateInitialDelayOfPlacementPolicyCheck(LedgerUnderreplicationManager urm, long timeSinceLastExecutedInSecs, + long auditorPeriodicPlacementPolicyCheckInterval, + ServerConfiguration servConf, BookKeeper bkc) + throws UnavailableException, UnknownHostException, InterruptedException { + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(AUDITOR_SCOPE); + TestOpStatsLogger placementPolicyCheckStatsLogger = (TestOpStatsLogger) statsLogger + .getOpStatsLogger(ReplicationStats.PLACEMENT_POLICY_CHECK_TIME); + servConf.setAuditorPeriodicPlacementPolicyCheckInterval(auditorPeriodicPlacementPolicyCheckInterval); + servConf.setAuditorPeriodicCheckInterval(0); + servConf.setAuditorPeriodicBookieCheckInterval(0); + + final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf, bkc, false, + statsLogger, null); + CountDownLatch latch = auditor.getLatch(); + assertEquals("PLACEMENT_POLICY_CHECK_TIME SuccessCount", 0, placementPolicyCheckStatsLogger.getSuccessCount()); + long curTimeBeforeStart = System.currentTimeMillis(); + long placementPolicyCheckCTime = -1; + long initialDelayInMsecs = -1; + long nextExpectedPlacementPolicyCheckExecutionTime = -1; + long bufferTimeInMsecs = 20000L; + if (timeSinceLastExecutedInSecs == -1) { + /* + * if we are setting placementPolicyCheckCTime to -1, it means that + * placementPolicyCheck hasn't run before. So initialDelay for + * placementPolicyCheck should be 0. + */ + placementPolicyCheckCTime = -1; + initialDelayInMsecs = 0; + } else { + placementPolicyCheckCTime = curTimeBeforeStart - timeSinceLastExecutedInSecs * 1000L; + initialDelayInMsecs = timeSinceLastExecutedInSecs > auditorPeriodicPlacementPolicyCheckInterval ? 0 + : (auditorPeriodicPlacementPolicyCheckInterval - timeSinceLastExecutedInSecs) * 1000L; + } + /* + * next placementPolicyCheck should happen atleast after + * nextExpectedPlacementPolicyCheckExecutionTime. + */ + nextExpectedPlacementPolicyCheckExecutionTime = curTimeBeforeStart + initialDelayInMsecs; + + urm.setPlacementPolicyCheckCTime(placementPolicyCheckCTime); + auditor.start(); + /* + * since auditorPeriodicPlacementPolicyCheckInterval are higher values (in the + * order of 100s of seconds), its ok bufferTimeInMsecs to be ` 20 secs. + */ + assertTrue("placementPolicyCheck should have executed with initialDelay " + initialDelayInMsecs, + latch.await(initialDelayInMsecs + bufferTimeInMsecs, TimeUnit.MILLISECONDS)); + for (int i = 0; i < 20; i++) { + Thread.sleep(100); + if (placementPolicyCheckStatsLogger.getSuccessCount() >= 1) { + break; + } + } + assertEquals("PLACEMENT_POLICY_CHECK_TIME SuccessCount", 1, placementPolicyCheckStatsLogger.getSuccessCount()); + long currentPlacementPolicyCheckCTime = urm.getPlacementPolicyCheckCTime(); + assertTrue( + "currentPlacementPolicyCheckCTime: " + currentPlacementPolicyCheckCTime + + " should be greater than nextExpectedPlacementPolicyCheckExecutionTime: " + + nextExpectedPlacementPolicyCheckExecutionTime, + currentPlacementPolicyCheckCTime > nextExpectedPlacementPolicyCheckExecutionTime); + assertTrue( + "currentPlacementPolicyCheckCTime: " + currentPlacementPolicyCheckCTime + + " should be lesser than nextExpectedPlacementPolicyCheckExecutionTime+bufferTimeInMsecs: " + + (nextExpectedPlacementPolicyCheckExecutionTime + bufferTimeInMsecs), + currentPlacementPolicyCheckCTime < (nextExpectedPlacementPolicyCheckExecutionTime + bufferTimeInMsecs)); + auditor.close(); + } + + @Test + public void testInitialDelayOfReplicasCheck() throws Exception { + for (AuditorElector e : auditorElectors.values()) { + e.shutdown(); + } + + LedgerHandle lh = bkc.createLedger(3, 2, DigestType.CRC32, "passwd".getBytes()); + for (int j = 0; j < 5; j++) { + lh.addEntry("testdata".getBytes()); + } + lh.close(); + + long ledgerId = 100000L; + lh = bkc.createLedgerAdv(ledgerId, 3, 2, 2, DigestType.CRC32, "passwd".getBytes(), null); + lh.close(); + + ledgerId = 100001234L; + lh = bkc.createLedgerAdv(ledgerId, 3, 3, 2, DigestType.CRC32, "passwd".getBytes(), null); + for (int j = 0; j < 4; j++) { + lh.addEntry(j, "testdata".getBytes()); + } + lh.close(); + + ledgerId = 991234L; + lh = bkc.createLedgerAdv(ledgerId, 3, 2, 2, DigestType.CRC32, "passwd".getBytes(), null); + lh.addEntry(0, "testdata".getBytes()); + lh.close(); + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerUnderreplicationManager urm = mFactory.newLedgerUnderreplicationManager(); + + ServerConfiguration servConf = new ServerConfiguration(confByIndex(0)); + validateInitialDelayOfReplicasCheck(urm, -1, 1000, servConf, bkc); + validateInitialDelayOfReplicasCheck(urm, 999, 1000, servConf, bkc); + validateInitialDelayOfReplicasCheck(urm, 1001, 1000, servConf, bkc); + } + + void validateInitialDelayOfReplicasCheck(LedgerUnderreplicationManager urm, long timeSinceLastExecutedInSecs, + long auditorPeriodicReplicasCheckInterval, ServerConfiguration servConf, + BookKeeper bkc) + throws UnavailableException, UnknownHostException, InterruptedException { + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(AUDITOR_SCOPE); + TestOpStatsLogger replicasCheckStatsLogger = (TestOpStatsLogger) statsLogger + .getOpStatsLogger(ReplicationStats.REPLICAS_CHECK_TIME); + servConf.setAuditorPeriodicReplicasCheckInterval(auditorPeriodicReplicasCheckInterval); + servConf.setAuditorPeriodicCheckInterval(0); + servConf.setAuditorPeriodicBookieCheckInterval(0); + final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf, bkc, false, + statsLogger, null); + CountDownLatch latch = auditor.getLatch(); + assertEquals("REPLICAS_CHECK_TIME SuccessCount", 0, replicasCheckStatsLogger.getSuccessCount()); + long curTimeBeforeStart = System.currentTimeMillis(); + long replicasCheckCTime = -1; + long initialDelayInMsecs = -1; + long nextExpectedReplicasCheckExecutionTime = -1; + long bufferTimeInMsecs = 20000L; + if (timeSinceLastExecutedInSecs == -1) { + /* + * if we are setting replicasCheckCTime to -1, it means that + * replicasCheck hasn't run before. So initialDelay for + * replicasCheck should be 0. + */ + replicasCheckCTime = -1; + initialDelayInMsecs = 0; + } else { + replicasCheckCTime = curTimeBeforeStart - timeSinceLastExecutedInSecs * 1000L; + initialDelayInMsecs = timeSinceLastExecutedInSecs > auditorPeriodicReplicasCheckInterval ? 0 + : (auditorPeriodicReplicasCheckInterval - timeSinceLastExecutedInSecs) * 1000L; + } + /* + * next replicasCheck should happen atleast after + * nextExpectedReplicasCheckExecutionTime. + */ + nextExpectedReplicasCheckExecutionTime = curTimeBeforeStart + initialDelayInMsecs; + + urm.setReplicasCheckCTime(replicasCheckCTime); + auditor.start(); + /* + * since auditorPeriodicReplicasCheckInterval are higher values (in the + * order of 100s of seconds), its ok bufferTimeInMsecs to be ` 20 secs. + */ + assertTrue("replicasCheck should have executed with initialDelay " + initialDelayInMsecs, + latch.await(initialDelayInMsecs + bufferTimeInMsecs, TimeUnit.MILLISECONDS)); + for (int i = 0; i < 20; i++) { + Thread.sleep(100); + if (replicasCheckStatsLogger.getSuccessCount() >= 1) { + break; + } + } + assertEquals("REPLICAS_CHECK_TIME SuccessCount", 1, replicasCheckStatsLogger.getSuccessCount()); + long currentReplicasCheckCTime = urm.getReplicasCheckCTime(); + assertTrue( + "currentReplicasCheckCTime: " + currentReplicasCheckCTime + + " should be greater than nextExpectedReplicasCheckExecutionTime: " + + nextExpectedReplicasCheckExecutionTime, + currentReplicasCheckCTime > nextExpectedReplicasCheckExecutionTime); + assertTrue( + "currentReplicasCheckCTime: " + currentReplicasCheckCTime + + " should be lesser than nextExpectedReplicasCheckExecutionTime+bufferTimeInMsecs: " + + (nextExpectedReplicasCheckExecutionTime + bufferTimeInMsecs), + currentReplicasCheckCTime < (nextExpectedReplicasCheckExecutionTime + bufferTimeInMsecs)); + auditor.close(); + } + + @Test + public void testDelayBookieAuditOfCheckAllLedgers() throws Exception { + for (AuditorElector e : auditorElectors.values()) { + e.shutdown(); + } + + final int numLedgers = 10; + List ids = new LinkedList(); + for (int i = 0; i < numLedgers; i++) { + LedgerHandle lh = bkc.createLedger(3, 3, DigestType.CRC32, "passwd".getBytes()); + ids.add(lh.getId()); + for (int j = 0; j < 2; j++) { + lh.addEntry("testdata".getBytes()); + } + lh.close(); + } + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerUnderreplicationManager urm = mFactory.newLedgerUnderreplicationManager(); + + ServerConfiguration servConf = new ServerConfiguration(confByIndex(0)); + + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(AUDITOR_SCOPE); + Counter numBookieAuditsDelayed = + statsLogger.getCounter(ReplicationStats.NUM_BOOKIE_AUDITS_DELAYED); + TestOpStatsLogger underReplicatedLedgerTotalSizeStatsLogger = (TestOpStatsLogger) statsLogger + .getOpStatsLogger(ReplicationStats.UNDER_REPLICATED_LEDGERS_TOTAL_SIZE); + + servConf.setAuditorPeriodicCheckInterval(1); + servConf.setAuditorPeriodicPlacementPolicyCheckInterval(0); + servConf.setAuditorPeriodicBookieCheckInterval(Long.MAX_VALUE); + + urm.setLostBookieRecoveryDelay(Integer.MAX_VALUE); + + AtomicBoolean canRun = new AtomicBoolean(false); + + final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf, bkc, + false, statsLogger, canRun); + final CountDownLatch latch = auditor.getLatch(); + + auditor.start(); + + killBookie(addressByIndex(0)); + + Awaitility.await().untilAsserted(() -> assertEquals(1, (long) numBookieAuditsDelayed.get())); + final Future auditTask = auditor.auditTask; + assertTrue(auditTask != null && !auditTask.isDone()); + + canRun.set(true); + + assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertTrue(auditor.auditTask.equals(auditTask) + && auditor.auditTask != null && !auditor.auditTask.isDone()); + // wrong num is numLedgers, right num is 0 + assertEquals("UNDER_REPLICATED_LEDGERS_TOTAL_SIZE", + 0, + underReplicatedLedgerTotalSizeStatsLogger.getSuccessCount()); + + auditor.close(); + } + + @Test + public void testDelayBookieAuditOfPlacementPolicy() throws Exception { + for (AuditorElector e : auditorElectors.values()) { + e.shutdown(); + } + + final int numLedgers = 10; + List ids = new LinkedList(); + for (int i = 0; i < numLedgers; i++) { + LedgerHandle lh = bkc.createLedger(3, 3, DigestType.CRC32, "passwd".getBytes()); + ids.add(lh.getId()); + for (int j = 0; j < 2; j++) { + lh.addEntry("testdata".getBytes()); + } + lh.close(); + } + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerUnderreplicationManager urm = mFactory.newLedgerUnderreplicationManager(); + + ServerConfiguration servConf = new ServerConfiguration(confByIndex(0)); + + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(AUDITOR_SCOPE); + Counter numBookieAuditsDelayed = + statsLogger.getCounter(ReplicationStats.NUM_BOOKIE_AUDITS_DELAYED); + TestOpStatsLogger placementPolicyCheckTime = (TestOpStatsLogger) statsLogger + .getOpStatsLogger(ReplicationStats.PLACEMENT_POLICY_CHECK_TIME); + + servConf.setAuditorPeriodicCheckInterval(0); + servConf.setAuditorPeriodicPlacementPolicyCheckInterval(1); + servConf.setAuditorPeriodicBookieCheckInterval(Long.MAX_VALUE); + + urm.setLostBookieRecoveryDelay(Integer.MAX_VALUE); + + AtomicBoolean canRun = new AtomicBoolean(false); + + final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf, bkc, + false, statsLogger, canRun); + final CountDownLatch latch = auditor.getLatch(); + + auditor.start(); + + killBookie(addressByIndex(0)); + + Awaitility.await().untilAsserted(() -> assertEquals(1, (long) numBookieAuditsDelayed.get())); + final Future auditTask = auditor.auditTask; + assertTrue(auditTask != null && !auditTask.isDone()); + assertEquals("PLACEMENT_POLICY_CHECK_TIME", 0, placementPolicyCheckTime.getSuccessCount()); + + canRun.set(true); + + assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertTrue(auditor.auditTask.equals(auditTask) + && auditor.auditTask != null && !auditor.auditTask.isDone()); + // wrong successCount is > 0, right successCount is = 0 + assertEquals("PLACEMENT_POLICY_CHECK_TIME", 0, placementPolicyCheckTime.getSuccessCount()); + + auditor.close(); + } + + @Test + public void testDelayBookieAuditOfReplicasCheck() throws Exception { + for (AuditorElector e : auditorElectors.values()) { + e.shutdown(); + } + + final int numLedgers = 10; + List ids = new LinkedList(); + for (int i = 0; i < numLedgers; i++) { + LedgerHandle lh = bkc.createLedger(3, 3, DigestType.CRC32, "passwd".getBytes()); + ids.add(lh.getId()); + for (int j = 0; j < 2; j++) { + lh.addEntry("testdata".getBytes()); + } + lh.close(); + } + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerUnderreplicationManager urm = mFactory.newLedgerUnderreplicationManager(); + + ServerConfiguration servConf = new ServerConfiguration(confByIndex(0)); + + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(AUDITOR_SCOPE); + Counter numBookieAuditsDelayed = + statsLogger.getCounter(ReplicationStats.NUM_BOOKIE_AUDITS_DELAYED); + TestOpStatsLogger replicasCheckTime = (TestOpStatsLogger) statsLogger + .getOpStatsLogger(ReplicationStats.REPLICAS_CHECK_TIME); + + servConf.setAuditorPeriodicCheckInterval(0); + servConf.setAuditorPeriodicPlacementPolicyCheckInterval(0); + servConf.setAuditorPeriodicBookieCheckInterval(Long.MAX_VALUE); + servConf.setAuditorPeriodicReplicasCheckInterval(1); + + urm.setLostBookieRecoveryDelay(Integer.MAX_VALUE); + + AtomicBoolean canRun = new AtomicBoolean(false); + + final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf, bkc, + false, statsLogger, canRun); + final CountDownLatch latch = auditor.getLatch(); + + auditor.start(); + + killBookie(addressByIndex(0)); + + Awaitility.await().untilAsserted(() -> assertEquals(1, (long) numBookieAuditsDelayed.get())); + final Future auditTask = auditor.auditTask; + assertTrue(auditTask != null && !auditTask.isDone()); + assertEquals("REPLICAS_CHECK_TIME", 0, replicasCheckTime.getSuccessCount()); + + canRun.set(true); + + assertTrue(latch.await(10, TimeUnit.SECONDS)); + assertTrue(auditor.auditTask.equals(auditTask) + && auditor.auditTask != null && !auditor.auditTask.isDone()); + // wrong successCount is > 0, right successCount is = 0 + assertEquals("REPLICAS_CHECK_TIME", 0, replicasCheckTime.getSuccessCount()); + + auditor.close(); + } + + static class TestAuditor extends Auditor { + + final AtomicReference latchRef = new AtomicReference(new CountDownLatch(1)); + + public TestAuditor(String bookieIdentifier, ServerConfiguration conf, BookKeeper bkc, boolean ownBkc, + StatsLogger statsLogger, AtomicBoolean exceptedRun) throws UnavailableException { + super(bookieIdentifier, conf, bkc, ownBkc, statsLogger); + renewAuditorTestWrapperTask(exceptedRun); + } + + public TestAuditor(String bookieIdentifier, ServerConfiguration conf, BookKeeper bkc, boolean ownBkc, + BookKeeperAdmin bkadmin, boolean ownadmin, StatsLogger statsLogger, + AtomicBoolean exceptedRun) throws UnavailableException { + super(bookieIdentifier, conf, bkc, ownBkc, bkadmin, ownadmin, statsLogger); + renewAuditorTestWrapperTask(exceptedRun); + } + + public TestAuditor(final String bookieIdentifier, ServerConfiguration conf, StatsLogger statsLogger, + AtomicBoolean exceptedRun) + throws UnavailableException { + super(bookieIdentifier, conf, statsLogger); + renewAuditorTestWrapperTask(exceptedRun); + } + + private void renewAuditorTestWrapperTask(AtomicBoolean exceptedRun) { + super.auditorCheckAllLedgersTask = + new AuditorTestWrapperTask(super.auditorCheckAllLedgersTask, latchRef, exceptedRun); + super.auditorPlacementPolicyCheckTask = + new AuditorTestWrapperTask(super.auditorPlacementPolicyCheckTask, latchRef, exceptedRun); + super.auditorReplicasCheckTask = + new AuditorTestWrapperTask(super.auditorReplicasCheckTask, latchRef, exceptedRun); + } + + CountDownLatch getLatch() { + return latchRef.get(); + } + + void setLatch(CountDownLatch latch) { + latchRef.set(latch); + } + + private static class AuditorTestWrapperTask extends AuditorTask { + private final AuditorTask innerTask; + private final AtomicReference latchRef; + private final AtomicBoolean exceptedRun; + + AuditorTestWrapperTask(AuditorTask innerTask, + AtomicReference latchRef, + AtomicBoolean exceptedRun) { + super(null, null, null, null, null, + null, null); + this.innerTask = innerTask; + this.latchRef = latchRef; + this.exceptedRun = exceptedRun; + } + + @Override + protected void runTask() { + if (exceptedRun == null || exceptedRun.get()) { + innerTask.runTask(); + latchRef.get().countDown(); + } + } + + @Override + public void shutdown() { + innerTask.shutdown(); + } + } + } + private BookieId replaceBookieWithWriteFailingBookie(LedgerHandle lh) throws Exception { int bookieIdx = -1; Long entryId = lh.getLedgerMetadata().getAllEnsembles().firstKey(); diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTaskTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTaskTest.java new file mode 100644 index 0000000000000..1bafb8589d91a --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTaskTest.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.bookkeeper.replication; + +import static org.apache.bookkeeper.replication.ReplicationStats.AUDITOR_SCOPE; +import static org.testng.AssertJUnit.assertEquals; +import java.util.LinkedList; +import java.util.List; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.bookkeeper.test.TestStatsProvider; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +/** + * Unit test {@link AuditorPlacementPolicyCheckTask}. + */ +public class AuditorPlacementPolicyCheckTaskTest extends BookKeeperClusterTestCase { + private static final Logger LOG = LoggerFactory + .getLogger(AuditorPlacementPolicyCheckTaskTest.class); + + private BookKeeperAdmin admin; + private LedgerManager ledgerManager; + private LedgerUnderreplicationManager ledgerUnderreplicationManager; + + public AuditorPlacementPolicyCheckTaskTest() throws Exception { + super(3); + baseConf.setPageLimit(1); + baseConf.setAutoRecoveryDaemonEnabled(false); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataClientDriver"); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataBookieDriver"); + } + + @BeforeMethod + @Override + public void setUp() throws Exception { + super.setUp(); + baseClientConf.setMetadataServiceUri( + metadataServiceUri.replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + final BookKeeper bookKeeper = new BookKeeper(baseClientConf); + admin = new BookKeeperAdmin(bookKeeper, NullStatsLogger.INSTANCE, new ClientConfiguration(baseClientConf)); + LedgerManagerFactory ledgerManagerFactory = bookKeeper.getLedgerManagerFactory(); + ledgerManager = ledgerManagerFactory.newLedgerManager(); + ledgerUnderreplicationManager = ledgerManagerFactory.newLedgerUnderreplicationManager(); + } + + @AfterMethod + @Override + public void tearDown() throws Exception { + if (ledgerManager != null) { + ledgerManager.close(); + } + if (ledgerUnderreplicationManager != null) { + ledgerUnderreplicationManager.close(); + } + super.tearDown(); + } + + @Test + public void testPlacementPolicyCheck() throws BKException, InterruptedException { + + // 1. create ledgers + final int numLedgers = 10; + List ids = new LinkedList(); + for (int i = 0; i < numLedgers; i++) { + LedgerHandle lh = bkc.createLedger(3, 3, BookKeeper.DigestType.CRC32, "passwd".getBytes()); + ids.add(lh.getId()); + for (int j = 0; j < 2; j++) { + lh.addEntry("testdata".getBytes()); + } + lh.close(); + } + + // 2. init auditorPlacementPolicyCheckTask + final TestStatsProvider statsProvider = new TestStatsProvider(); + final TestStatsProvider.TestStatsLogger statsLogger = statsProvider.getStatsLogger(AUDITOR_SCOPE); + final AuditorStats auditorStats = new AuditorStats(statsLogger); + + AuditorPlacementPolicyCheckTask auditorPlacementPolicyCheckTask = new AuditorPlacementPolicyCheckTask( + baseConf, auditorStats, admin, ledgerManager, + ledgerUnderreplicationManager, null, (flag, throwable) -> flag.set(false)); + + // 3. placementPolicyCheck + auditorPlacementPolicyCheckTask.runTask(); + + // 4. verify + assertEquals("PLACEMENT_POLICY_CHECK_TIME", 1, ((TestStatsProvider.TestOpStatsLogger) + statsLogger.getOpStatsLogger(ReplicationStats.PLACEMENT_POLICY_CHECK_TIME)).getSuccessCount()); + assertEquals("numOfClosedLedgersAuditedInPlacementPolicyCheck", + numLedgers, + auditorPlacementPolicyCheckTask.getNumOfClosedLedgersAuditedInPlacementPolicyCheck().get()); + assertEquals("numOfLedgersFoundNotAdheringInPlacementPolicyCheck", + numLedgers, + auditorPlacementPolicyCheckTask.getNumOfLedgersFoundNotAdheringInPlacementPolicyCheck().get()); + } + +} \ No newline at end of file diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java new file mode 100644 index 0000000000000..159a4e88a33bd --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java @@ -0,0 +1,861 @@ +/* + * 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.replication; + +import static org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.REPP_DNS_RESOLVER_CLASS; +import static org.apache.bookkeeper.replication.ReplicationStats.AUDITOR_SCOPE; +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertTrue; +import java.net.URI; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import org.apache.bookkeeper.bookie.BookieImpl; +import org.apache.bookkeeper.client.LedgerMetadataBuilder; +import org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy; +import org.apache.bookkeeper.client.ZoneawareEnsemblePlacementPolicy; +import org.apache.bookkeeper.client.api.DigestType; +import org.apache.bookkeeper.client.api.LedgerMetadata; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.meta.MetadataBookieDriver; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.meta.exceptions.MetadataException; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.replication.AuditorPeriodicCheckTest.TestAuditor; +import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; +import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; +import org.apache.bookkeeper.stats.Gauge; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.bookkeeper.test.TestStatsProvider; +import org.apache.bookkeeper.test.TestStatsProvider.TestOpStatsLogger; +import org.apache.bookkeeper.test.TestStatsProvider.TestStatsLogger; +import org.apache.bookkeeper.util.StaticDNSResolver; +import org.apache.commons.lang3.mutable.MutableObject; +import org.apache.zookeeper.KeeperException; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +/** + * Tests the logic of Auditor's PlacementPolicyCheck. + */ +public class AuditorPlacementPolicyCheckTest extends BookKeeperClusterTestCase { + private MetadataBookieDriver driver; + + public AuditorPlacementPolicyCheckTest() throws Exception { + super(1); + baseConf.setPageLimit(1); // to make it easy to push ledger out of cache + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataClientDriver"); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataBookieDriver"); + } + + @BeforeMethod + @Override + public void setUp() throws Exception { + super.setUp(); + StaticDNSResolver.reset(); + + URI uri = URI.create(confByIndex(0).getMetadataServiceUri().replaceAll("zk://", "metadata-store:") + .replaceAll("/ledgers", "")); + driver = MetadataDrivers.getBookieDriver(uri); + ServerConfiguration serverConfiguration = new ServerConfiguration(confByIndex(0)); + serverConfiguration.setMetadataServiceUri( + serverConfiguration.getMetadataServiceUri().replaceAll("zk://", "metadata-store:") + .replaceAll("/ledgers", "")); + driver.initialize(serverConfiguration, NullStatsLogger.INSTANCE); + } + + @AfterMethod + @Override + public void tearDown() throws Exception { + if (null != driver) { + driver.close(); + } + super.tearDown(); + } + + @Test + public void testPlacementPolicyCheckWithBookiesFromDifferentRacks() throws Exception { + int numOfBookies = 5; + List bookieAddresses = new ArrayList<>(); + BookieSocketAddress bookieAddress; + RegistrationManager regManager = driver.createRegistrationManager(); + // all the numOfBookies (5) are going to be in different racks + for (int i = 0; i < numOfBookies; i++) { + bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181); + StaticDNSResolver.addNodeToRack(bookieAddress.getHostName(), "/rack" + (i)); + bookieAddresses.add(bookieAddress.toBookieId()); + regManager.registerBookie(bookieAddress.toBookieId(), false, BookieServiceInfo.EMPTY); + } + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerManager lm = mFactory.newLedgerManager(); + int ensembleSize = 5; + int writeQuorumSize = 4; + int ackQuorumSize = 2; + int minNumRacksPerWriteQuorumConfValue = 4; + Collections.shuffle(bookieAddresses); + + // closed ledger + LedgerMetadata initMeta = LedgerMetadataBuilder.create() + .withId(1L) + .withEnsembleSize(ensembleSize) + .withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize) + .newEnsembleEntry(0L, bookieAddresses) + .withClosedState() + .withLastEntryId(100) + .withLength(10000) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(1L, initMeta).get(); + + Collections.shuffle(bookieAddresses); + ensembleSize = 4; + // closed ledger with multiple segments + initMeta = LedgerMetadataBuilder.create() + .withId(2L) + .withEnsembleSize(ensembleSize) + .withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize) + .newEnsembleEntry(0L, bookieAddresses.subList(0, 4)) + .newEnsembleEntry(20L, bookieAddresses.subList(1, 5)) + .newEnsembleEntry(60L, bookieAddresses.subList(0, 4)) + .withClosedState() + .withLastEntryId(100) + .withLength(10000) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(2L, initMeta).get(); + + Collections.shuffle(bookieAddresses); + // non-closed ledger + initMeta = LedgerMetadataBuilder.create() + .withId(3L) + .withEnsembleSize(ensembleSize) + .withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize) + .newEnsembleEntry(0L, bookieAddresses.subList(0, 4)) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(3L, initMeta).get(); + + Collections.shuffle(bookieAddresses); + // non-closed ledger with multiple segments + initMeta = LedgerMetadataBuilder.create() + .withId(4L) + .withEnsembleSize(ensembleSize) + .withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize) + .newEnsembleEntry(0L, bookieAddresses.subList(0, 4)) + .newEnsembleEntry(20L, bookieAddresses.subList(1, 5)) + .newEnsembleEntry(60L, bookieAddresses.subList(0, 4)) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(4L, initMeta).get(); + + ServerConfiguration servConf = new ServerConfiguration(confByIndex(0)); + servConf.setMinNumRacksPerWriteQuorum(minNumRacksPerWriteQuorumConfValue); + setServerConfigPropertiesForRackPlacement(servConf); + MutableObject auditorRef = new MutableObject(); + try { + TestStatsLogger statsLogger = startAuditorAndWaitForPlacementPolicyCheck(servConf, auditorRef); + Gauge ledgersNotAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY); + Gauge ledgersSoftlyAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY); + /* + * since all of the bookies are in different racks, there shouldn't be any ledger not adhering + * to placement policy. + */ + assertEquals("NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY guage value", 0, + ledgersNotAdheringToPlacementPolicyGuage.getSample()); + /* + * since all of the bookies are in different racks, there shouldn't be any ledger softly adhering + * to placement policy. + */ + assertEquals("NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY guage value", 0, + ledgersSoftlyAdheringToPlacementPolicyGuage.getSample()); + } finally { + Auditor auditor = auditorRef.getValue(); + if (auditor != null) { + auditor.close(); + } + regManager.close(); + } + } + + @Test + public void testPlacementPolicyCheckWithLedgersNotAdheringToPlacementPolicy() throws Exception { + int numOfBookies = 5; + int numOfLedgersNotAdheringToPlacementPolicy = 0; + List bookieAddresses = new ArrayList<>(); + RegistrationManager regManager = driver.createRegistrationManager(); + for (int i = 0; i < numOfBookies; i++) { + BookieId bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181).toBookieId(); + bookieAddresses.add(bookieAddress); + regManager.registerBookie(bookieAddress, false, BookieServiceInfo.EMPTY); + } + + // only three racks + StaticDNSResolver.addNodeToRack("98.98.98.0", "/rack1"); + StaticDNSResolver.addNodeToRack("98.98.98.1", "/rack2"); + StaticDNSResolver.addNodeToRack("98.98.98.2", "/rack3"); + StaticDNSResolver.addNodeToRack("98.98.98.3", "/rack1"); + StaticDNSResolver.addNodeToRack("98.98.98.4", "/rack2"); + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerManager lm = mFactory.newLedgerManager(); + int ensembleSize = 5; + int writeQuorumSize = 3; + int ackQuorumSize = 2; + int minNumRacksPerWriteQuorumConfValue = 3; + + /* + * this closed ledger doesn't adhere to placement policy because there are only + * 3 racks, and the ensembleSize is 5. + */ + LedgerMetadata initMeta = LedgerMetadataBuilder.create() + .withId(1L) + .withEnsembleSize(ensembleSize) + .withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize) + .newEnsembleEntry(0L, bookieAddresses) + .withClosedState() + .withLastEntryId(100) + .withLength(10000) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(1L, initMeta).get(); + numOfLedgersNotAdheringToPlacementPolicy++; + + /* + * this is non-closed ledger, so it shouldn't count as ledger not + * adhering to placement policy + */ + initMeta = LedgerMetadataBuilder.create() + .withId(2L) + .withEnsembleSize(ensembleSize) + .withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize) + .newEnsembleEntry(0L, bookieAddresses) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(2L, initMeta).get(); + + ServerConfiguration servConf = new ServerConfiguration(confByIndex(0)); + servConf.setMinNumRacksPerWriteQuorum(minNumRacksPerWriteQuorumConfValue); + setServerConfigPropertiesForRackPlacement(servConf); + MutableObject auditorRef = new MutableObject(); + try { + TestStatsLogger statsLogger = startAuditorAndWaitForPlacementPolicyCheck(servConf, auditorRef); + Gauge ledgersNotAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY); + assertEquals("NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY guage value", + numOfLedgersNotAdheringToPlacementPolicy, ledgersNotAdheringToPlacementPolicyGuage.getSample()); + Gauge ledgersSoftlyAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY); + assertEquals("NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY guage value", + 0, ledgersSoftlyAdheringToPlacementPolicyGuage.getSample()); + } finally { + Auditor auditor = auditorRef.getValue(); + if (auditor != null) { + auditor.close(); + } + regManager.close(); + } + } + + @Test + public void testPlacementPolicyCheckWithLedgersNotAdheringToPlacementPolicyAndNotMarkToUnderreplication() + throws Exception { + int numOfBookies = 5; + int numOfLedgersNotAdheringToPlacementPolicy = 0; + List bookieAddresses = new ArrayList<>(); + RegistrationManager regManager = driver.createRegistrationManager(); + for (int i = 0; i < numOfBookies; i++) { + BookieId bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181).toBookieId(); + bookieAddresses.add(bookieAddress); + regManager.registerBookie(bookieAddress, false, BookieServiceInfo.EMPTY); + } + + // only three racks + StaticDNSResolver.addNodeToRack("98.98.98.0", "/rack1"); + StaticDNSResolver.addNodeToRack("98.98.98.1", "/rack2"); + StaticDNSResolver.addNodeToRack("98.98.98.2", "/rack3"); + StaticDNSResolver.addNodeToRack("98.98.98.3", "/rack1"); + StaticDNSResolver.addNodeToRack("98.98.98.4", "/rack2"); + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerManager lm = mFactory.newLedgerManager(); + int ensembleSize = 5; + int writeQuorumSize = 3; + int ackQuorumSize = 2; + int minNumRacksPerWriteQuorumConfValue = 3; + + /* + * this closed ledger doesn't adhere to placement policy because there are only + * 3 racks, and the ensembleSize is 5. + */ + LedgerMetadata initMeta = LedgerMetadataBuilder.create() + .withId(1L) + .withEnsembleSize(ensembleSize) + .withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize) + .newEnsembleEntry(0L, bookieAddresses) + .withClosedState() + .withLastEntryId(100) + .withLength(10000) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(1L, initMeta).get(); + numOfLedgersNotAdheringToPlacementPolicy++; + + ServerConfiguration servConf = new ServerConfiguration(confByIndex(0)); + servConf.setMinNumRacksPerWriteQuorum(minNumRacksPerWriteQuorumConfValue); + setServerConfigPropertiesForRackPlacement(servConf); + MutableObject auditorRef = new MutableObject(); + try { + TestStatsLogger statsLogger = startAuditorAndWaitForPlacementPolicyCheck(servConf, auditorRef); + Gauge ledgersNotAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY); + assertEquals("NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY guage value", + numOfLedgersNotAdheringToPlacementPolicy, ledgersNotAdheringToPlacementPolicyGuage.getSample()); + Gauge ledgersSoftlyAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY); + assertEquals("NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY guage value", + 0, ledgersSoftlyAdheringToPlacementPolicyGuage.getSample()); + } finally { + Auditor auditor = auditorRef.getValue(); + if (auditor != null) { + auditor.close(); + } + regManager.close(); + } + LedgerUnderreplicationManager underreplicationManager = mFactory.newLedgerUnderreplicationManager(); + long unnderReplicateLedgerId = underreplicationManager.pollLedgerToRereplicate(); + assertEquals(unnderReplicateLedgerId, -1); + } + + @Test + public void testPlacementPolicyCheckWithLedgersNotAdheringToPlacementPolicyAndMarkToUnderreplication() + throws Exception { + int numOfBookies = 5; + int numOfLedgersNotAdheringToPlacementPolicy = 0; + List bookieAddresses = new ArrayList<>(); + RegistrationManager regManager = driver.createRegistrationManager(); + for (int i = 0; i < numOfBookies; i++) { + BookieId bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181).toBookieId(); + bookieAddresses.add(bookieAddress); + regManager.registerBookie(bookieAddress, false, BookieServiceInfo.EMPTY); + } + + // only three racks + StaticDNSResolver.addNodeToRack("98.98.98.0", "/rack1"); + StaticDNSResolver.addNodeToRack("98.98.98.1", "/rack2"); + StaticDNSResolver.addNodeToRack("98.98.98.2", "/rack3"); + StaticDNSResolver.addNodeToRack("98.98.98.3", "/rack1"); + StaticDNSResolver.addNodeToRack("98.98.98.4", "/rack2"); + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerManager lm = mFactory.newLedgerManager(); + int ensembleSize = 5; + int writeQuorumSize = 3; + int ackQuorumSize = 2; + int minNumRacksPerWriteQuorumConfValue = 3; + + /* + * this closed ledger doesn't adhere to placement policy because there are only + * 3 racks, and the ensembleSize is 5. + */ + LedgerMetadata initMeta = LedgerMetadataBuilder.create() + .withId(1L) + .withEnsembleSize(ensembleSize) + .withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize) + .newEnsembleEntry(0L, bookieAddresses) + .withClosedState() + .withLastEntryId(100) + .withLength(10000) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(1L, initMeta).get(); + numOfLedgersNotAdheringToPlacementPolicy++; + + ServerConfiguration servConf = new ServerConfiguration(confByIndex(0)); + servConf.setMinNumRacksPerWriteQuorum(minNumRacksPerWriteQuorumConfValue); + servConf.setRepairedPlacementPolicyNotAdheringBookieEnable(true); + setServerConfigPropertiesForRackPlacement(servConf); + MutableObject auditorRef = new MutableObject(); + try { + TestStatsLogger statsLogger = startAuditorAndWaitForPlacementPolicyCheck(servConf, auditorRef); + Gauge ledgersNotAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY); + assertEquals("NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY guage value", + numOfLedgersNotAdheringToPlacementPolicy, ledgersNotAdheringToPlacementPolicyGuage.getSample()); + Gauge ledgersSoftlyAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY); + assertEquals("NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY guage value", + 0, ledgersSoftlyAdheringToPlacementPolicyGuage.getSample()); + } finally { + Auditor auditor = auditorRef.getValue(); + if (auditor != null) { + auditor.close(); + } + regManager.close(); + } + LedgerUnderreplicationManager underreplicationManager = mFactory.newLedgerUnderreplicationManager(); + long unnderReplicateLedgerId = underreplicationManager.pollLedgerToRereplicate(); + assertEquals(unnderReplicateLedgerId, 1L); + } + + @Test + public void testPlacementPolicyCheckForURLedgersElapsedRecoveryGracePeriod() throws Exception { + testPlacementPolicyCheckWithURLedgers(true); + } + + @Test + public void testPlacementPolicyCheckForURLedgersNotElapsedRecoveryGracePeriod() throws Exception { + testPlacementPolicyCheckWithURLedgers(false); + } + + public void testPlacementPolicyCheckWithURLedgers(boolean timeElapsed) throws Exception { + int numOfBookies = 4; + /* + * in timeElapsed=true scenario, set some low value, otherwise set some + * highValue. + */ + int underreplicatedLedgerRecoveryGracePeriod = timeElapsed ? 1 : 1000; + int numOfURLedgersElapsedRecoveryGracePeriod = 0; + List bookieAddresses = new ArrayList(); + RegistrationManager regManager = driver.createRegistrationManager(); + for (int i = 0; i < numOfBookies; i++) { + BookieId bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181).toBookieId(); + bookieAddresses.add(bookieAddress); + regManager.registerBookie(bookieAddress, false, BookieServiceInfo.EMPTY); + } + + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerManager lm = mFactory.newLedgerManager(); + LedgerUnderreplicationManager underreplicationManager = mFactory.newLedgerUnderreplicationManager(); + int ensembleSize = 4; + int writeQuorumSize = 3; + int ackQuorumSize = 2; + + long ledgerId1 = 1L; + LedgerMetadata initMeta = LedgerMetadataBuilder.create() + .withId(ledgerId1) + .withEnsembleSize(ensembleSize) + .withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize) + .newEnsembleEntry(0L, bookieAddresses) + .withClosedState() + .withLastEntryId(100) + .withLength(10000) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(ledgerId1, initMeta).get(); + underreplicationManager.markLedgerUnderreplicated(ledgerId1, bookieAddresses.get(0).toString()); + if (timeElapsed) { + numOfURLedgersElapsedRecoveryGracePeriod++; + } + + /* + * this is non-closed ledger, it should also be reported as + * URLedgersElapsedRecoveryGracePeriod + */ + ensembleSize = 3; + long ledgerId2 = 21234561L; + initMeta = LedgerMetadataBuilder.create() + .withId(ledgerId2) + .withEnsembleSize(ensembleSize) + .withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize) + .newEnsembleEntry(0L, + Arrays.asList(bookieAddresses.get(0), bookieAddresses.get(1), bookieAddresses.get(2))) + .newEnsembleEntry(100L, + Arrays.asList(bookieAddresses.get(3), bookieAddresses.get(1), bookieAddresses.get(2))) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(ledgerId2, initMeta).get(); + underreplicationManager.markLedgerUnderreplicated(ledgerId2, bookieAddresses.get(0).toString()); + if (timeElapsed) { + numOfURLedgersElapsedRecoveryGracePeriod++; + } + + /* + * this ledger is not marked underreplicated. + */ + long ledgerId3 = 31234561L; + initMeta = LedgerMetadataBuilder.create() + .withId(ledgerId3) + .withEnsembleSize(ensembleSize) + .withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize) + .newEnsembleEntry(0L, + Arrays.asList(bookieAddresses.get(1), bookieAddresses.get(2), bookieAddresses.get(3))) + .withClosedState() + .withLastEntryId(100) + .withLength(10000) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(ledgerId3, initMeta).get(); + + if (timeElapsed) { + /* + * in timeelapsed scenario, by waiting for + * underreplicatedLedgerRecoveryGracePeriod, recovery time must be + * elapsed. + */ + Thread.sleep((underreplicatedLedgerRecoveryGracePeriod + 1) * 1000); + } else { + /* + * in timeElapsed=false scenario, since + * underreplicatedLedgerRecoveryGracePeriod is set to some high + * value, there is no value in waiting. So just wait for some time + * and make sure urledgers are not reported as recoverytime elapsed + * urledgers. + */ + Thread.sleep(5000); + } + + ServerConfiguration servConf = new ServerConfiguration(confByIndex(0)); + servConf.setUnderreplicatedLedgerRecoveryGracePeriod(underreplicatedLedgerRecoveryGracePeriod); + setServerConfigPropertiesForRackPlacement(servConf); + MutableObject auditorRef = new MutableObject(); + try { + TestStatsLogger statsLogger = startAuditorAndWaitForPlacementPolicyCheck(servConf, auditorRef); + Gauge underreplicatedLedgersElapsedRecoveryGracePeriodGuage = statsLogger + .getGauge(ReplicationStats.NUM_UNDERREPLICATED_LEDGERS_ELAPSED_RECOVERY_GRACE_PERIOD); + assertEquals("NUM_UNDERREPLICATED_LEDGERS_ELAPSED_RECOVERY_GRACE_PERIOD guage value", + numOfURLedgersElapsedRecoveryGracePeriod, + underreplicatedLedgersElapsedRecoveryGracePeriodGuage.getSample()); + } finally { + Auditor auditor = auditorRef.getValue(); + if (auditor != null) { + auditor.close(); + } + regManager.close(); + } + } + + @Test + public void testPlacementPolicyCheckWithLedgersNotAdheringToPolicyWithMultipleSegments() throws Exception { + int numOfBookies = 7; + int numOfLedgersNotAdheringToPlacementPolicy = 0; + List bookieAddresses = new ArrayList<>(); + RegistrationManager regManager = driver.createRegistrationManager(); + for (int i = 0; i < numOfBookies; i++) { + BookieId bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181).toBookieId(); + bookieAddresses.add(bookieAddress); + regManager.registerBookie(bookieAddress, false, BookieServiceInfo.EMPTY); + } + + // only three racks + StaticDNSResolver.addNodeToRack("98.98.98.0", "/rack1"); + StaticDNSResolver.addNodeToRack("98.98.98.1", "/rack2"); + StaticDNSResolver.addNodeToRack("98.98.98.2", "/rack3"); + StaticDNSResolver.addNodeToRack("98.98.98.3", "/rack4"); + StaticDNSResolver.addNodeToRack("98.98.98.4", "/rack1"); + StaticDNSResolver.addNodeToRack("98.98.98.5", "/rack2"); + StaticDNSResolver.addNodeToRack("98.98.98.6", "/rack3"); + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerManager lm = mFactory.newLedgerManager(); + int ensembleSize = 5; + int writeQuorumSize = 5; + int ackQuorumSize = 2; + int minNumRacksPerWriteQuorumConfValue = 4; + + /* + * this closed ledger in each writeQuorumSize (5), there would be + * atleast minNumRacksPerWriteQuorumConfValue (4) racks. So it wont be + * counted as ledgers not adhering to placement policy. + */ + LedgerMetadata initMeta = LedgerMetadataBuilder.create() + .withId(1L) + .withEnsembleSize(ensembleSize) + .withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize) + .newEnsembleEntry(0L, bookieAddresses.subList(0, 5)) + .newEnsembleEntry(20L, bookieAddresses.subList(1, 6)) + .withClosedState() + .withLastEntryId(100) + .withLength(10000) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(1L, initMeta).get(); + + /* + * for the second segment bookies are from /rack1, /rack2 and /rack3, + * which is < minNumRacksPerWriteQuorumConfValue (4). So it is not + * adhering to placement policy. + * + * also for the third segment are from /rack1, /rack2 and /rack3, which + * is < minNumRacksPerWriteQuorumConfValue (4). So it is not adhering to + * placement policy. + * + * Though there are multiple segments are not adhering to placement + * policy, it should be counted as single ledger. + */ + initMeta = LedgerMetadataBuilder.create() + .withId(2L) + .withEnsembleSize(ensembleSize) + .withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize) + .newEnsembleEntry(0L, bookieAddresses.subList(0, 5)) + .newEnsembleEntry(20L, + Arrays.asList(bookieAddresses.get(0), bookieAddresses.get(1), bookieAddresses.get(2), + bookieAddresses.get(4), bookieAddresses.get(5))) + .newEnsembleEntry(40L, + Arrays.asList(bookieAddresses.get(0), bookieAddresses.get(1), bookieAddresses.get(2), + bookieAddresses.get(4), bookieAddresses.get(6))) + .withClosedState() + .withLastEntryId(100) + .withLength(10000) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(2L, initMeta).get(); + numOfLedgersNotAdheringToPlacementPolicy++; + + ServerConfiguration servConf = new ServerConfiguration(confByIndex(0)); + servConf.setMinNumRacksPerWriteQuorum(minNumRacksPerWriteQuorumConfValue); + setServerConfigPropertiesForRackPlacement(servConf); + MutableObject auditorRef = new MutableObject(); + try { + TestStatsLogger statsLogger = startAuditorAndWaitForPlacementPolicyCheck(servConf, auditorRef); + Gauge ledgersNotAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY); + assertEquals("NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY gauge value", + numOfLedgersNotAdheringToPlacementPolicy, ledgersNotAdheringToPlacementPolicyGuage.getSample()); + Gauge ledgersSoftlyAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY); + assertEquals("NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY gauge value", + 0, ledgersSoftlyAdheringToPlacementPolicyGuage.getSample()); + } finally { + Auditor auditor = auditorRef.getValue(); + if (auditor != null) { + auditor.close(); + } + regManager.close(); + } + } + + @Test + public void testZoneawarePlacementPolicyCheck() throws Exception { + int numOfBookies = 6; + int numOfLedgersNotAdheringToPlacementPolicy = 0; + int numOfLedgersSoftlyAdheringToPlacementPolicy = 0; + List bookieAddresses = new ArrayList(); + RegistrationManager regManager = driver.createRegistrationManager(); + /* + * 6 bookies - 3 zones and 2 uds + */ + for (int i = 0; i < numOfBookies; i++) { + BookieSocketAddress bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181); + bookieAddresses.add(bookieAddress.toBookieId()); + regManager.registerBookie(bookieAddress.toBookieId(), false, BookieServiceInfo.EMPTY); + String zone = "/zone" + (i % 3); + String upgradeDomain = "/ud" + (i % 2); + String networkLocation = zone + upgradeDomain; + StaticDNSResolver.addNodeToRack(bookieAddress.getHostName(), networkLocation); + } + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerManager lm = mFactory.newLedgerManager(); + + ServerConfiguration servConf = new ServerConfiguration(confByIndex(0)); + servConf.setDesiredNumZonesPerWriteQuorum(3); + servConf.setMinNumZonesPerWriteQuorum(2); + setServerConfigPropertiesForZonePlacement(servConf); + + /* + * this closed ledger adheres to ZoneAwarePlacementPolicy, since + * ensemble is spread across 3 zones and 2 UDs + */ + LedgerMetadata initMeta = LedgerMetadataBuilder.create() + .withId(1L) + .withEnsembleSize(6) + .withWriteQuorumSize(6) + .withAckQuorumSize(2) + .newEnsembleEntry(0L, bookieAddresses) + .withClosedState() + .withLastEntryId(100) + .withLength(10000) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(1L, initMeta).get(); + + /* + * this is non-closed ledger, so though ensemble is not adhering to + * placement policy (since ensemble is not multiple of writeQuorum), + * this shouldn't be reported + */ + initMeta = LedgerMetadataBuilder.create() + .withId(2L) + .withEnsembleSize(6) + .withWriteQuorumSize(5) + .withAckQuorumSize(2) + .newEnsembleEntry(0L, bookieAddresses) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(2L, initMeta).get(); + + /* + * this is closed ledger, since ensemble is not multiple of writeQuorum, + * this ledger is not adhering to placement policy. + */ + initMeta = LedgerMetadataBuilder.create() + .withId(3L) + .withEnsembleSize(6) + .withWriteQuorumSize(5) + .withAckQuorumSize(2) + .newEnsembleEntry(0L, bookieAddresses) + .withClosedState() + .withLastEntryId(100) + .withLength(10000) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(3L, initMeta).get(); + numOfLedgersNotAdheringToPlacementPolicy++; + + /* + * this closed ledger adheres softly to ZoneAwarePlacementPolicy, since + * ensemble/writeQuorum of size 4 has spread across just + * minNumZonesPerWriteQuorum (2). + */ + List newEnsemble = new ArrayList(); + newEnsemble.add(bookieAddresses.get(0)); + newEnsemble.add(bookieAddresses.get(1)); + newEnsemble.add(bookieAddresses.get(3)); + newEnsemble.add(bookieAddresses.get(4)); + initMeta = LedgerMetadataBuilder.create() + .withId(4L) + .withEnsembleSize(4) + .withWriteQuorumSize(4) + .withAckQuorumSize(2) + .newEnsembleEntry(0L, newEnsemble) + .withClosedState() + .withLastEntryId(100) + .withLength(10000) + .withDigestType(DigestType.DUMMY) + .withPassword(new byte[0]) + .build(); + lm.createLedgerMetadata(4L, initMeta).get(); + numOfLedgersSoftlyAdheringToPlacementPolicy++; + + MutableObject auditorRef = new MutableObject(); + try { + TestStatsLogger statsLogger = startAuditorAndWaitForPlacementPolicyCheck(servConf, auditorRef); + Gauge ledgersNotAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY); + assertEquals("NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY guage value", + numOfLedgersNotAdheringToPlacementPolicy, ledgersNotAdheringToPlacementPolicyGuage.getSample()); + Gauge ledgersSoftlyAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY); + assertEquals("NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY guage value", + numOfLedgersSoftlyAdheringToPlacementPolicy, + ledgersSoftlyAdheringToPlacementPolicyGuage.getSample()); + } finally { + Auditor auditor = auditorRef.getValue(); + if (auditor != null) { + auditor.close(); + } + regManager.close(); + } + } + + private void setServerConfigPropertiesForRackPlacement(ServerConfiguration servConf) { + setServerConfigProperties(servConf, RackawareEnsemblePlacementPolicy.class.getName()); + } + + private void setServerConfigPropertiesForZonePlacement(ServerConfiguration servConf) { + setServerConfigProperties(servConf, ZoneawareEnsemblePlacementPolicy.class.getName()); + } + + private void setServerConfigProperties(ServerConfiguration servConf, String ensemblePlacementPolicyClass) { + servConf.setProperty(REPP_DNS_RESOLVER_CLASS, StaticDNSResolver.class.getName()); + servConf.setProperty(ClientConfiguration.ENSEMBLE_PLACEMENT_POLICY, ensemblePlacementPolicyClass); + servConf.setAuditorPeriodicCheckInterval(0); + servConf.setAuditorPeriodicBookieCheckInterval(0); + servConf.setAuditorPeriodicReplicasCheckInterval(0); + servConf.setAuditorPeriodicPlacementPolicyCheckInterval(1000); + } + + private TestStatsLogger startAuditorAndWaitForPlacementPolicyCheck(ServerConfiguration servConf, + MutableObject auditorRef) throws MetadataException, CompatibilityException, KeeperException, + InterruptedException, UnavailableException, UnknownHostException { + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerUnderreplicationManager urm = mFactory.newLedgerUnderreplicationManager(); + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(AUDITOR_SCOPE); + TestOpStatsLogger placementPolicyCheckStatsLogger = (TestOpStatsLogger) statsLogger + .getOpStatsLogger(ReplicationStats.PLACEMENT_POLICY_CHECK_TIME); + + final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf, + statsLogger, null); + auditorRef.setValue(auditor); + CountDownLatch latch = auditor.getLatch(); + assertEquals("PLACEMENT_POLICY_CHECK_TIME SuccessCount", 0, placementPolicyCheckStatsLogger.getSuccessCount()); + urm.setPlacementPolicyCheckCTime(-1); + auditor.start(); + /* + * since placementPolicyCheckCTime is set to -1, placementPolicyCheck should be + * scheduled to run with no initialdelay + */ + assertTrue("placementPolicyCheck should have executed", latch.await(20, TimeUnit.SECONDS)); + for (int i = 0; i < 20; i++) { + Thread.sleep(100); + if (placementPolicyCheckStatsLogger.getSuccessCount() >= 1) { + break; + } + } + assertEquals("PLACEMENT_POLICY_CHECK_TIME SuccessCount", 1, placementPolicyCheckStatsLogger.getSuccessCount()); + return statsLogger; + } +} diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTaskTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTaskTest.java new file mode 100644 index 0000000000000..21dd2807b75d3 --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTaskTest.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.replication; + +import static org.apache.bookkeeper.replication.ReplicationStats.AUDITOR_SCOPE; +import static org.testng.AssertJUnit.assertEquals; +import java.util.LinkedList; +import java.util.List; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.bookkeeper.test.TestStatsProvider; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +/** + * Unit test {@link AuditorReplicasCheckTask}. + */ +public class AuditorReplicasCheckTaskTest extends BookKeeperClusterTestCase { + private static final Logger LOG = LoggerFactory + .getLogger(AuditorReplicasCheckTaskTest.class); + + private BookKeeperAdmin admin; + private LedgerManager ledgerManager; + private LedgerUnderreplicationManager ledgerUnderreplicationManager; + + public AuditorReplicasCheckTaskTest() throws Exception { + super(3); + baseConf.setPageLimit(1); + baseConf.setAutoRecoveryDaemonEnabled(false); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataClientDriver"); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataBookieDriver"); + } + + @BeforeMethod + @Override + public void setUp() throws Exception { + super.setUp(); + baseClientConf.setMetadataServiceUri( + metadataServiceUri.replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + final BookKeeper bookKeeper = new BookKeeper(baseClientConf); + admin = new BookKeeperAdmin(bookKeeper, NullStatsLogger.INSTANCE, new ClientConfiguration(baseClientConf)); + LedgerManagerFactory ledgerManagerFactory = bookKeeper.getLedgerManagerFactory(); + ledgerManager = ledgerManagerFactory.newLedgerManager(); + ledgerUnderreplicationManager = ledgerManagerFactory.newLedgerUnderreplicationManager(); + } + + @AfterMethod + @Override + public void tearDown() throws Exception { + if (ledgerManager != null) { + ledgerManager.close(); + } + if (ledgerUnderreplicationManager != null) { + ledgerUnderreplicationManager.close(); + } + if (admin != null) { + admin.close(); + } + super.tearDown(); + } + + @Test + public void testReplicasCheck() throws BKException, InterruptedException { + + // 1. create ledgers + final int numLedgers = 10; + List ids = new LinkedList(); + for (int i = 0; i < numLedgers; i++) { + LedgerHandle lh = bkc.createLedger(3, 3, BookKeeper.DigestType.CRC32, "passwd".getBytes()); + ids.add(lh.getId()); + for (int j = 0; j < 2; j++) { + lh.addEntry("testdata".getBytes()); + } + lh.close(); + } + + // 2. init auditorReplicasCheckTask + final TestStatsProvider statsProvider = new TestStatsProvider(); + final TestStatsProvider.TestStatsLogger statsLogger = statsProvider.getStatsLogger(AUDITOR_SCOPE); + final AuditorStats auditorStats = new AuditorStats(statsLogger); + AuditorReplicasCheckTask auditorReplicasCheckTask = new AuditorReplicasCheckTask( + baseConf, auditorStats, admin, ledgerManager, + ledgerUnderreplicationManager, null, (flag, throwable) -> flag.set(false)); + + // 3. replicasCheck + auditorReplicasCheckTask.runTask(); + + // 4. verify + assertEquals("REPLICAS_CHECK_TIME", 1, ((TestStatsProvider.TestOpStatsLogger) + statsLogger.getOpStatsLogger(ReplicationStats.REPLICAS_CHECK_TIME)).getSuccessCount()); + } +} diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java new file mode 100644 index 0000000000000..a4d6d86deced2 --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java @@ -0,0 +1,937 @@ +/* + * 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.replication; + +import static org.apache.bookkeeper.replication.ReplicationStats.AUDITOR_SCOPE; +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertTrue; +import java.net.URI; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Collections; +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.ExecutionException; +import java.util.concurrent.TimeUnit; +import org.apache.bookkeeper.bookie.BookieException; +import org.apache.bookkeeper.bookie.BookieImpl; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.BookKeeperAdmin; +import org.apache.bookkeeper.client.LedgerMetadataBuilder; +import org.apache.bookkeeper.client.api.DigestType; +import org.apache.bookkeeper.client.api.LedgerMetadata; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.discover.BookieServiceInfo; +import org.apache.bookkeeper.discover.RegistrationManager; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.meta.MetadataBookieDriver; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.meta.exceptions.MetadataException; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.replication.AuditorPeriodicCheckTest.TestAuditor; +import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; +import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; +import org.apache.bookkeeper.stats.Gauge; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.bookkeeper.test.TestStatsProvider; +import org.apache.bookkeeper.test.TestStatsProvider.TestOpStatsLogger; +import org.apache.bookkeeper.test.TestStatsProvider.TestStatsLogger; +import org.apache.bookkeeper.util.AvailabilityOfEntriesOfLedger; +import org.apache.bookkeeper.util.StaticDNSResolver; +import org.apache.commons.collections4.map.MultiKeyMap; +import org.apache.commons.lang3.mutable.MutableObject; +import org.apache.zookeeper.KeeperException; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +/** + * Tests the logic of Auditor's ReplicasCheck. + */ +public class AuditorReplicasCheckTest extends BookKeeperClusterTestCase { + private MetadataBookieDriver driver; + private RegistrationManager regManager; + + public AuditorReplicasCheckTest() throws Exception { + super(1); + baseConf.setPageLimit(1); // to make it easy to push ledger out of cache + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataClientDriver"); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataBookieDriver"); + } + + @BeforeMethod + @Override + public void setUp() throws Exception { + super.setUp(); + StaticDNSResolver.reset(); + + URI uri = URI.create(confByIndex(0).getMetadataServiceUri().replaceAll("zk://", "metadata-store:") + .replaceAll("/ledgers", "")); + driver = MetadataDrivers.getBookieDriver(uri); + ServerConfiguration serverConfiguration = new ServerConfiguration(confByIndex(0)); + serverConfiguration.setMetadataServiceUri( + serverConfiguration.getMetadataServiceUri().replaceAll("zk://", "metadata-store:") + .replaceAll("/ledgers", "")); + driver.initialize(serverConfiguration, NullStatsLogger.INSTANCE); + regManager = driver.createRegistrationManager(); + } + + @AfterMethod + @Override + public void tearDown() throws Exception { + if (null != regManager) { + regManager.close(); + } + if (null != driver) { + driver.close(); + } + super.tearDown(); + } + + private class TestBookKeeperAdmin extends BookKeeperAdmin { + + private final MultiKeyMap returnAvailabilityOfEntriesOfLedger; + private final MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger; + + public TestBookKeeperAdmin(BookKeeper bkc, StatsLogger statsLogger, + MultiKeyMap returnAvailabilityOfEntriesOfLedger, + MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger) { + super(bkc, statsLogger, baseClientConf); + this.returnAvailabilityOfEntriesOfLedger = returnAvailabilityOfEntriesOfLedger; + this.errorReturnValueForGetAvailabilityOfEntriesOfLedger = + errorReturnValueForGetAvailabilityOfEntriesOfLedger; + } + + @Override + public CompletableFuture asyncGetListOfEntriesOfLedger( + BookieId address, long ledgerId) { + CompletableFuture futureResult = + new CompletableFuture(); + Integer errorReturnValue = errorReturnValueForGetAvailabilityOfEntriesOfLedger.get(address.toString(), + Long.toString(ledgerId)); + if (errorReturnValue != null) { + futureResult.completeExceptionally(BKException.create(errorReturnValue).fillInStackTrace()); + } else { + AvailabilityOfEntriesOfLedger availabilityOfEntriesOfLedger = returnAvailabilityOfEntriesOfLedger + .get(address.toString(), Long.toString(ledgerId)); + futureResult.complete(availabilityOfEntriesOfLedger); + } + return futureResult; + } + } + + private TestStatsLogger startAuditorAndWaitForReplicasCheck(ServerConfiguration servConf, + MutableObject auditorRef, + MultiKeyMap expectedReturnAvailabilityOfEntriesOfLedger, + MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger) + throws MetadataException, CompatibilityException, KeeperException, InterruptedException, + UnavailableException, UnknownHostException { + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerUnderreplicationManager urm = mFactory.newLedgerUnderreplicationManager(); + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(AUDITOR_SCOPE); + TestOpStatsLogger replicasCheckStatsLogger = (TestOpStatsLogger) statsLogger + .getOpStatsLogger(ReplicationStats.REPLICAS_CHECK_TIME); + + final TestAuditor auditor = new TestAuditor(BookieImpl.getBookieId(servConf).toString(), servConf, bkc, true, + new TestBookKeeperAdmin(bkc, statsLogger, expectedReturnAvailabilityOfEntriesOfLedger, + errorReturnValueForGetAvailabilityOfEntriesOfLedger), + true, statsLogger, null); + auditorRef.setValue(auditor); + CountDownLatch latch = auditor.getLatch(); + assertEquals("REPLICAS_CHECK_TIME SuccessCount", 0, replicasCheckStatsLogger.getSuccessCount()); + urm.setReplicasCheckCTime(-1); + auditor.start(); + /* + * since replicasCheckCTime is set to -1, replicasCheck should be + * scheduled to run with no initialdelay + */ + assertTrue("replicasCheck should have executed", latch.await(20, TimeUnit.SECONDS)); + for (int i = 0; i < 200; i++) { + Thread.sleep(100); + if (replicasCheckStatsLogger.getSuccessCount() >= 1) { + break; + } + } + assertEquals("REPLICAS_CHECK_TIME SuccessCount", 1, replicasCheckStatsLogger.getSuccessCount()); + return statsLogger; + } + + private void setServerConfigProperties(ServerConfiguration servConf) { + servConf.setAuditorPeriodicCheckInterval(0); + servConf.setAuditorPeriodicBookieCheckInterval(0); + servConf.setAuditorPeriodicPlacementPolicyCheckInterval(0); + servConf.setAuditorPeriodicReplicasCheckInterval(1000); + } + + List addAndRegisterBookies(int numOfBookies) + throws BookieException { + BookieId bookieAddress; + List bookieAddresses = new ArrayList(); + for (int i = 0; i < numOfBookies; i++) { + bookieAddress = new BookieSocketAddress("98.98.98." + i, 2181).toBookieId(); + bookieAddresses.add(bookieAddress); + regManager.registerBookie(bookieAddress, false, BookieServiceInfo.EMPTY); + } + return bookieAddresses; + } + + private void createClosedLedgerMetadata(LedgerManager lm, long ledgerId, int ensembleSize, int writeQuorumSize, + int ackQuorumSize, Map> segmentEnsembles, long lastEntryId, int length, + DigestType digestType, byte[] password) throws InterruptedException, ExecutionException { + LedgerMetadataBuilder ledgerMetadataBuilder = LedgerMetadataBuilder.create(); + ledgerMetadataBuilder.withId(ledgerId).withEnsembleSize(ensembleSize).withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize).withClosedState().withLastEntryId(lastEntryId).withLength(length) + .withDigestType(digestType).withPassword(password); + for (Map.Entry> mapEntry : segmentEnsembles.entrySet()) { + ledgerMetadataBuilder.newEnsembleEntry(mapEntry.getKey(), mapEntry.getValue()); + } + LedgerMetadata initMeta = ledgerMetadataBuilder.build(); + lm.createLedgerMetadata(ledgerId, initMeta).get(); + } + + private void createNonClosedLedgerMetadata(LedgerManager lm, long ledgerId, int ensembleSize, int writeQuorumSize, + int ackQuorumSize, Map> segmentEnsembles, DigestType digestType, + byte[] password) throws InterruptedException, ExecutionException { + LedgerMetadataBuilder ledgerMetadataBuilder = LedgerMetadataBuilder.create(); + ledgerMetadataBuilder.withId(ledgerId).withEnsembleSize(ensembleSize).withWriteQuorumSize(writeQuorumSize) + .withAckQuorumSize(ackQuorumSize).withDigestType(digestType).withPassword(password); + for (Map.Entry> mapEntry : segmentEnsembles.entrySet()) { + ledgerMetadataBuilder.newEnsembleEntry(mapEntry.getKey(), mapEntry.getValue()); + } + LedgerMetadata initMeta = ledgerMetadataBuilder.build(); + lm.createLedgerMetadata(ledgerId, initMeta).get(); + } + + private void runTestScenario(MultiKeyMap returnAvailabilityOfEntriesOfLedger, + MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger, + int expectedNumLedgersFoundHavingNoReplicaOfAnEntry, + int expectedNumLedgersHavingLessThanAQReplicasOfAnEntry, + int expectedNumLedgersHavingLessThanWQReplicasOfAnEntry) throws Exception { + ServerConfiguration servConf = new ServerConfiguration(confByIndex(0)); + setServerConfigProperties(servConf); + MutableObject auditorRef = new MutableObject(); + try { + TestStatsLogger statsLogger = startAuditorAndWaitForReplicasCheck(servConf, auditorRef, + returnAvailabilityOfEntriesOfLedger, errorReturnValueForGetAvailabilityOfEntriesOfLedger); + checkReplicasCheckStats(statsLogger, expectedNumLedgersFoundHavingNoReplicaOfAnEntry, + expectedNumLedgersHavingLessThanAQReplicasOfAnEntry, + expectedNumLedgersHavingLessThanWQReplicasOfAnEntry); + } finally { + Auditor auditor = auditorRef.getValue(); + if (auditor != null) { + auditor.close(); + } + } + } + + private void checkReplicasCheckStats(TestStatsLogger statsLogger, + int expectedNumLedgersFoundHavingNoReplicaOfAnEntry, + int expectedNumLedgersHavingLessThanAQReplicasOfAnEntry, + int expectedNumLedgersHavingLessThanWQReplicasOfAnEntry) { + Gauge numLedgersFoundHavingNoReplicaOfAnEntryGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_HAVING_NO_REPLICA_OF_AN_ENTRY); + Gauge numLedgersHavingLessThanAQReplicasOfAnEntryGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_HAVING_LESS_THAN_AQ_REPLICAS_OF_AN_ENTRY); + Gauge numLedgersHavingLessThanWQReplicasOfAnEntryGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_HAVING_LESS_THAN_WQ_REPLICAS_OF_AN_ENTRY); + + assertEquals("NUM_LEDGERS_HAVING_NO_REPLICA_OF_AN_ENTRY guage value", + expectedNumLedgersFoundHavingNoReplicaOfAnEntry, + numLedgersFoundHavingNoReplicaOfAnEntryGuage.getSample()); + assertEquals("NUM_LEDGERS_HAVING_LESS_THAN_AQ_REPLICAS_OF_AN_ENTRY guage value", + expectedNumLedgersHavingLessThanAQReplicasOfAnEntry, + numLedgersHavingLessThanAQReplicasOfAnEntryGuage.getSample()); + assertEquals("NUM_LEDGERS_HAVING_LESS_THAN_WQ_REPLICAS_OF_AN_ENTRY guage value", + expectedNumLedgersHavingLessThanWQReplicasOfAnEntry, + numLedgersHavingLessThanWQReplicasOfAnEntryGuage.getSample()); + } + + /* + * For all the ledgers and for all the bookies, + * asyncGetListOfEntriesOfLedger would return + * BookieHandleNotAvailableException, so these ledgers wouldn't be counted + * against expectedNumLedgersFoundHavingNoReplicaOfAnEntry / + * LessThanAQReplicasOfAnEntry / LessThanWQReplicasOfAnEntry. + */ + @Test + public void testReplicasCheckForBookieHandleNotAvailable() throws Exception { + int numOfBookies = 5; + MultiKeyMap returnAvailabilityOfEntriesOfLedger = + new MultiKeyMap(); + MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger = + new MultiKeyMap(); + List bookieAddresses = addAndRegisterBookies(numOfBookies); + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerManager lm = mFactory.newLedgerManager(); + int ensembleSize = 5; + int writeQuorumSize = 4; + int ackQuorumSize = 2; + long lastEntryId = 100; + int length = 10000; + DigestType digestType = DigestType.DUMMY; + byte[] password = new byte[0]; + Collections.shuffle(bookieAddresses); + + /* + * closed ledger + * + * for this ledger, for all the bookies we are setting + * errorReturnValueForGetAvailabilityOfEntriesOfLedger to + * BookieHandleNotAvailableException so asyncGetListOfEntriesOfLedger will + * return BookieHandleNotAvailableException. + */ + Map> segmentEnsembles = new LinkedHashMap>(); + segmentEnsembles.put(0L, bookieAddresses); + long ledgerId = 1L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + for (BookieId bookieSocketAddress : bookieAddresses) { + errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), + Long.toString(ledgerId), BKException.Code.BookieHandleNotAvailableException); + } + + ensembleSize = 4; + /* + * closed ledger with multiple segments + * + * for this ledger, for all the bookies we are setting + * errorReturnValueForGetAvailabilityOfEntriesOfLedger to + * BookieHandleNotAvailableException so asyncGetListOfEntriesOfLedger will + * return BookieHandleNotAvailableException. + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(20L, bookieAddresses.subList(1, 5)); + segmentEnsembles.put(60L, bookieAddresses.subList(0, 4)); + ledgerId = 2L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + for (BookieId bookieSocketAddress : bookieAddresses) { + errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), + Long.toString(ledgerId), BKException.Code.BookieHandleNotAvailableException); + } + + /* + * non-closed ledger + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + ledgerId = 3L; + createNonClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + digestType, password); + for (BookieId bookieSocketAddress : bookieAddresses) { + errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), + Long.toString(ledgerId), BKException.Code.BookieHandleNotAvailableException); + } + + /* + * non-closed ledger with multiple segments + * + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(20L, bookieAddresses.subList(1, 5)); + segmentEnsembles.put(60L, bookieAddresses.subList(0, 4)); + ledgerId = 4L; + createNonClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + digestType, password); + for (BookieId bookieSocketAddress : bookieAddresses) { + errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), + Long.toString(ledgerId), BKException.Code.BookieHandleNotAvailableException); + } + + runTestScenario(returnAvailabilityOfEntriesOfLedger, errorReturnValueForGetAvailabilityOfEntriesOfLedger, 0, 0, + 0); + } + + /* + * In this testscenario all the ledgers have a missing entry. So all closed + * ledgers should be counted towards + * numLedgersFoundHavingNoReplicaOfAnEntry. + */ + @Test + public void testReplicasCheckForLedgersFoundHavingNoReplica() throws Exception { + int numOfBookies = 5; + MultiKeyMap returnAvailabilityOfEntriesOfLedger = + new MultiKeyMap(); + MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger = + new MultiKeyMap(); + List bookieAddresses = addAndRegisterBookies(numOfBookies); + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerManager lm = mFactory.newLedgerManager(); + int ensembleSize = 5; + int writeQuorumSize = 4; + int ackQuorumSize = 2; + long lastEntryId = 100; + int length = 10000; + DigestType digestType = DigestType.DUMMY; + byte[] password = new byte[0]; + Collections.shuffle(bookieAddresses); + + int numLedgersFoundHavingNoReplicaOfAnEntry = 0; + + /* + * closed ledger + * + * for this ledger we are setting returnAvailabilityOfEntriesOfLedger to + * Empty one for all of the bookies, so this ledger would be counted in + * ledgersFoundHavingNoReplicaOfAnEntry . + */ + Map> segmentEnsembles = new LinkedHashMap>(); + segmentEnsembles.put(0L, bookieAddresses); + long ledgerId = 1L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + for (BookieId bookieSocketAddress : bookieAddresses) { + returnAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), Long.toString(ledgerId), + AvailabilityOfEntriesOfLedger.EMPTY_AVAILABILITYOFENTRIESOFLEDGER); + } + numLedgersFoundHavingNoReplicaOfAnEntry++; + + ensembleSize = 4; + /* + * closed ledger with multiple segments + * + * for this ledger we are setting + * errorReturnValueForGetAvailabilityOfEntriesOfLedger to + * NoSuchLedgerExistsException. This is equivalent to + * EMPTY_AVAILABILITYOFENTRIESOFLEDGER. So this ledger would be counted + * in ledgersFoundHavingNoReplicaOfAnEntry + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(20L, bookieAddresses.subList(1, 5)); + segmentEnsembles.put(60L, bookieAddresses.subList(0, 4)); + ledgerId = 2L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + for (BookieId bookieSocketAddress : bookieAddresses) { + errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), + Long.toString(ledgerId), BKException.Code.NoSuchLedgerExistsException); + } + numLedgersFoundHavingNoReplicaOfAnEntry++; + + /* + * non-closed ledger + * + * since this is non-closed ledger, it should not be counted in + * ledgersFoundHavingNoReplicaOfAnEntry + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + ledgerId = 3L; + createNonClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + digestType, password); + for (BookieId bookieSocketAddress : bookieAddresses) { + returnAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), Long.toString(ledgerId), + AvailabilityOfEntriesOfLedger.EMPTY_AVAILABILITYOFENTRIESOFLEDGER); + } + + ensembleSize = 3; + writeQuorumSize = 3; + ackQuorumSize = 2; + lastEntryId = 1; + length = 1000; + /* + * closed ledger + * + * for this ledger we are setting returnAvailabilityOfEntriesOfLedger to + * just {0l} for all of the bookies and entry 1l is missing for all of + * the bookies, so this ledger would be counted in + * ledgersFoundHavingNoReplicaOfAnEntry + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 3)); + ledgerId = 4L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + for (BookieId bookieSocketAddress : bookieAddresses) { + returnAvailabilityOfEntriesOfLedger.put(bookieSocketAddress.toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0L })); + } + numLedgersFoundHavingNoReplicaOfAnEntry++; + + /* + * For this closed ledger, entry 1 is missing. So it should be counted + * towards numLedgersFoundHavingNoReplicaOfAnEntry. + */ + ensembleSize = 4; + writeQuorumSize = 3; + ackQuorumSize = 2; + lastEntryId = 3; + length = 10000; + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + ledgerId = 5L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 2, 3 })); + numLedgersFoundHavingNoReplicaOfAnEntry++; + + runTestScenario(returnAvailabilityOfEntriesOfLedger, errorReturnValueForGetAvailabilityOfEntriesOfLedger, + numLedgersFoundHavingNoReplicaOfAnEntry, 0, 0); + } + + /* + * In this testscenario all the ledgers have an entry with less than AQ + * number of copies. So all closed ledgers should be counted towards + * numLedgersFoundHavingLessThanAQReplicasOfAnEntry. + */ + @Test + public void testReplicasCheckForLedgersFoundHavingLessThanAQReplicasOfAnEntry() throws Exception { + int numOfBookies = 5; + MultiKeyMap returnAvailabilityOfEntriesOfLedger = + new MultiKeyMap(); + MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger = + new MultiKeyMap(); + List bookieAddresses = addAndRegisterBookies(numOfBookies); + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerManager lm = mFactory.newLedgerManager(); + DigestType digestType = DigestType.DUMMY; + byte[] password = new byte[0]; + Collections.shuffle(bookieAddresses); + + int numLedgersFoundHavingLessThanAQReplicasOfAnEntry = 0; + + /* + * closed ledger + * + * for this ledger there is only one copy of entry 2, so this ledger + * would be counted towards + * ledgersFoundHavingLessThanAQReplicasOfAnEntry. + */ + Map> segmentEnsembles = new LinkedHashMap>(); + int ensembleSize = 4; + int writeQuorumSize = 3; + int ackQuorumSize = 2; + long lastEntryId = 3; + int length = 10000; + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + long ledgerId = 1L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1, 2, 3 })); + numLedgersFoundHavingLessThanAQReplicasOfAnEntry++; + + /* + * closed ledger with multiple segments. + * + * for this ledger there is only one copy of entry 2, so this ledger + * would be counted towards + * ledgersFoundHavingLessThanAQReplicasOfAnEntry. + * + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(2L, bookieAddresses.subList(1, 5)); + ledgerId = 2L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] {})); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 2, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(4).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 3 })); + numLedgersFoundHavingLessThanAQReplicasOfAnEntry++; + + /* + * closed ledger with multiple segments + * + * for this ledger entry 2 is overrreplicated, but it has only one copy + * in the set of bookies it is supposed to be. So it should be counted + * towards ledgersFoundHavingLessThanAQReplicasOfAnEntry. + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(2L, bookieAddresses.subList(1, 5)); + ledgerId = 3L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 2 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 2, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(4).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 3 })); + numLedgersFoundHavingLessThanAQReplicasOfAnEntry++; + + /* + * non-closed ledger + * + * since this is non-closed ledger, it should not be counted towards + * ledgersFoundHavingLessThanAQReplicasOfAnEntry + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(2L, bookieAddresses.subList(1, 5)); + ledgerId = 4L; + createNonClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] {})); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 2, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(4).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 3 })); + + /* + * this is closed ledger. + * + * For third bookie, asyncGetListOfEntriesOfLedger will fail with + * BookieHandleNotAvailableException, so this should not be counted + * against missing copies of an entry. Other than that, for both entries + * 0 and 1, two copies are missing. Hence this should be counted towards + * numLedgersFoundHavingLessThanAQReplicasOfAnEntry. + */ + ensembleSize = 3; + writeQuorumSize = 3; + ackQuorumSize = 2; + lastEntryId = 1; + length = 1000; + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 3)); + ledgerId = 5L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + AvailabilityOfEntriesOfLedger.EMPTY_AVAILABILITYOFENTRIESOFLEDGER); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + AvailabilityOfEntriesOfLedger.EMPTY_AVAILABILITYOFENTRIESOFLEDGER); + errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), + Long.toString(ledgerId), BKException.Code.BookieHandleNotAvailableException); + numLedgersFoundHavingLessThanAQReplicasOfAnEntry++; + + runTestScenario(returnAvailabilityOfEntriesOfLedger, errorReturnValueForGetAvailabilityOfEntriesOfLedger, 0, + numLedgersFoundHavingLessThanAQReplicasOfAnEntry, 0); + } + + /* + * In this testscenario all the ledgers have an entry with less than WQ + * number of copies but greater than AQ. So all closed ledgers should be + * counted towards numLedgersFoundHavingLessThanWQReplicasOfAnEntry. + */ + @Test + public void testReplicasCheckForLedgersFoundHavingLessThanWQReplicasOfAnEntry() throws Exception { + int numOfBookies = 5; + MultiKeyMap returnAvailabilityOfEntriesOfLedger = + new MultiKeyMap(); + MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger = + new MultiKeyMap(); + List bookieAddresses = addAndRegisterBookies(numOfBookies); + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerManager lm = mFactory.newLedgerManager(); + DigestType digestType = DigestType.DUMMY; + byte[] password = new byte[0]; + Collections.shuffle(bookieAddresses); + + int numLedgersFoundHavingLessThanWQReplicasOfAnEntry = 0; + + /* + * closed ledger + * + * for this ledger a copy of entry 3, so this ledger would be counted + * towards ledgersFoundHavingLessThanWQReplicasOfAnEntry. + */ + Map> segmentEnsembles = new LinkedHashMap>(); + int ensembleSize = 4; + int writeQuorumSize = 3; + int ackQuorumSize = 2; + long lastEntryId = 3; + int length = 10000; + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + long ledgerId = 1L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 2 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1, 2, 3 })); + numLedgersFoundHavingLessThanWQReplicasOfAnEntry++; + + /* + * closed ledger with multiple segments + * + * for this ledger a copy of entry 0 and entry 2 are missing, so this + * ledger would be counted towards + * ledgersFoundHavingLessThanWQReplicasOfAnEntry. + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(2L, bookieAddresses.subList(1, 5)); + ledgerId = 2L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] {})); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 2, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(4).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 2, 3 })); + numLedgersFoundHavingLessThanWQReplicasOfAnEntry++; + + /* + * non-closed ledger with multiple segments + * + * since this is non-closed ledger, it should not be counted towards + * ledgersFoundHavingLessThanWQReplicasOfAnEntry + */ + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(2L, bookieAddresses.subList(1, 5)); + ledgerId = 3L; + createNonClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + digestType, password); + errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), + Long.toString(ledgerId), BKException.Code.NoSuchLedgerExistsException); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 2, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(4).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 2, 3 })); + + /* + * closed ledger. + * + * for this ledger entry 0 is overrreplicated, but a copy is missing in + * the set of bookies it is supposed to be. So it should be counted + * towards ledgersFoundHavingLessThanWQReplicasOfAnEntry. + */ + ensembleSize = 4; + writeQuorumSize = 3; + ackQuorumSize = 2; + lastEntryId = 1; + length = 1000; + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + ledgerId = 4L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 2, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1, 3 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0 })); + numLedgersFoundHavingLessThanWQReplicasOfAnEntry++; + + /* + * this is closed ledger. + * + * For third bookie, asyncGetListOfEntriesOfLedger will fail with + * BookieHandleNotAvailableException, so this should not be counted + * against missing copies of an entry. Other than that, for both entries + * 0 and 1, a copy is missing. Hence this should be counted towards + * numLedgersFoundHavingLessThanWQReplicasOfAnEntry. + */ + ensembleSize = 3; + writeQuorumSize = 3; + ackQuorumSize = 2; + lastEntryId = 1; + length = 1000; + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 3)); + ledgerId = 5L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + AvailabilityOfEntriesOfLedger.EMPTY_AVAILABILITYOFENTRIESOFLEDGER); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1 })); + errorReturnValueForGetAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), + Long.toString(ledgerId), BKException.Code.BookieHandleNotAvailableException); + numLedgersFoundHavingLessThanWQReplicasOfAnEntry++; + + runTestScenario(returnAvailabilityOfEntriesOfLedger, errorReturnValueForGetAvailabilityOfEntriesOfLedger, 0, 0, + numLedgersFoundHavingLessThanWQReplicasOfAnEntry); + } + + /* + * In this testscenario all the ledgers have empty segments. + */ + @Test + public void testReplicasCheckForLedgersWithEmptySegments() throws Exception { + int numOfBookies = 5; + MultiKeyMap returnAvailabilityOfEntriesOfLedger = + new MultiKeyMap(); + MultiKeyMap errorReturnValueForGetAvailabilityOfEntriesOfLedger = + new MultiKeyMap(); + List bookieAddresses = addAndRegisterBookies(numOfBookies); + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerManager lm = mFactory.newLedgerManager(); + DigestType digestType = DigestType.DUMMY; + byte[] password = new byte[0]; + Collections.shuffle(bookieAddresses); + + int numLedgersFoundHavingNoReplicaOfAnEntry = 0; + int numLedgersFoundHavingLessThanAQReplicasOfAnEntry = 0; + int numLedgersFoundHavingLessThanWQReplicasOfAnEntry = 0; + + /* + * closed ledger. + * + * This closed Ledger has no entry. So it should not be counted towards + * numLedgersFoundHavingNoReplicaOfAnEntry/LessThanAQReplicasOfAnEntry + * /WQReplicasOfAnEntry. + */ + Map> segmentEnsembles = new LinkedHashMap>(); + int ensembleSize = 4; + int writeQuorumSize = 3; + int ackQuorumSize = 2; + long lastEntryId = -1L; + int length = 0; + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + long ledgerId = 1L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + + /* + * closed ledger with multiple segments. + * + * This ledger has empty last segment, but all the entries have + * writeQuorumSize number of copies, So it should not be counted towards + * numLedgersFoundHavingNoReplicaOfAnEntry/LessThanAQReplicasOfAnEntry/ + * WQReplicasOfAnEntry. + */ + lastEntryId = 2; + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put((lastEntryId + 1), bookieAddresses.subList(1, 5)); + ledgerId = 2L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 2 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 2 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1, 2 })); + + /* + * Closed ledger with multiple segments. + * + * Segment0, Segment1, Segment3, Segment5 and Segment6 are empty. + * Entries from entryid 3 are missing. So it should be counted towards + * numLedgersFoundHavingNoReplicaOfAnEntry. + */ + lastEntryId = 5; + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(1, 5)); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(4L, bookieAddresses.subList(1, 5)); + segmentEnsembles.put(4L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put((lastEntryId + 1), bookieAddresses.subList(1, 5)); + segmentEnsembles.put((lastEntryId + 1), bookieAddresses.subList(0, 4)); + ledgerId = 3L; + createClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + lastEntryId, length, digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 2 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 2 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1, 2 })); + numLedgersFoundHavingNoReplicaOfAnEntry++; + + /* + * non-closed ledger with multiple segments + * + * since this is non-closed ledger, it should not be counted towards + * ledgersFoundHavingLessThanWQReplicasOfAnEntry + */ + lastEntryId = 2; + segmentEnsembles.clear(); + segmentEnsembles.put(0L, bookieAddresses.subList(0, 4)); + segmentEnsembles.put(0L, bookieAddresses.subList(1, 5)); + segmentEnsembles.put((lastEntryId + 1), bookieAddresses.subList(1, 5)); + ledgerId = 4L; + createNonClosedLedgerMetadata(lm, ledgerId, ensembleSize, writeQuorumSize, ackQuorumSize, segmentEnsembles, + digestType, password); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(0).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 2 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(1).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(2).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 0, 1, 2 })); + returnAvailabilityOfEntriesOfLedger.put(bookieAddresses.get(3).toString(), Long.toString(ledgerId), + new AvailabilityOfEntriesOfLedger(new long[] { 1, 2 })); + + runTestScenario(returnAvailabilityOfEntriesOfLedger, errorReturnValueForGetAvailabilityOfEntriesOfLedger, + numLedgersFoundHavingNoReplicaOfAnEntry, numLedgersFoundHavingLessThanAQReplicasOfAnEntry, + numLedgersFoundHavingLessThanWQReplicasOfAnEntry); + } +} diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorRollingRestartTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorRollingRestartTest.java new file mode 100644 index 0000000000000..2c458d635f528 --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorRollingRestartTest.java @@ -0,0 +1,112 @@ +/* + * 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.replication; + +import static org.apache.bookkeeper.meta.MetadataDrivers.runFunctionWithLedgerManagerFactory; +import static org.testng.AssertJUnit.assertEquals; +import com.google.common.util.concurrent.UncheckedExecutionException; +import lombok.Cleanup; +import org.apache.bookkeeper.client.BookKeeper.DigestType; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.LedgerAuditorManager; +import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.bookkeeper.test.TestCallbacks; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +/** + * Test auditor behaviours during a rolling restart. + */ +public class AuditorRollingRestartTest extends BookKeeperClusterTestCase { + + public AuditorRollingRestartTest() throws Exception { + super(3, 600); + // run the daemon within the bookie + setAutoRecoveryEnabled(true); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataClientDriver"); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataBookieDriver"); + } + + @BeforeMethod + @Override + public void setUp() throws Exception { + super.setUp(); + } + + @AfterMethod + @Override + public void tearDown() throws Exception { + super.tearDown(); + } + + @Override + protected void startBKCluster(String metadataServiceUri) throws Exception { + super.startBKCluster(metadataServiceUri.replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + } + + /** + * Test no auditing during restart if disabled. + */ + @Test + public void testAuditingDuringRollingRestart() throws Exception { + confByIndex(0).setMetadataServiceUri( + zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + runFunctionWithLedgerManagerFactory( + confByIndex(0), + mFactory -> { + try { + testAuditingDuringRollingRestart(mFactory); + } catch (Exception e) { + throw new UncheckedExecutionException(e.getMessage(), e); + } + return null; + } + ); + } + + private void testAuditingDuringRollingRestart(LedgerManagerFactory mFactory) throws Exception { + final LedgerUnderreplicationManager underReplicationManager = mFactory.newLedgerUnderreplicationManager(); + + LedgerHandle lh = bkc.createLedger(3, 3, DigestType.CRC32, "passwd".getBytes()); + for (int i = 0; i < 10; i++) { + lh.asyncAddEntry("foobar".getBytes(), new TestCallbacks.AddCallbackFuture(i), null); + } + lh.addEntry("foobar".getBytes()); + lh.close(); + + assertEquals("shouldn't be anything under replicated", + underReplicationManager.pollLedgerToRereplicate(), -1); + underReplicationManager.disableLedgerReplication(); + + @Cleanup + LedgerAuditorManager lam = mFactory.newLedgerAuditorManager(); + BookieId auditor = lam.getCurrentAuditor(); + ServerConfiguration conf = killBookie(auditor); + Thread.sleep(2000); + startBookie(conf); + Thread.sleep(2000); // give it time to run + assertEquals("shouldn't be anything under replicated", -1, + underReplicationManager.pollLedgerToRereplicate()); + } +} diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuthAutoRecoveryTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuthAutoRecoveryTest.java new file mode 100644 index 0000000000000..db338d1bb4b39 --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuthAutoRecoveryTest.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.replication; + +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertTrue; +import org.apache.bookkeeper.auth.AuthCallbacks; +import org.apache.bookkeeper.auth.AuthToken; +import org.apache.bookkeeper.auth.ClientAuthProvider; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.proto.ClientConnectionPeer; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +/** + * This test verifies the auditor bookie scenarios from the auth point-of-view. + */ +public class AuthAutoRecoveryTest extends BookKeeperClusterTestCase { + + private static final Logger LOG = LoggerFactory + .getLogger(AuthAutoRecoveryTest.class); + + public static final String TEST_AUTH_PROVIDER_PLUGIN_NAME = "TestAuthProviderPlugin"; + + private static String clientSideRole; + + private static class AuditorClientAuthInterceptorFactory + implements ClientAuthProvider.Factory { + + @Override + public String getPluginName() { + return TEST_AUTH_PROVIDER_PLUGIN_NAME; + } + + @Override + public void init(ClientConfiguration conf) { + clientSideRole = conf.getClientRole(); + } + + @Override + public ClientAuthProvider newProvider(ClientConnectionPeer addr, + final AuthCallbacks.GenericCallback completeCb) { + return new ClientAuthProvider() { + public void init(AuthCallbacks.GenericCallback cb) { + completeCb.operationComplete(BKException.Code.OK, null); + } + + public void process(AuthToken m, AuthCallbacks.GenericCallback cb) { + } + }; + } + } + + protected ServerConfiguration newServerConfiguration() throws Exception { + ServerConfiguration conf = super.newServerConfiguration(); + conf.setClientAuthProviderFactoryClass(AuditorClientAuthInterceptorFactory.class.getName()); + return conf; + } + + public AuthAutoRecoveryTest() { + super(6); + } + + @BeforeMethod + @Override + public void setUp() throws Exception { + super.setUp(); + } + + @AfterMethod + @Override + public void tearDown() throws Exception { + super.tearDown(); + } + + /* + * test the client role of the auditor + */ + @Test + public void testAuthClientRole() throws Exception { + ServerConfiguration config = confByIndex(0); + assertEquals(AuditorClientAuthInterceptorFactory.class.getName(), config.getClientAuthProviderFactoryClass()); + AutoRecoveryMain main = new AutoRecoveryMain(config); + try { + main.start(); + Thread.sleep(500); + assertTrue("AuditorElector should be running", + main.auditorElector.isRunning()); + assertTrue("Replication worker should be running", + main.replicationWorker.isRunning()); + } finally { + main.shutdown(); + } + assertEquals(ClientConfiguration.CLIENT_ROLE_SYSTEM, clientSideRole); + } + +} diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java index d12ee177ece69..62416968142b1 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java @@ -22,10 +22,10 @@ import static org.testng.AssertJUnit.assertFalse; import static org.testng.AssertJUnit.assertNotNull; import static org.testng.AssertJUnit.assertTrue; - import java.io.IOException; import java.lang.reflect.Field; import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.bookie.BookieImpl; import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.test.BookKeeperClusterTestCase; @@ -35,12 +35,14 @@ import org.apache.pulsar.metadata.impl.ZKMetadataStore; import org.apache.zookeeper.ZooKeeper; import org.awaitility.Awaitility; +import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; /** * Test the AuditorPeer. */ +@Slf4j public class AutoRecoveryMainTest extends BookKeeperClusterTestCase { public AutoRecoveryMainTest() throws Exception { @@ -55,12 +57,63 @@ public void setUp() throws Exception { super.setUp(); } + @AfterMethod + @Override + public void tearDown() throws Exception { + super.tearDown(); + } + + /** + * Test the startup of the auditorElector and RW. + */ + @Test + public void testStartup() throws Exception { + log.info("testStartup()"); + confByIndex(0).setMetadataServiceUri( + zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + AutoRecoveryMain main = new AutoRecoveryMain(confByIndex(0)); + try { + main.start(); + Thread.sleep(500); + assertTrue("AuditorElector should be running", + main.auditorElector.isRunning()); + assertTrue("Replication worker should be running", + main.replicationWorker.isRunning()); + } finally { + main.shutdown(); + } + } + + /* + * Test the shutdown of all daemons + */ + @Test + public void testShutdown() throws Exception { + log.info("testShutdown()"); + confByIndex(0).setMetadataServiceUri( + zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + AutoRecoveryMain main = new AutoRecoveryMain(confByIndex(0)); + main.start(); + Thread.sleep(500); + assertTrue("AuditorElector should be running", + main.auditorElector.isRunning()); + assertTrue("Replication worker should be running", + main.replicationWorker.isRunning()); + + main.shutdown(); + assertFalse("AuditorElector should not be running", + main.auditorElector.isRunning()); + assertFalse("Replication worker should not be running", + main.replicationWorker.isRunning()); + } + /** * Test that, if an autorecovery looses its ZK connection/session it will * shutdown. */ @Test public void testAutoRecoverySessionLoss() throws Exception { + log.info("testAutoRecoverySessionLoss()"); confByIndex(0).setMetadataServiceUri( zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); confByIndex(1).setMetadataServiceUri( diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java new file mode 100644 index 0000000000000..c8c76302b89e1 --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java @@ -0,0 +1,652 @@ +/* + * 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.replication; + +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertNotNull; +import static org.testng.AssertJUnit.assertNull; +import static org.testng.AssertJUnit.assertTrue; +import java.net.URI; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.SortedMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper.DigestType; +import org.apache.bookkeeper.client.BookKeeperTestClient; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.common.util.OrderedScheduler; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.meta.MetadataClientDriver; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.proto.BookieServer; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.Watcher.Event.EventType; +import org.apache.zookeeper.data.Stat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +/** + * Integration tests verifies the complete functionality of the + * Auditor-rereplication process: Auditor will publish the bookie failures, + * consequently ReplicationWorker will get the notifications and act on it. + */ +public class BookieAutoRecoveryTest extends BookKeeperClusterTestCase { + private static final Logger LOG = LoggerFactory + .getLogger(BookieAutoRecoveryTest.class); + private static final byte[] PASSWD = "admin".getBytes(); + private static final byte[] data = "TESTDATA".getBytes(); + private static final String openLedgerRereplicationGracePeriod = "3000"; // milliseconds + + private DigestType digestType; + private MetadataClientDriver metadataClientDriver; + private LedgerManagerFactory mFactory; + private LedgerUnderreplicationManager underReplicationManager; + private LedgerManager ledgerManager; + private OrderedScheduler scheduler; + + private final String underreplicatedPath = "/ledgers/underreplication/ledgers"; + + public BookieAutoRecoveryTest() throws Exception { + super(3); + + baseConf.setLedgerManagerFactoryClassName( + "org.apache.pulsar.metadata.bookkeeper.PulsarLedgerManagerFactory"); + baseConf.setOpenLedgerRereplicationGracePeriod(openLedgerRereplicationGracePeriod); + baseConf.setRwRereplicateBackoffMs(500); + baseClientConf.setLedgerManagerFactoryClassName( + "org.apache.pulsar.metadata.bookkeeper.PulsarLedgerManagerFactory"); + this.digestType = DigestType.MAC; + setAutoRecoveryEnabled(true); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataClientDriver"); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataBookieDriver"); + } + + @BeforeMethod + @Override + public void setUp() throws Exception { + super.setUp(); + baseConf.setMetadataServiceUri( + zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + baseClientConf.setMetadataServiceUri( + zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + + scheduler = OrderedScheduler.newSchedulerBuilder() + .name("test-scheduler") + .numThreads(1) + .build(); + + metadataClientDriver = MetadataDrivers.getClientDriver( + URI.create(baseClientConf.getMetadataServiceUri())); + metadataClientDriver.initialize( + baseClientConf, + scheduler, + NullStatsLogger.INSTANCE, + Optional.empty()); + + // initialize urReplicationManager + mFactory = metadataClientDriver.getLedgerManagerFactory(); + underReplicationManager = mFactory.newLedgerUnderreplicationManager(); + ledgerManager = mFactory.newLedgerManager(); + } + + @AfterMethod + @Override + public void tearDown() throws Exception { + super.tearDown(); + + if (null != underReplicationManager) { + underReplicationManager.close(); + underReplicationManager = null; + } + if (null != ledgerManager) { + ledgerManager.close(); + ledgerManager = null; + } + if (null != metadataClientDriver) { + metadataClientDriver.close(); + metadataClientDriver = null; + } + if (null != scheduler) { + scheduler.shutdown(); + } + } + + /** + * Test verifies publish urLedger by Auditor and replication worker is + * picking up the entries and finishing the rereplication of open ledger. + */ + @Test + public void testOpenLedgers() throws Exception { + List listOfLedgerHandle = createLedgersAndAddEntries(1, 5); + LedgerHandle lh = listOfLedgerHandle.get(0); + int ledgerReplicaIndex = 0; + BookieId replicaToKillAddr = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + + final String urLedgerZNode = getUrLedgerZNode(lh); + ledgerReplicaIndex = getReplicaIndexInLedger(lh, replicaToKillAddr); + + CountDownLatch latch = new CountDownLatch(1); + assertNull("UrLedger already exists!", + watchUrLedgerNode(urLedgerZNode, latch)); + + LOG.info("Killing Bookie :" + replicaToKillAddr); + killBookie(replicaToKillAddr); + + // waiting to publish urLedger znode by Auditor + latch.await(); + latch = new CountDownLatch(1); + LOG.info("Watching on urLedgerPath:" + urLedgerZNode + + " to know the status of rereplication process"); + assertNotNull("UrLedger doesn't exists!", + watchUrLedgerNode(urLedgerZNode, latch)); + + // starting the replication service, so that he will be able to act as + // target bookie + startNewBookie(); + int newBookieIndex = lastBookieIndex(); + BookieServer newBookieServer = serverByIndex(newBookieIndex); + + if (LOG.isDebugEnabled()) { + LOG.debug("Waiting to finish the replication of failed bookie : " + + replicaToKillAddr); + } + latch.await(); + + // grace period to update the urledger metadata in zookeeper + LOG.info("Waiting to update the urledger metadata in zookeeper"); + + verifyLedgerEnsembleMetadataAfterReplication(newBookieServer, + listOfLedgerHandle.get(0), ledgerReplicaIndex); + } + + /** + * Test verifies publish urLedger by Auditor and replication worker is + * picking up the entries and finishing the rereplication of closed ledgers. + */ + @Test + public void testClosedLedgers() throws Exception { + List listOfReplicaIndex = new ArrayList(); + List listOfLedgerHandle = createLedgersAndAddEntries(1, 5); + closeLedgers(listOfLedgerHandle); + LedgerHandle lhandle = listOfLedgerHandle.get(0); + int ledgerReplicaIndex = 0; + BookieId replicaToKillAddr = lhandle.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + + CountDownLatch latch = new CountDownLatch(listOfLedgerHandle.size()); + for (LedgerHandle lh : listOfLedgerHandle) { + ledgerReplicaIndex = getReplicaIndexInLedger(lh, replicaToKillAddr); + listOfReplicaIndex.add(ledgerReplicaIndex); + assertNull("UrLedger already exists!", + watchUrLedgerNode(getUrLedgerZNode(lh), latch)); + } + + LOG.info("Killing Bookie :" + replicaToKillAddr); + killBookie(replicaToKillAddr); + + // waiting to publish urLedger znode by Auditor + latch.await(); + + // Again watching the urLedger znode to know the replication status + latch = new CountDownLatch(listOfLedgerHandle.size()); + for (LedgerHandle lh : listOfLedgerHandle) { + String urLedgerZNode = getUrLedgerZNode(lh); + LOG.info("Watching on urLedgerPath:" + urLedgerZNode + + " to know the status of rereplication process"); + assertNotNull("UrLedger doesn't exists!", + watchUrLedgerNode(urLedgerZNode, latch)); + } + + // starting the replication service, so that he will be able to act as + // target bookie + startNewBookie(); + int newBookieIndex = lastBookieIndex(); + BookieServer newBookieServer = serverByIndex(newBookieIndex); + + if (LOG.isDebugEnabled()) { + LOG.debug("Waiting to finish the replication of failed bookie : " + + replicaToKillAddr); + } + + // waiting to finish replication + latch.await(); + + // grace period to update the urledger metadata in zookeeper + LOG.info("Waiting to update the urledger metadata in zookeeper"); + + for (int index = 0; index < listOfLedgerHandle.size(); index++) { + verifyLedgerEnsembleMetadataAfterReplication(newBookieServer, + listOfLedgerHandle.get(index), + listOfReplicaIndex.get(index)); + } + } + + /** + * Test stopping replica service while replication in progress. Considering + * when there is an exception will shutdown Auditor and RW processes. After + * restarting should be able to finish the re-replication activities + */ + @Test + public void testStopWhileReplicationInProgress() throws Exception { + int numberOfLedgers = 2; + List listOfReplicaIndex = new ArrayList(); + List listOfLedgerHandle = createLedgersAndAddEntries( + numberOfLedgers, 5); + closeLedgers(listOfLedgerHandle); + LedgerHandle handle = listOfLedgerHandle.get(0); + BookieId replicaToKillAddr = handle.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + LOG.info("Killing Bookie:" + replicaToKillAddr); + + // Each ledger, there will be two events : create urLedger and after + // rereplication delete urLedger + CountDownLatch latch = new CountDownLatch(listOfLedgerHandle.size()); + for (int i = 0; i < listOfLedgerHandle.size(); i++) { + final String urLedgerZNode = getUrLedgerZNode(listOfLedgerHandle + .get(i)); + assertNull("UrLedger already exists!", + watchUrLedgerNode(urLedgerZNode, latch)); + int replicaIndexInLedger = getReplicaIndexInLedger( + listOfLedgerHandle.get(i), replicaToKillAddr); + listOfReplicaIndex.add(replicaIndexInLedger); + } + + LOG.info("Killing Bookie :" + replicaToKillAddr); + killBookie(replicaToKillAddr); + + // waiting to publish urLedger znode by Auditor + latch.await(); + + // Again watching the urLedger znode to know the replication status + latch = new CountDownLatch(listOfLedgerHandle.size()); + for (LedgerHandle lh : listOfLedgerHandle) { + String urLedgerZNode = getUrLedgerZNode(lh); + LOG.info("Watching on urLedgerPath:" + urLedgerZNode + + " to know the status of rereplication process"); + assertNotNull("UrLedger doesn't exists!", + watchUrLedgerNode(urLedgerZNode, latch)); + } + + // starting the replication service, so that he will be able to act as + // target bookie + startNewBookie(); + int newBookieIndex = lastBookieIndex(); + BookieServer newBookieServer = serverByIndex(newBookieIndex); + + if (LOG.isDebugEnabled()) { + LOG.debug("Waiting to finish the replication of failed bookie : " + + replicaToKillAddr); + } + while (true) { + if (latch.getCount() < numberOfLedgers || latch.getCount() <= 0) { + stopReplicationService(); + LOG.info("Latch Count is:" + latch.getCount()); + break; + } + // grace period to take breath + Thread.sleep(1000); + } + + startReplicationService(); + + LOG.info("Waiting to finish rereplication processes"); + latch.await(); + + // grace period to update the urledger metadata in zookeeper + LOG.info("Waiting to update the urledger metadata in zookeeper"); + + for (int index = 0; index < listOfLedgerHandle.size(); index++) { + verifyLedgerEnsembleMetadataAfterReplication(newBookieServer, + listOfLedgerHandle.get(index), + listOfReplicaIndex.get(index)); + } + } + + /** + * Verify the published urledgers of deleted ledgers(those ledgers where + * deleted after publishing as urledgers by Auditor) should be cleared off + * by the newly selected replica bookie. + */ + @Test + public void testNoSuchLedgerExists() throws Exception { + List listOfLedgerHandle = createLedgersAndAddEntries(2, 5); + CountDownLatch latch = new CountDownLatch(listOfLedgerHandle.size()); + for (LedgerHandle lh : listOfLedgerHandle) { + assertNull("UrLedger already exists!", + watchUrLedgerNode(getUrLedgerZNode(lh), latch)); + } + BookieId replicaToKillAddr = listOfLedgerHandle.get(0) + .getLedgerMetadata().getAllEnsembles() + .get(0L).get(0); + killBookie(replicaToKillAddr); + replicaToKillAddr = listOfLedgerHandle.get(0) + .getLedgerMetadata().getAllEnsembles() + .get(0L).get(0); + killBookie(replicaToKillAddr); + // waiting to publish urLedger znode by Auditor + latch.await(); + + latch = new CountDownLatch(listOfLedgerHandle.size()); + for (LedgerHandle lh : listOfLedgerHandle) { + assertNotNull("UrLedger doesn't exists!", + watchUrLedgerNode(getUrLedgerZNode(lh), latch)); + } + + // delete ledgers + for (LedgerHandle lh : listOfLedgerHandle) { + bkc.deleteLedger(lh.getId()); + } + startNewBookie(); + + // waiting to delete published urledgers, since it doesn't exists + latch.await(); + + for (LedgerHandle lh : listOfLedgerHandle) { + assertNull("UrLedger still exists after rereplication", + watchUrLedgerNode(getUrLedgerZNode(lh), latch)); + } + } + + /** + * Test that if a empty ledger loses the bookie not in the quorum for entry 0, it will + * still be openable when it loses enough bookies to lose a whole quorum. + */ + @Test + public void testEmptyLedgerLosesQuorumEventually() throws Exception { + LedgerHandle lh = bkc.createLedger(3, 2, 2, DigestType.CRC32, PASSWD); + CountDownLatch latch = new CountDownLatch(1); + String urZNode = getUrLedgerZNode(lh); + watchUrLedgerNode(urZNode, latch); + + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(2); + LOG.info("Killing last bookie, {}, in ensemble {}", replicaToKill, + lh.getLedgerMetadata().getAllEnsembles().get(0L)); + killBookie(replicaToKill); + startNewBookie(); + + getAuditor(10, TimeUnit.SECONDS).submitAuditTask().get(); // ensure auditor runs + + assertTrue("Should be marked as underreplicated", latch.await(5, TimeUnit.SECONDS)); + latch = new CountDownLatch(1); + Stat s = watchUrLedgerNode(urZNode, latch); // should be marked as replicated + if (s != null) { + assertTrue("Should be marked as replicated", latch.await(15, TimeUnit.SECONDS)); + } + + replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(1); + LOG.info("Killing second bookie, {}, in ensemble {}", replicaToKill, + lh.getLedgerMetadata().getAllEnsembles().get(0L)); + killBookie(replicaToKill); + + getAuditor(10, TimeUnit.SECONDS).submitAuditTask().get(); // ensure auditor runs + + assertTrue("Should be marked as underreplicated", latch.await(5, TimeUnit.SECONDS)); + latch = new CountDownLatch(1); + s = watchUrLedgerNode(urZNode, latch); // should be marked as replicated + + startNewBookie(); + getAuditor(10, TimeUnit.SECONDS).submitAuditTask().get(); // ensure auditor runs + + if (s != null) { + assertTrue("Should be marked as replicated", latch.await(20, TimeUnit.SECONDS)); + } + + // should be able to open ledger without issue + bkc.openLedger(lh.getId(), DigestType.CRC32, PASSWD); + } + + /** + * Test verifies bookie recovery, the host (recorded via ipaddress in + * ledgermetadata). + */ + @Test + public void testLedgerMetadataContainsIpAddressAsBookieID() + throws Exception { + stopBKCluster(); + bkc = new BookKeeperTestClient(baseClientConf); + // start bookie with useHostNameAsBookieID=false, as old bookie + ServerConfiguration serverConf1 = newServerConfiguration(); + // start 2 more bookies with useHostNameAsBookieID=true + ServerConfiguration serverConf2 = newServerConfiguration(); + serverConf2.setUseHostNameAsBookieID(true); + ServerConfiguration serverConf3 = newServerConfiguration(); + serverConf3.setUseHostNameAsBookieID(true); + startAndAddBookie(serverConf1); + startAndAddBookie(serverConf2); + startAndAddBookie(serverConf3); + + List listOfLedgerHandle = createLedgersAndAddEntries(1, 5); + LedgerHandle lh = listOfLedgerHandle.get(0); + int ledgerReplicaIndex = 0; + final SortedMap> ensembles = lh.getLedgerMetadata().getAllEnsembles(); + final List bkAddresses = ensembles.get(0L); + BookieId replicaToKillAddr = bkAddresses.get(0); + for (BookieId bookieSocketAddress : bkAddresses) { + if (!isCreatedFromIp(bookieSocketAddress)) { + replicaToKillAddr = bookieSocketAddress; + LOG.info("Kill bookie which has registered using hostname"); + break; + } + } + + final String urLedgerZNode = getUrLedgerZNode(lh); + ledgerReplicaIndex = getReplicaIndexInLedger(lh, replicaToKillAddr); + + CountDownLatch latch = new CountDownLatch(1); + assertNull("UrLedger already exists!", + watchUrLedgerNode(urLedgerZNode, latch)); + + LOG.info("Killing Bookie :" + replicaToKillAddr); + killBookie(replicaToKillAddr); + + // waiting to publish urLedger znode by Auditor + latch.await(); + latch = new CountDownLatch(1); + LOG.info("Watching on urLedgerPath:" + urLedgerZNode + + " to know the status of rereplication process"); + assertNotNull("UrLedger doesn't exists!", + watchUrLedgerNode(urLedgerZNode, latch)); + + // starting the replication service, so that he will be able to act as + // target bookie + ServerConfiguration serverConf = newServerConfiguration(); + serverConf.setUseHostNameAsBookieID(false); + startAndAddBookie(serverConf); + + int newBookieIndex = lastBookieIndex(); + BookieServer newBookieServer = serverByIndex(newBookieIndex); + + if (LOG.isDebugEnabled()) { + LOG.debug("Waiting to finish the replication of failed bookie : " + + replicaToKillAddr); + } + latch.await(); + + // grace period to update the urledger metadata in zookeeper + LOG.info("Waiting to update the urledger metadata in zookeeper"); + + verifyLedgerEnsembleMetadataAfterReplication(newBookieServer, + listOfLedgerHandle.get(0), ledgerReplicaIndex); + + } + + /** + * Test verifies bookie recovery, the host (recorded via useHostName in + * ledgermetadata). + */ + @Test + public void testLedgerMetadataContainsHostNameAsBookieID() + throws Exception { + stopBKCluster(); + + bkc = new BookKeeperTestClient(baseClientConf); + // start bookie with useHostNameAsBookieID=false, as old bookie + ServerConfiguration serverConf1 = newServerConfiguration(); + // start 2 more bookies with useHostNameAsBookieID=true + ServerConfiguration serverConf2 = newServerConfiguration(); + serverConf2.setUseHostNameAsBookieID(true); + ServerConfiguration serverConf3 = newServerConfiguration(); + serverConf3.setUseHostNameAsBookieID(true); + startAndAddBookie(serverConf1); + startAndAddBookie(serverConf2); + startAndAddBookie(serverConf3); + + List listOfLedgerHandle = createLedgersAndAddEntries(1, 5); + LedgerHandle lh = listOfLedgerHandle.get(0); + int ledgerReplicaIndex = 0; + final SortedMap> ensembles = lh.getLedgerMetadata().getAllEnsembles(); + final List bkAddresses = ensembles.get(0L); + BookieId replicaToKillAddr = bkAddresses.get(0); + for (BookieId bookieSocketAddress : bkAddresses) { + if (isCreatedFromIp(bookieSocketAddress)) { + replicaToKillAddr = bookieSocketAddress; + LOG.info("Kill bookie which has registered using ipaddress"); + break; + } + } + + final String urLedgerZNode = getUrLedgerZNode(lh); + ledgerReplicaIndex = getReplicaIndexInLedger(lh, replicaToKillAddr); + + CountDownLatch latch = new CountDownLatch(1); + assertNull("UrLedger already exists!", + watchUrLedgerNode(urLedgerZNode, latch)); + + LOG.info("Killing Bookie :" + replicaToKillAddr); + killBookie(replicaToKillAddr); + + // waiting to publish urLedger znode by Auditor + latch.await(); + latch = new CountDownLatch(1); + LOG.info("Watching on urLedgerPath:" + urLedgerZNode + + " to know the status of rereplication process"); + assertNotNull("UrLedger doesn't exists!", + watchUrLedgerNode(urLedgerZNode, latch)); + + // creates new bkclient + bkc = new BookKeeperTestClient(baseClientConf); + // starting the replication service, so that he will be able to act as + // target bookie + ServerConfiguration serverConf = newServerConfiguration(); + serverConf.setUseHostNameAsBookieID(true); + startAndAddBookie(serverConf); + + int newBookieIndex = lastBookieIndex(); + BookieServer newBookieServer = serverByIndex(newBookieIndex); + + if (LOG.isDebugEnabled()) { + LOG.debug("Waiting to finish the replication of failed bookie : " + + replicaToKillAddr); + } + latch.await(); + + // grace period to update the urledger metadata in zookeeper + LOG.info("Waiting to update the urledger metadata in zookeeper"); + + verifyLedgerEnsembleMetadataAfterReplication(newBookieServer, + listOfLedgerHandle.get(0), ledgerReplicaIndex); + + } + + private int getReplicaIndexInLedger(LedgerHandle lh, BookieId replicaToKill) { + SortedMap> ensembles = lh.getLedgerMetadata().getAllEnsembles(); + int ledgerReplicaIndex = -1; + for (BookieId addr : ensembles.get(0L)) { + ++ledgerReplicaIndex; + if (addr.equals(replicaToKill)) { + break; + } + } + return ledgerReplicaIndex; + } + + private void verifyLedgerEnsembleMetadataAfterReplication( + BookieServer newBookieServer, LedgerHandle lh, + int ledgerReplicaIndex) throws Exception { + LedgerHandle openLedger = bkc + .openLedger(lh.getId(), digestType, PASSWD); + + BookieId inetSocketAddress = openLedger.getLedgerMetadata().getAllEnsembles().get(0L) + .get(ledgerReplicaIndex); + assertEquals("Rereplication has been failed and ledgerReplicaIndex :" + + ledgerReplicaIndex, newBookieServer.getBookieId(), + inetSocketAddress); + openLedger.close(); + } + + private void closeLedgers(List listOfLedgerHandle) + throws InterruptedException, BKException { + for (LedgerHandle lh : listOfLedgerHandle) { + lh.close(); + } + } + + private List createLedgersAndAddEntries(int numberOfLedgers, + int numberOfEntries) + throws InterruptedException, BKException { + List listOfLedgerHandle = new ArrayList( + numberOfLedgers); + for (int index = 0; index < numberOfLedgers; index++) { + LedgerHandle lh = bkc.createLedger(3, 3, digestType, PASSWD); + listOfLedgerHandle.add(lh); + for (int i = 0; i < numberOfEntries; i++) { + lh.addEntry(data); + } + } + return listOfLedgerHandle; + } + + private String getUrLedgerZNode(LedgerHandle lh) { + return ZkLedgerUnderreplicationManager.getUrLedgerZnode( + underreplicatedPath, lh.getId()); + } + + private Stat watchUrLedgerNode(final String znode, + final CountDownLatch latch) throws KeeperException, + InterruptedException { + return zkc.exists(znode, new Watcher() { + @Override + public void process(WatchedEvent event) { + if (event.getType() == EventType.NodeDeleted) { + LOG.info("Received Ledger rereplication completion event :" + + event.getType()); + latch.countDown(); + } + if (event.getType() == EventType.NodeCreated) { + LOG.info("Received urLedger publishing event :" + + event.getType()); + latch.countDown(); + } + } + }); + } +} diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookieLedgerIndexTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookieLedgerIndexTest.java new file mode 100644 index 0000000000000..eb9f95ffdf7a5 --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookieLedgerIndexTest.java @@ -0,0 +1,248 @@ +/* + * 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.replication; + +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertTrue; +import static org.testng.AssertJUnit.fail; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper.DigestType; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.meta.LayoutManager; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.pulsar.metadata.api.MetadataStoreConfig; +import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; +import org.apache.pulsar.metadata.bookkeeper.PulsarLayoutManager; +import org.apache.pulsar.metadata.bookkeeper.PulsarLedgerManagerFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +/** + * Tests verifies bookie vs ledger mapping generating by the BookieLedgerIndexer. + */ +public class BookieLedgerIndexTest extends BookKeeperClusterTestCase { + + // Depending on the taste, select the amount of logging + // by decommenting one of the two lines below + // private final static Logger LOG = Logger.getRootLogger(); + private static final Logger LOG = LoggerFactory + .getLogger(BookieLedgerIndexTest.class); + + private Random rng; // Random Number Generator + private ArrayList entries; // generated entries + private final DigestType digestType = DigestType.CRC32; + private int numberOfLedgers = 3; + private List ledgerList; + private LedgerManagerFactory newLedgerManagerFactory; + private LedgerManager ledgerManager; + + public BookieLedgerIndexTest() throws Exception { + this("org.apache.pulsar.metadata.bookkeeper.PulsarLedgerManagerFactory"); + } + + BookieLedgerIndexTest(String ledgerManagerFactory) throws Exception { + super(3); + LOG.info("Running test case using ledger manager : " + + ledgerManagerFactory); + // set ledger manager name + baseConf.setLedgerManagerFactoryClassName(ledgerManagerFactory); + baseClientConf.setLedgerManagerFactoryClassName(ledgerManagerFactory); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataClientDriver"); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataBookieDriver"); + } + + @BeforeMethod + public void setUp() throws Exception { + super.setUp(); + baseConf.setMetadataServiceUri( + zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + rng = new Random(System.currentTimeMillis()); // Initialize the Random + // Number Generator + entries = new ArrayList(); // initialize the entries list + ledgerList = new ArrayList(3); + + String ledgersRoot = "/ledgers"; + String storeUri = metadataServiceUri.replaceAll("zk://", "").replaceAll("/ledgers", ""); + MetadataStoreExtended store = MetadataStoreExtended.create(storeUri, + MetadataStoreConfig.builder().fsyncEnable(false).build()); + LayoutManager layoutManager = new PulsarLayoutManager(store, ledgersRoot); + newLedgerManagerFactory = new PulsarLedgerManagerFactory(); + + ClientConfiguration conf = new ClientConfiguration(); + conf.setZkLedgersRootPath(ledgersRoot); + newLedgerManagerFactory.initialize(conf, layoutManager, 1); + ledgerManager = newLedgerManagerFactory.newLedgerManager(); + } + + @AfterMethod + public void tearDown() throws Exception { + super.tearDown(); + if (null != newLedgerManagerFactory) { + newLedgerManagerFactory.close(); + newLedgerManagerFactory = null; + } + if (null != ledgerManager) { + ledgerManager.close(); + ledgerManager = null; + } + } + + /** + * Verify the bookie-ledger mapping with minimum number of bookies and few + * ledgers. + */ + @Test + public void testSimpleBookieLedgerMapping() throws Exception { + + for (int i = 0; i < numberOfLedgers; i++) { + createAndAddEntriesToLedger().close(); + } + + BookieLedgerIndexer bookieLedgerIndex = new BookieLedgerIndexer( + ledgerManager); + + Map> bookieToLedgerIndex = bookieLedgerIndex + .getBookieToLedgerIndex(); + + assertEquals("Missed few bookies in the bookie-ledger mapping!", 3, + bookieToLedgerIndex.size()); + Collection> bk2ledgerEntry = bookieToLedgerIndex.values(); + for (Set ledgers : bk2ledgerEntry) { + assertEquals("Missed few ledgers in the bookie-ledger mapping!", 3, + ledgers.size()); + for (Long ledgerId : ledgers) { + assertTrue("Unknown ledger-bookie mapping", ledgerList + .contains(ledgerId)); + } + } + } + + /** + * Verify ledger index with failed bookies and throws exception. + */ + @SuppressWarnings("deprecation") +// @Test +// public void testWithoutZookeeper() throws Exception { +// // This test case is for ledger metadata that stored in ZooKeeper. As +// // far as MSLedgerManagerFactory, ledger metadata are stored in other +// // storage. So this test is not suitable for MSLedgerManagerFactory. +// if (newLedgerManagerFactory instanceof org.apache.bookkeeper.meta.MSLedgerManagerFactory) { +// return; +// } +// +// for (int i = 0; i < numberOfLedgers; i++) { +// createAndAddEntriesToLedger().close(); +// } +// +// BookieLedgerIndexer bookieLedgerIndex = new BookieLedgerIndexer( +// ledgerManager); +// stopZKCluster(); +// try { +// bookieLedgerIndex.getBookieToLedgerIndex(); +// fail("Must throw exception as zookeeper are not running!"); +// } catch (BKAuditException bkAuditException) { +// // expected behaviour +// } +// } + + /** + * Verify indexing with multiple ensemble reformation. + */ + @Test + public void testEnsembleReformation() throws Exception { + try { + LedgerHandle lh1 = createAndAddEntriesToLedger(); + LedgerHandle lh2 = createAndAddEntriesToLedger(); + + startNewBookie(); + shutdownBookie(lastBookieIndex() - 1); + + // add few more entries after ensemble reformation + for (int i = 0; i < 10; i++) { + ByteBuffer entry = ByteBuffer.allocate(4); + entry.putInt(rng.nextInt(Integer.MAX_VALUE)); + entry.position(0); + + entries.add(entry.array()); + lh1.addEntry(entry.array()); + lh2.addEntry(entry.array()); + } + + BookieLedgerIndexer bookieLedgerIndex = new BookieLedgerIndexer( + ledgerManager); + + Map> bookieToLedgerIndex = bookieLedgerIndex + .getBookieToLedgerIndex(); + assertEquals("Missed few bookies in the bookie-ledger mapping!", 4, + bookieToLedgerIndex.size()); + Collection> bk2ledgerEntry = bookieToLedgerIndex.values(); + for (Set ledgers : bk2ledgerEntry) { + assertEquals( + "Missed few ledgers in the bookie-ledger mapping!", 2, + ledgers.size()); + for (Long ledgerNode : ledgers) { + assertTrue("Unknown ledger-bookie mapping", ledgerList + .contains(ledgerNode)); + } + } + } catch (BKException e) { + LOG.error("Test failed", e); + fail("Test failed due to BookKeeper exception"); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOG.error("Test failed", e); + fail("Test failed due to interruption"); + } + } + + private void shutdownBookie(int bkShutdownIndex) throws Exception { + killBookie(bkShutdownIndex); + } + + private LedgerHandle createAndAddEntriesToLedger() throws BKException, + InterruptedException { + int numEntriesToWrite = 20; + // Create a ledger + LedgerHandle lh = bkc.createLedger(digestType, "admin".getBytes()); + LOG.info("Ledger ID: " + lh.getId()); + for (int i = 0; i < numEntriesToWrite; i++) { + ByteBuffer entry = ByteBuffer.allocate(4); + entry.putInt(rng.nextInt(Integer.MAX_VALUE)); + entry.position(0); + + entries.add(entry.array()); + lh.addEntry(entry.array()); + } + ledgerList.add(lh.getId()); + return lh; + } +} diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/ReplicationTestUtil.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/ReplicationTestUtil.java new file mode 100644 index 0000000000000..4360bf3254675 --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/ReplicationTestUtil.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.bookkeeper.replication; + +import java.util.List; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooKeeper; + +/** + * Utility class for replication tests. + */ +public class ReplicationTestUtil { + + /** + * Checks whether ledger is in under-replication. + */ + public static boolean isLedgerInUnderReplication(ZooKeeper zkc, long id, + String basePath) throws KeeperException, InterruptedException { + List children; + try { + children = zkc.getChildren(basePath, true); + } catch (KeeperException.NoNodeException nne) { + return false; + } + + boolean isMatched = false; + for (String child : children) { + if (child.startsWith("urL") && child.contains(String.valueOf(id))) { + isMatched = true; + break; + } else { + String path = basePath + '/' + child; + try { + if (zkc.getChildren(path, false).size() > 0) { + isMatched = isLedgerInUnderReplication(zkc, id, path); + } + } catch (KeeperException.NoNodeException nne) { + return false; + } + } + + } + return isMatched; + } +} diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java new file mode 100644 index 0000000000000..8a2e7f2747a22 --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.bookkeeper.replication; + +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertTrue; +import java.util.Enumeration; +import java.util.List; +import java.util.Map.Entry; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.LedgerEntry; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.bookkeeper.util.BookKeeperConstants; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +/** + * Test auto recovery. + */ +public class TestAutoRecoveryAlongWithBookieServers extends + BookKeeperClusterTestCase { + + private String basePath = ""; + + public TestAutoRecoveryAlongWithBookieServers() throws Exception { + super(3); + setAutoRecoveryEnabled(true); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataClientDriver"); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataBookieDriver"); + } + + @BeforeMethod + @Override + public void setUp() throws Exception { + super.setUp(); + basePath = BookKeeperConstants.DEFAULT_ZK_LEDGERS_ROOT_PATH + '/' + + BookKeeperConstants.UNDER_REPLICATION_NODE + + BookKeeperConstants.DEFAULT_ZK_LEDGERS_ROOT_PATH; + } + + @AfterMethod + @Override + public void tearDown() throws Exception { + super.tearDown(); + } + + @Override + protected void startBKCluster(String metadataServiceUri) throws Exception { + super.startBKCluster(metadataServiceUri.replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + } + + /** + * Tests that the auto recovery service along with Bookie servers itself. + */ + @Test + public void testAutoRecoveryAlongWithBookieServers() throws Exception { + LedgerHandle lh = bkc.createLedger(3, 3, BookKeeper.DigestType.CRC32, + "testpasswd".getBytes()); + byte[] testData = "testBuiltAutoRecovery".getBytes(); + + for (int i = 0; i < 10; i++) { + lh.addEntry(testData); + } + lh.close(); + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + + killBookie(replicaToKill); + + BookieId newBkAddr = startNewBookieAndReturnBookieId(); + + while (ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh.getId(), + basePath)) { + Thread.sleep(100); + } + + // Killing all bookies except newly replicated bookie + for (Entry> entry : + lh.getLedgerMetadata().getAllEnsembles().entrySet()) { + List bookies = entry.getValue(); + for (BookieId bookie : bookies) { + if (bookie.equals(newBkAddr)) { + continue; + } + killBookie(bookie); + } + } + + // Should be able to read the entries from 0-9 + LedgerHandle lhs = bkc.openLedgerNoRecovery(lh.getId(), + BookKeeper.DigestType.CRC32, "testpasswd".getBytes()); + Enumeration entries = lhs.readEntries(0, 9); + assertTrue("Should have the elements", entries.hasMoreElements()); + while (entries.hasMoreElements()) { + LedgerEntry entry = entries.nextElement(); + assertEquals("testBuiltAutoRecovery", new String(entry.getEntry())); + } + } +} diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java new file mode 100644 index 0000000000000..ca02f91d1de36 --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java @@ -0,0 +1,1249 @@ +/* + * 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.replication; + +import static org.apache.bookkeeper.replication.ReplicationStats.AUDITOR_SCOPE; +import static org.apache.bookkeeper.replication.ReplicationStats.NUM_ENTRIES_UNABLE_TO_READ_FOR_REPLICATION; +import static org.apache.bookkeeper.replication.ReplicationStats.REPLICATION_SCOPE; +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertFalse; +import static org.testng.AssertJUnit.assertNotNull; +import static org.testng.AssertJUnit.assertNull; +import static org.testng.AssertJUnit.assertTrue; +import static org.testng.AssertJUnit.fail; +import io.netty.util.HashedWheelTimer; +import java.io.IOException; +import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.net.URI; +import java.net.UnknownHostException; +import java.util.Enumeration; +import java.util.List; +import java.util.Map.Entry; +import java.util.Objects; +import java.util.Optional; +import java.util.TimerTask; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.BiConsumer; +import java.util.stream.Collectors; +import lombok.Cleanup; +import org.apache.bookkeeper.bookie.BookieImpl; +import org.apache.bookkeeper.client.BKException; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.BookKeeperTestClient; +import org.apache.bookkeeper.client.ClientUtil; +import org.apache.bookkeeper.client.EnsemblePlacementPolicy; +import org.apache.bookkeeper.client.LedgerEntry; +import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy; +import org.apache.bookkeeper.client.ZoneawareEnsemblePlacementPolicy; +import org.apache.bookkeeper.client.api.LedgerMetadata; +import org.apache.bookkeeper.common.util.OrderedScheduler; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.feature.FeatureProvider; +import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.meta.LedgerManagerFactory; +import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; +import org.apache.bookkeeper.meta.MetadataBookieDriver; +import org.apache.bookkeeper.meta.MetadataClientDriver; +import org.apache.bookkeeper.meta.MetadataDrivers; +import org.apache.bookkeeper.meta.ZkLedgerUnderreplicationManager; +import org.apache.bookkeeper.meta.exceptions.MetadataException; +import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.net.DNSToSwitchMapping; +import org.apache.bookkeeper.proto.BookieAddressResolver; +import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException; +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.Gauge; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.test.BookKeeperClusterTestCase; +import org.apache.bookkeeper.test.TestStatsProvider; +import org.apache.bookkeeper.test.TestStatsProvider.TestStatsLogger; +import org.apache.bookkeeper.util.BookKeeperConstants; +import org.apache.bookkeeper.util.StaticDNSResolver; +import org.apache.bookkeeper.zookeeper.ZooKeeperClient; +import org.apache.commons.lang3.mutable.MutableObject; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.pulsar.metadata.bookkeeper.PulsarLedgerManagerFactory; +import org.apache.pulsar.metadata.bookkeeper.PulsarMetadataClientDriver; +import org.apache.pulsar.metadata.impl.ZKMetadataStore; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.data.Stat; +import org.awaitility.Awaitility; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +/** + * Test the ReplicationWroker, where it has to replicate the fragments from + * failed Bookies to given target Bookie. + */ +public class TestReplicationWorker extends BookKeeperClusterTestCase { + + private static final byte[] TESTPASSWD = "testpasswd".getBytes(); + private static final Logger LOG = LoggerFactory + .getLogger(TestReplicationWorker.class); + private String basePath = ""; + private String baseLockPath = ""; + private MetadataBookieDriver driver; + private LedgerManagerFactory mFactory; + private LedgerUnderreplicationManager underReplicationManager; + private LedgerManager ledgerManager; + private static byte[] data = "TestReplicationWorker".getBytes(); + private OrderedScheduler scheduler; + private String zkLedgersRootPath; + + public TestReplicationWorker() throws Exception { + this("org.apache.pulsar.metadata.bookkeeper.PulsarLedgerManagerFactory"); + } + + TestReplicationWorker(String ledgerManagerFactory) throws Exception { + super(3, 300); + LOG.info("Running test case using ledger manager : " + + ledgerManagerFactory); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataClientDriver"); + Class.forName("org.apache.pulsar.metadata.bookkeeper.PulsarMetadataBookieDriver"); + // set ledger manager name + baseConf.setLedgerManagerFactoryClassName(ledgerManagerFactory); + baseClientConf.setLedgerManagerFactoryClassName(ledgerManagerFactory); + baseConf.setRereplicationEntryBatchSize(3); + baseConf.setZkTimeout(7000); + baseConf.setZkRetryBackoffMaxMs(500); + baseConf.setZkRetryBackoffStartMs(10); + } + + @BeforeMethod + @Override + public void setUp() throws Exception { + super.setUp(); + zkLedgersRootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(baseClientConf); + basePath = zkLedgersRootPath + '/' + + BookKeeperConstants.UNDER_REPLICATION_NODE + + BookKeeperConstants.DEFAULT_ZK_LEDGERS_ROOT_PATH; + baseLockPath = zkLedgersRootPath + '/' + + BookKeeperConstants.UNDER_REPLICATION_NODE + + "/locks"; + baseClientConf.setMetadataServiceUri( + zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + baseConf.setMetadataServiceUri( + zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); + this.scheduler = OrderedScheduler.newSchedulerBuilder() + .name("test-scheduler") + .numThreads(1) + .build(); + + this.driver = MetadataDrivers.getBookieDriver( + URI.create(baseConf.getMetadataServiceUri())); + this.driver.initialize( + baseConf, + NullStatsLogger.INSTANCE); + // initialize urReplicationManager + mFactory = driver.getLedgerManagerFactory(); + ledgerManager = mFactory.newLedgerManager(); + underReplicationManager = mFactory.newLedgerUnderreplicationManager(); + } + + @AfterMethod + @Override + public void tearDown() throws Exception { + super.tearDown(); + if (null != ledgerManager) { + ledgerManager.close(); + ledgerManager = null; + } + if (null != underReplicationManager) { + underReplicationManager.close(); + underReplicationManager = null; + } + if (null != driver) { + driver.close(); + } + if (null != scheduler) { + scheduler.shutdown(); + scheduler = null; + } + if (null != mFactory) { + mFactory.close(); + } + } + + /** + * Tests that replication worker should replicate the failed bookie + * fragments to target bookie given to the worker. + */ + @Test + public void testRWShouldReplicateFragmentsToTargetBookie() throws Exception { + LedgerHandle lh = bkc.createLedger(3, 3, BookKeeper.DigestType.CRC32, + TESTPASSWD); + + for (int i = 0; i < 10; i++) { + lh.addEntry(data); + } + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + + LOG.info("Killing Bookie : {}", replicaToKill); + killBookie(replicaToKill); + + BookieId newBkAddr = startNewBookieAndReturnBookieId(); + LOG.info("New Bookie addr : {}", newBkAddr); + + for (int i = 0; i < 10; i++) { + lh.addEntry(data); + } + + ReplicationWorker rw = new ReplicationWorker(baseConf); + + rw.start(); + try { + + underReplicationManager.markLedgerUnderreplicated(lh.getId(), + replicaToKill.toString()); + + while (ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh + .getId(), basePath)) { + Thread.sleep(100); + } + + killAllBookies(lh, newBkAddr); + + // Should be able to read the entries from 0-9 + verifyRecoveredLedgers(lh, 0, 9); + } finally { + rw.shutdown(); + } + } + + /** + * Tests that replication worker should retry for replication until enough + * bookies available for replication. + */ + @Test + public void testRWShouldRetryUntilThereAreEnoughBksAvailableForReplication() + throws Exception { + LedgerHandle lh = bkc.createLedger(1, 1, BookKeeper.DigestType.CRC32, + TESTPASSWD); + + for (int i = 0; i < 10; i++) { + lh.addEntry(data); + } + lh.close(); + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + LOG.info("Killing Bookie : {}", replicaToKill); + ServerConfiguration killedBookieConfig = killBookie(replicaToKill); + + BookieId newBkAddr = startNewBookieAndReturnBookieId(); + LOG.info("New Bookie addr :" + newBkAddr); + + killAllBookies(lh, newBkAddr); + ReplicationWorker rw = new ReplicationWorker(baseConf); + + rw.start(); + try { + underReplicationManager.markLedgerUnderreplicated(lh.getId(), + replicaToKill.toString()); + int counter = 30; + while (counter-- > 0) { + assertTrue("Expecting that replication should not complete", + ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh + .getId(), basePath)); + Thread.sleep(100); + } + // restart killed bookie + startAndAddBookie(killedBookieConfig); + while (ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh + .getId(), basePath)) { + Thread.sleep(100); + } + // Should be able to read the entries from 0-9 + verifyRecoveredLedgers(lh, 0, 9); + } finally { + rw.shutdown(); + } + } + + /** + * Tests that replication worker1 should take one fragment replication and + * other replication worker also should compete for the replication. + */ + @Test + public void test2RWsShouldCompeteForReplicationOf2FragmentsAndCompleteReplication() + throws Exception { + LedgerHandle lh = bkc.createLedger(2, 2, BookKeeper.DigestType.CRC32, + TESTPASSWD); + + for (int i = 0; i < 10; i++) { + lh.addEntry(data); + } + lh.close(); + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + LOG.info("Killing Bookie : {}", replicaToKill); + ServerConfiguration killedBookieConfig = killBookie(replicaToKill); + + killAllBookies(lh, null); + // Starte RW1 + BookieId newBkAddr1 = startNewBookieAndReturnBookieId(); + LOG.info("New Bookie addr : {}", newBkAddr1); + ReplicationWorker rw1 = new ReplicationWorker(baseConf); + + // Starte RW2 + BookieId newBkAddr2 = startNewBookieAndReturnBookieId(); + LOG.info("New Bookie addr : {}", newBkAddr2); + ReplicationWorker rw2 = new ReplicationWorker(baseConf); + rw1.start(); + rw2.start(); + + try { + underReplicationManager.markLedgerUnderreplicated(lh.getId(), + replicaToKill.toString()); + int counter = 10; + while (counter-- > 0) { + assertTrue("Expecting that replication should not complete", + ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh + .getId(), basePath)); + Thread.sleep(100); + } + // restart killed bookie + startAndAddBookie(killedBookieConfig); + while (ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh + .getId(), basePath)) { + Thread.sleep(100); + } + // Should be able to read the entries from 0-9 + verifyRecoveredLedgers(lh, 0, 9); + } finally { + rw1.shutdown(); + rw2.shutdown(); + } + } + + /** + * Tests that Replication worker should clean the leadger under replication + * node of the ledger already deleted. + */ + @Test + public void testRWShouldCleanTheLedgerFromUnderReplicationIfLedgerAlreadyDeleted() + throws Exception { + LedgerHandle lh = bkc.createLedger(2, 2, BookKeeper.DigestType.CRC32, + TESTPASSWD); + + for (int i = 0; i < 10; i++) { + lh.addEntry(data); + } + lh.close(); + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + LOG.info("Killing Bookie : {}", replicaToKill); + killBookie(replicaToKill); + + BookieId newBkAddr = startNewBookieAndReturnBookieId(); + LOG.info("New Bookie addr : {}", newBkAddr); + ReplicationWorker rw = new ReplicationWorker(baseConf); + rw.start(); + + try { + bkc.deleteLedger(lh.getId()); // Deleting the ledger + // Also mark ledger as in UnderReplication + underReplicationManager.markLedgerUnderreplicated(lh.getId(), + replicaToKill.toString()); + while (ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh + .getId(), basePath)) { + Thread.sleep(100); + } + } finally { + rw.shutdown(); + } + + } + + @Test + public void testMultipleLedgerReplicationWithReplicationWorker() + throws Exception { + // Ledger1 + LedgerHandle lh1 = bkc.createLedger(3, 3, BookKeeper.DigestType.CRC32, + TESTPASSWD); + + for (int i = 0; i < 10; i++) { + lh1.addEntry(data); + } + BookieId replicaToKillFromFirstLedger = lh1.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + + LOG.info("Killing Bookie : {}", replicaToKillFromFirstLedger); + + // Ledger2 + LedgerHandle lh2 = bkc.createLedger(3, 3, BookKeeper.DigestType.CRC32, + TESTPASSWD); + + for (int i = 0; i < 10; i++) { + lh2.addEntry(data); + } + BookieId replicaToKillFromSecondLedger = lh2.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + + LOG.info("Killing Bookie : {}", replicaToKillFromSecondLedger); + + // Kill ledger1 + killBookie(replicaToKillFromFirstLedger); + lh1.close(); + // Kill ledger2 + killBookie(replicaToKillFromFirstLedger); + lh2.close(); + + BookieId newBkAddr = startNewBookieAndReturnBookieId(); + LOG.info("New Bookie addr : {}", newBkAddr); + + ReplicationWorker rw = new ReplicationWorker(baseConf); + + rw.start(); + try { + + // Mark ledger1 and 2 as underreplicated + underReplicationManager.markLedgerUnderreplicated(lh1.getId(), + replicaToKillFromFirstLedger.toString()); + underReplicationManager.markLedgerUnderreplicated(lh2.getId(), + replicaToKillFromSecondLedger.toString()); + + while (ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh1 + .getId(), basePath)) { + Thread.sleep(100); + } + + while (ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh2 + .getId(), basePath)) { + Thread.sleep(100); + } + + killAllBookies(lh1, newBkAddr); + + // Should be able to read the entries from 0-9 + verifyRecoveredLedgers(lh1, 0, 9); + verifyRecoveredLedgers(lh2, 0, 9); + } finally { + rw.shutdown(); + } + + } + + /** + * Tests that ReplicationWorker should fence the ledger and release ledger + * lock after timeout. Then replication should happen normally. + */ + @Test + public void testRWShouldReplicateTheLedgersAfterTimeoutIfLastFragmentIsUR() + throws Exception { + LedgerHandle lh = bkc.createLedger(3, 3, BookKeeper.DigestType.CRC32, + TESTPASSWD); + + for (int i = 0; i < 10; i++) { + lh.addEntry(data); + } + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + + LOG.info("Killing Bookie : {}", replicaToKill); + killBookie(replicaToKill); + + BookieId newBkAddr = startNewBookieAndReturnBookieId(); + LOG.info("New Bookie addr : {}", newBkAddr); + + // set to 3s instead of default 30s + baseConf.setOpenLedgerRereplicationGracePeriod("3000"); + ReplicationWorker rw = new ReplicationWorker(baseConf); + + @Cleanup MetadataClientDriver clientDriver = MetadataDrivers.getClientDriver( + URI.create(baseClientConf.getMetadataServiceUri())); + clientDriver.initialize(baseClientConf, scheduler, NullStatsLogger.INSTANCE, Optional.empty()); + + LedgerManagerFactory mFactory = clientDriver.getLedgerManagerFactory(); + + LedgerUnderreplicationManager underReplicationManager = mFactory + .newLedgerUnderreplicationManager(); + rw.start(); + try { + + underReplicationManager.markLedgerUnderreplicated(lh.getId(), + replicaToKill.toString()); + while (ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh + .getId(), basePath)) { + Thread.sleep(100); + } + killAllBookies(lh, newBkAddr); + // Should be able to read the entries from 0-9 + verifyRecoveredLedgers(lh, 0, 9); + lh = bkc.openLedgerNoRecovery(lh.getId(), + BookKeeper.DigestType.CRC32, TESTPASSWD); + assertFalse("Ledger must have been closed by RW", ClientUtil + .isLedgerOpen(lh)); + } finally { + rw.shutdown(); + underReplicationManager.close(); + } + + } + + @Test + public void testBookiesNotAvailableScenarioForReplicationWorker() throws Exception { + int ensembleSize = 3; + LedgerHandle lh = bkc.createLedger(ensembleSize, ensembleSize, BookKeeper.DigestType.CRC32, TESTPASSWD); + + int numOfEntries = 7; + for (int i = 0; i < numOfEntries; i++) { + lh.addEntry(data); + } + lh.close(); + + BookieId[] bookiesKilled = new BookieId[ensembleSize]; + ServerConfiguration[] killedBookiesConfig = new ServerConfiguration[ensembleSize]; + + // kill all bookies + for (int i = 0; i < ensembleSize; i++) { + bookiesKilled[i] = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(i); + killedBookiesConfig[i] = getBkConf(bookiesKilled[i]); + LOG.info("Killing Bookie : {}", bookiesKilled[i]); + killBookie(bookiesKilled[i]); + } + + // start new bookiesToKill number of bookies + for (int i = 0; i < ensembleSize; i++) { + BookieId newBkAddr = startNewBookieAndReturnBookieId(); + } + + // create couple of replicationworkers + ServerConfiguration newRWConf = new ServerConfiguration(baseConf); + newRWConf.setLockReleaseOfFailedLedgerGracePeriod("64"); + ReplicationWorker rw1 = new ReplicationWorker(newRWConf); + ReplicationWorker rw2 = new ReplicationWorker(newRWConf); + + @Cleanup + MetadataClientDriver clientDriver = MetadataDrivers + .getClientDriver(URI.create(baseClientConf.getMetadataServiceUri())); + clientDriver.initialize(baseClientConf, scheduler, NullStatsLogger.INSTANCE, Optional.empty()); + + LedgerManagerFactory mFactory = clientDriver.getLedgerManagerFactory(); + + LedgerUnderreplicationManager underReplicationManager = mFactory.newLedgerUnderreplicationManager(); + try { + //mark ledger underreplicated + for (int i = 0; i < bookiesKilled.length; i++) { + underReplicationManager.markLedgerUnderreplicated(lh.getId(), bookiesKilled[i].toString()); + } + while (!ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh.getId(), basePath)) { + Thread.sleep(100); + } + rw1.start(); + rw2.start(); + + AtomicBoolean isBookieRestarted = new AtomicBoolean(false); + + (new Thread(new Runnable() { + @Override + public void run() { + try { + Thread.sleep(3000); + isBookieRestarted.set(true); + /* + * after sleeping for 3000 msecs, restart one of the + * bookie, so that replication can succeed. + */ + startBookie(killedBookiesConfig[0]); + } catch (Exception e) { + e.printStackTrace(); + } + } + })).start(); + + int rw1PrevFailedAttemptsCount = 0; + int rw2PrevFailedAttemptsCount = 0; + while (!isBookieRestarted.get()) { + /* + * since all the bookies containing the ledger entries are down + * replication wouldnt have succeeded. + */ + assertTrue("Ledger: " + lh.getId() + " should be underreplicated", + ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh.getId(), basePath)); + + // the number of failed attempts should have increased. + int rw1CurFailedAttemptsCount = rw1.replicationFailedLedgers.get(lh.getId()).get(); + assertTrue( + "The current number of failed attempts: " + rw1CurFailedAttemptsCount + + " should be greater than or equal to previous value: " + rw1PrevFailedAttemptsCount, + rw1CurFailedAttemptsCount >= rw1PrevFailedAttemptsCount); + rw1PrevFailedAttemptsCount = rw1CurFailedAttemptsCount; + + int rw2CurFailedAttemptsCount = rw2.replicationFailedLedgers.get(lh.getId()).get(); + assertTrue( + "The current number of failed attempts: " + rw2CurFailedAttemptsCount + + " should be greater than or equal to previous value: " + rw2PrevFailedAttemptsCount, + rw2CurFailedAttemptsCount >= rw2PrevFailedAttemptsCount); + rw2PrevFailedAttemptsCount = rw2CurFailedAttemptsCount; + + Thread.sleep(50); + } + + /** + * since one of the killed bookie is restarted, replicationworker + * should succeed in replicating this under replicated ledger and it + * shouldn't be under replicated anymore. + */ + int timeToWaitForReplicationToComplete = 20000; + int timeWaited = 0; + while (ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh.getId(), basePath)) { + Thread.sleep(100); + timeWaited += 100; + if (timeWaited == timeToWaitForReplicationToComplete) { + fail("Ledger should be replicated by now"); + } + } + + rw1PrevFailedAttemptsCount = rw1.replicationFailedLedgers.get(lh.getId()).get(); + rw2PrevFailedAttemptsCount = rw2.replicationFailedLedgers.get(lh.getId()).get(); + Thread.sleep(2000); + // now since the ledger is replicated, number of failed attempts + // counter shouldn't be increased even after sleeping for sometime. + assertEquals("rw1 failedattempts", rw1PrevFailedAttemptsCount, + rw1.replicationFailedLedgers.get(lh.getId()).get()); + assertEquals("rw2 failed attempts ", rw2PrevFailedAttemptsCount, + rw2.replicationFailedLedgers.get(lh.getId()).get()); + + /* + * Since these entries are eventually available, and replication has + * eventually succeeded, in one of the RW + * unableToReadEntriesForReplication should be 0. + */ + int rw1UnableToReadEntriesForReplication = rw1.unableToReadEntriesForReplication.get(lh.getId()).size(); + int rw2UnableToReadEntriesForReplication = rw2.unableToReadEntriesForReplication.get(lh.getId()).size(); + assertTrue( + "unableToReadEntriesForReplication in RW1: " + rw1UnableToReadEntriesForReplication + + " in RW2: " + + rw2UnableToReadEntriesForReplication, + (rw1UnableToReadEntriesForReplication == 0) + || (rw2UnableToReadEntriesForReplication == 0)); + } finally { + rw1.shutdown(); + rw2.shutdown(); + underReplicationManager.close(); + } + } + + class InjectedReplicationWorker extends ReplicationWorker { + CopyOnWriteArrayList delayReplicationPeriods; + + public InjectedReplicationWorker(ServerConfiguration conf, StatsLogger statsLogger, + CopyOnWriteArrayList delayReplicationPeriods) + throws CompatibilityException, ReplicationException.UnavailableException, + InterruptedException, IOException { + super(conf, statsLogger); + this.delayReplicationPeriods = delayReplicationPeriods; + } + + @Override + protected void scheduleTaskWithDelay(TimerTask timerTask, long delayPeriod) { + delayReplicationPeriods.add(delayPeriod); + super.scheduleTaskWithDelay(timerTask, delayPeriod); + } + } + + @Test + public void testDeferLedgerLockReleaseForReplicationWorker() throws Exception { + int ensembleSize = 3; + LedgerHandle lh = bkc.createLedger(ensembleSize, ensembleSize, BookKeeper.DigestType.CRC32, TESTPASSWD); + int numOfEntries = 7; + for (int i = 0; i < numOfEntries; i++) { + lh.addEntry(data); + } + lh.close(); + + BookieId[] bookiesKilled = new BookieId[ensembleSize]; + ServerConfiguration[] killedBookiesConfig = new ServerConfiguration[ensembleSize]; + + // kill all bookies + for (int i = 0; i < ensembleSize; i++) { + bookiesKilled[i] = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(i); + killedBookiesConfig[i] = getBkConf(bookiesKilled[i]); + LOG.info("Killing Bookie : {}", bookiesKilled[i]); + killBookie(bookiesKilled[i]); + } + + // start new bookiesToKill number of bookies + for (int i = 0; i < ensembleSize; i++) { + startNewBookieAndReturnBookieId(); + } + + // create couple of replicationworkers + long lockReleaseOfFailedLedgerGracePeriod = 64L; + long baseBackoffForLockReleaseOfFailedLedger = lockReleaseOfFailedLedgerGracePeriod + / (int) Math.pow(2, ReplicationWorker.NUM_OF_EXPONENTIAL_BACKOFF_RETRIALS); + ServerConfiguration newRWConf = new ServerConfiguration(baseConf); + newRWConf.setLockReleaseOfFailedLedgerGracePeriod(Long.toString(lockReleaseOfFailedLedgerGracePeriod)); + newRWConf.setRereplicationEntryBatchSize(1000); + CopyOnWriteArrayList rw1DelayReplicationPeriods = new CopyOnWriteArrayList(); + CopyOnWriteArrayList rw2DelayReplicationPeriods = new CopyOnWriteArrayList(); + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger1 = statsProvider.getStatsLogger("rw1"); + TestStatsLogger statsLogger2 = statsProvider.getStatsLogger("rw2"); + ReplicationWorker rw1 = new InjectedReplicationWorker(newRWConf, statsLogger1, rw1DelayReplicationPeriods); + ReplicationWorker rw2 = new InjectedReplicationWorker(newRWConf, statsLogger2, rw2DelayReplicationPeriods); + + Counter numEntriesUnableToReadForReplication1 = statsLogger1 + .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_REPLICATION); + Counter numEntriesUnableToReadForReplication2 = statsLogger2 + .getCounter(NUM_ENTRIES_UNABLE_TO_READ_FOR_REPLICATION); + @Cleanup + MetadataClientDriver clientDriver = MetadataDrivers + .getClientDriver(URI.create(baseClientConf.getMetadataServiceUri())); + clientDriver.initialize(baseClientConf, scheduler, NullStatsLogger.INSTANCE, Optional.empty()); + + LedgerManagerFactory mFactory = clientDriver.getLedgerManagerFactory(); + + LedgerUnderreplicationManager underReplicationManager = mFactory.newLedgerUnderreplicationManager(); + try { + // mark ledger underreplicated + for (int i = 0; i < bookiesKilled.length; i++) { + underReplicationManager.markLedgerUnderreplicated(lh.getId(), bookiesKilled[i].toString()); + } + while (!ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh.getId(), basePath)) { + Thread.sleep(100); + } + rw1.start(); + rw2.start(); + + // wait for RWs to complete 'numOfAttemptsToWaitFor' failed attempts + int numOfAttemptsToWaitFor = 10; + while ((rw1.replicationFailedLedgers.get(lh.getId()).get() < numOfAttemptsToWaitFor) + || rw2.replicationFailedLedgers.get(lh.getId()).get() < numOfAttemptsToWaitFor) { + Thread.sleep(500); + } + + /* + * since all the bookies containing the ledger entries are down + * replication wouldn't have succeeded. + */ + assertTrue("Ledger: " + lh.getId() + " should be underreplicated", + ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh.getId(), basePath)); + + /* + * since RW failed 'numOfAttemptsToWaitFor' number of times, we + * should have atleast (numOfAttemptsToWaitFor - 1) + * delayReplicationPeriods and their value should be + * (lockReleaseOfFailedLedgerGracePeriod/16) , 2 * previous value,.. + * with max : lockReleaseOfFailedLedgerGracePeriod + */ + for (int i = 0; i < ((numOfAttemptsToWaitFor - 1)); i++) { + long expectedDelayValue = Math.min(lockReleaseOfFailedLedgerGracePeriod, + baseBackoffForLockReleaseOfFailedLedger * (1 << i)); + assertEquals("RW1 delayperiod", (Long) expectedDelayValue, rw1DelayReplicationPeriods.get(i)); + assertEquals("RW2 delayperiod", (Long) expectedDelayValue, rw2DelayReplicationPeriods.get(i)); + } + + /* + * RW wont try to replicate until and unless RW succeed in reading + * those failed entries before proceeding with replication of under + * replicated fragment, so the numEntriesUnableToReadForReplication + * should be just 'numOfEntries', though RW failed to replicate + * multiple times. + */ + assertEquals("numEntriesUnableToReadForReplication for RW1", Long.valueOf((long) numOfEntries), + numEntriesUnableToReadForReplication1.get()); + assertEquals("numEntriesUnableToReadForReplication for RW2", Long.valueOf((long) numOfEntries), + numEntriesUnableToReadForReplication2.get()); + + /* + * Since these entries are unavailable, + * unableToReadEntriesForReplication should be of size numOfEntries. + */ + assertEquals("RW1 unabletoreadentries", numOfEntries, + rw1.unableToReadEntriesForReplication.get(lh.getId()).size()); + assertEquals("RW2 unabletoreadentries", numOfEntries, + rw2.unableToReadEntriesForReplication.get(lh.getId()).size()); + } finally { + rw1.shutdown(); + rw2.shutdown(); + underReplicationManager.close(); + } + } + + /** + * Tests that ReplicationWorker should not have identified for postponing + * the replication if ledger is in open state and lastFragment is not in + * underReplication state. Note that RW should not fence such ledgers. + */ + @Test + public void testRWShouldReplicateTheLedgersAfterTimeoutIfLastFragmentIsNotUR() + throws Exception { + LedgerHandle lh = bkc.createLedger(3, 3, BookKeeper.DigestType.CRC32, + TESTPASSWD); + + for (int i = 0; i < 10; i++) { + lh.addEntry(data); + } + BookieId replicaToKill = lh.getLedgerMetadata().getAllEnsembles().get(0L).get(0); + + LOG.info("Killing Bookie : {}", replicaToKill); + killBookie(replicaToKill); + + BookieId newBkAddr = startNewBookieAndReturnBookieId(); + LOG.info("New Bookie addr : {}", newBkAddr); + + // Reform ensemble...Making sure that last fragment is not in + // under-replication + for (int i = 0; i < 10; i++) { + lh.addEntry(data); + } + + ReplicationWorker rw = new ReplicationWorker(baseConf); + + baseClientConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri()); + + @Cleanup MetadataClientDriver driver = MetadataDrivers.getClientDriver( + URI.create(baseClientConf.getMetadataServiceUri())); + driver.initialize(baseClientConf, scheduler, NullStatsLogger.INSTANCE, Optional.empty()); + + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + + LedgerUnderreplicationManager underReplicationManager = mFactory + .newLedgerUnderreplicationManager(); + + rw.start(); + try { + + underReplicationManager.markLedgerUnderreplicated(lh.getId(), + replicaToKill.toString()); + while (ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh + .getId(), basePath)) { + Thread.sleep(100); + } + + killAllBookies(lh, newBkAddr); + + // Should be able to read the entries from 0-9 + verifyRecoveredLedgers(lh, 0, 9); + lh = bkc.openLedgerNoRecovery(lh.getId(), + BookKeeper.DigestType.CRC32, TESTPASSWD); + + // Ledger should be still in open state + assertTrue("Ledger must have been closed by RW", ClientUtil + .isLedgerOpen(lh)); + } finally { + rw.shutdown(); + underReplicationManager.close(); + } + + } + + /** + * Test that the replication worker will not shutdown on a simple ZK disconnection. + */ + @Test + public void testRWZKConnectionLost() throws Exception { + try (ZooKeeperClient zk = ZooKeeperClient.newBuilder() + .connectString(zkUtil.getZooKeeperConnectString()) + .sessionTimeoutMs(10000) + .build()) { + + ReplicationWorker rw = new ReplicationWorker(baseConf); + rw.start(); + for (int i = 0; i < 10; i++) { + if (rw.isRunning()) { + break; + } + Thread.sleep(1000); + } + assertTrue("Replication worker should be running", rw.isRunning()); + + stopZKCluster(); + // ZK is down for shorter period than reconnect timeout + Thread.sleep(1000); + startZKCluster(); + + assertTrue("Replication worker should not shutdown", rw.isRunning()); + } + } + + /** + * Test that the replication worker shuts down on non-recoverable ZK connection loss. + */ + @Test + public void testRWZKConnectionLostOnNonRecoverableZkError() throws Exception { + for (int j = 0; j < 3; j++) { + LedgerHandle lh = bkc.createLedger(1, 1, 1, + BookKeeper.DigestType.CRC32, TESTPASSWD, + null); + final long createdLedgerId = lh.getId(); + for (int i = 0; i < 10; i++) { + lh.addEntry(data); + } + lh.close(); + } + + killBookie(2); + killBookie(1); + startNewBookie(); + startNewBookie(); + + servers.get(0).getConfiguration().setRwRereplicateBackoffMs(100); + servers.get(0).startAutoRecovery(); + + Auditor auditor = getAuditor(10, TimeUnit.SECONDS); + ReplicationWorker rw = servers.get(0).getReplicationWorker(); + + ZkLedgerUnderreplicationManager ledgerUnderreplicationManager = + (ZkLedgerUnderreplicationManager) FieldUtils.readField(auditor, + "ledgerUnderreplicationManager", true); + + ZooKeeper zkc = (ZooKeeper) FieldUtils.readField(ledgerUnderreplicationManager, "zkc", true); + auditor.submitAuditTask().get(); + + assertTrue(zkc.getState().isConnected()); + zkc.close(); + assertFalse(zkc.getState().isConnected()); + + auditor.submitAuditTask(); + rw.run(); + + for (int i = 0; i < 10; i++) { + if (!rw.isRunning() && !auditor.isRunning()) { + break; + } + Thread.sleep(1000); + } + assertFalse("Replication worker should NOT be running", rw.isRunning()); + assertFalse("Auditor should NOT be running", auditor.isRunning()); + } + + private void killAllBookies(LedgerHandle lh, BookieId excludeBK) + throws Exception { + // Killing all bookies except newly replicated bookie + for (Entry> entry : + lh.getLedgerMetadata().getAllEnsembles().entrySet()) { + List bookies = entry.getValue(); + for (BookieId bookie : bookies) { + if (bookie.equals(excludeBK)) { + continue; + } + killBookie(bookie); + } + } + } + + private void verifyRecoveredLedgers(LedgerHandle lh, long startEntryId, + long endEntryId) throws BKException, InterruptedException { + LedgerHandle lhs = bkc.openLedgerNoRecovery(lh.getId(), + BookKeeper.DigestType.CRC32, TESTPASSWD); + Enumeration entries = lhs.readEntries(startEntryId, + endEntryId); + assertTrue("Should have the elements", entries.hasMoreElements()); + while (entries.hasMoreElements()) { + LedgerEntry entry = entries.nextElement(); + assertEquals("TestReplicationWorker", new String(entry.getEntry())); + } + } + + @Test + public void testReplicateEmptyOpenStateLedger() throws Exception { + LedgerHandle lh = bkc.createLedger(3, 3, 2, BookKeeper.DigestType.CRC32, TESTPASSWD); + assertFalse(lh.getLedgerMetadata().isClosed()); + + List firstEnsemble = lh.getLedgerMetadata().getAllEnsembles().firstEntry().getValue(); + List ensemble = lh.getLedgerMetadata().getAllEnsembles().entrySet().iterator().next().getValue(); + killBookie(ensemble.get(1)); + + startNewBookie(); + baseConf.setOpenLedgerRereplicationGracePeriod(String.valueOf(30)); + ReplicationWorker replicationWorker = new ReplicationWorker(baseConf); + replicationWorker.start(); + + try { + underReplicationManager.markLedgerUnderreplicated(lh.getId(), ensemble.get(1).toString()); + Awaitility.waitAtMost(60, TimeUnit.SECONDS).untilAsserted(() -> + assertFalse(ReplicationTestUtil.isLedgerInUnderReplication(zkc, lh.getId(), basePath)) + ); + + LedgerHandle lh1 = bkc.openLedgerNoRecovery(lh.getId(), BookKeeper.DigestType.CRC32, TESTPASSWD); + assertTrue(lh1.getLedgerMetadata().isClosed()); + } finally { + replicationWorker.shutdown(); + } + } + + @Test + public void testRepairedNotAdheringPlacementPolicyLedgerFragmentsOnRack() throws Exception { + testRepairedNotAdheringPlacementPolicyLedgerFragments(RackawareEnsemblePlacementPolicy.class, null); + } + + @Test + public void testReplicationStats() throws Exception { + BiConsumer checkReplicationStats = (first, rw) -> { + try { + final Method rereplicate = rw.getClass().getDeclaredMethod("rereplicate"); + rereplicate.setAccessible(true); + final Object result = rereplicate.invoke(rw); + final Field statsLoggerField = rw.getClass().getDeclaredField("statsLogger"); + statsLoggerField.setAccessible(true); + final TestStatsLogger statsLogger = (TestStatsLogger) statsLoggerField.get(rw); + + final Counter numDeferLedgerLockReleaseOfFailedLedgerCounter = + statsLogger.getCounter(ReplicationStats.NUM_DEFER_LEDGER_LOCK_RELEASE_OF_FAILED_LEDGER); + final Counter numLedgersReplicatedCounter = + statsLogger.getCounter(ReplicationStats.NUM_FULL_OR_PARTIAL_LEDGERS_REPLICATED); + final Counter numNotAdheringPlacementLedgersCounter = statsLogger + .getCounter(ReplicationStats.NUM_NOT_ADHERING_PLACEMENT_LEDGERS_REPLICATED); + + assertEquals("NUM_DEFER_LEDGER_LOCK_RELEASE_OF_FAILED_LEDGER", + 1, numDeferLedgerLockReleaseOfFailedLedgerCounter.get().longValue()); + + if (first) { + assertFalse((boolean) result); + assertEquals("NUM_FULL_OR_PARTIAL_LEDGERS_REPLICATED", + 0, numLedgersReplicatedCounter.get().longValue()); + assertEquals("NUM_NOT_ADHERING_PLACEMENT_LEDGERS_REPLICATED", + 0, numNotAdheringPlacementLedgersCounter.get().longValue()); + + } else { + assertTrue((boolean) result); + assertEquals("NUM_FULL_OR_PARTIAL_LEDGERS_REPLICATED", + 1, numLedgersReplicatedCounter.get().longValue()); + assertEquals("NUM_NOT_ADHERING_PLACEMENT_LEDGERS_REPLICATED", + 1, numNotAdheringPlacementLedgersCounter.get().longValue()); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + }; + testRepairedNotAdheringPlacementPolicyLedgerFragments( + RackawareEnsemblePlacementPolicy.class, checkReplicationStats); + } + + private void testRepairedNotAdheringPlacementPolicyLedgerFragments( + Class placementPolicyClass, + BiConsumer checkReplicationStats) throws Exception { + List firstThreeBookies = servers.stream().map(ele -> { + try { + return ele.getServer().getBookieId(); + } catch (UnknownHostException e) { + return null; + } + }).filter(Objects::nonNull).collect(Collectors.toList()); + + baseClientConf.setProperty("reppDnsResolverClass", StaticDNSResolver.class.getName()); + baseClientConf.setProperty("enforceStrictZoneawarePlacement", false); + bkc.close(); + bkc = new BookKeeperTestClient(baseClientConf) { + @Override + protected EnsemblePlacementPolicy initializeEnsemblePlacementPolicy(ClientConfiguration conf, + DNSToSwitchMapping dnsResolver, + HashedWheelTimer timer, + FeatureProvider featureProvider, + StatsLogger statsLogger, + BookieAddressResolver bookieAddressResolver) + throws IOException { + EnsemblePlacementPolicy ensemblePlacementPolicy = null; + if (ZoneawareEnsemblePlacementPolicy.class == placementPolicyClass) { + ensemblePlacementPolicy = buildZoneAwareEnsemblePlacementPolicy(firstThreeBookies); + } else if (RackawareEnsemblePlacementPolicy.class == placementPolicyClass) { + ensemblePlacementPolicy = buildRackAwareEnsemblePlacementPolicy(firstThreeBookies); + } + ensemblePlacementPolicy.initialize(conf, Optional.ofNullable(dnsResolver), timer, + featureProvider, statsLogger, bookieAddressResolver); + return ensemblePlacementPolicy; + } + }; + + //This ledger not adhering placement policy, the combine(0,1,2) rack is 1. + LedgerHandle lh = bkc.createLedger(3, 3, 3, BookKeeper.DigestType.CRC32, TESTPASSWD); + + int entrySize = 10; + for (int i = 0; i < entrySize; i++) { + lh.addEntry(data); + } + lh.close(); + + int minNumRacksPerWriteQuorumConfValue = 2; + + ServerConfiguration servConf = new ServerConfiguration(confByIndex(0)); + servConf.setMinNumRacksPerWriteQuorum(minNumRacksPerWriteQuorumConfValue); + servConf.setProperty("reppDnsResolverClass", StaticDNSResolver.class.getName()); + servConf.setAuditorPeriodicPlacementPolicyCheckInterval(1000); + servConf.setRepairedPlacementPolicyNotAdheringBookieEnable(true); + + MutableObject auditorRef = new MutableObject(); + try { + TestStatsLogger statsLogger = startAuditorAndWaitForPlacementPolicyCheck(servConf, auditorRef); + Gauge ledgersNotAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY); + assertEquals("NUM_LEDGERS_NOT_ADHERING_TO_PLACEMENT_POLICY guage value", + 1, ledgersNotAdheringToPlacementPolicyGuage.getSample()); + Gauge ledgersSoftlyAdheringToPlacementPolicyGuage = statsLogger + .getGauge(ReplicationStats.NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY); + assertEquals("NUM_LEDGERS_SOFTLY_ADHERING_TO_PLACEMENT_POLICY guage value", + 0, ledgersSoftlyAdheringToPlacementPolicyGuage.getSample()); + } finally { + Auditor auditor = auditorRef.getValue(); + if (auditor != null) { + auditor.close(); + } + } + + ZooKeeper zk = getZk((PulsarMetadataClientDriver) bkc.getMetadataClientDriver()); + + + Stat stat = zk.exists("/ledgers/underreplication/ledgers/0000/0000/0000/0000/urL0000000000", false); + assertNotNull(stat); + + baseConf.setRepairedPlacementPolicyNotAdheringBookieEnable(true); + BookKeeper bookKeeper = new BookKeeperTestClient(baseClientConf) { + @Override + protected EnsemblePlacementPolicy initializeEnsemblePlacementPolicy(ClientConfiguration conf, + DNSToSwitchMapping dnsResolver, + HashedWheelTimer timer, + FeatureProvider featureProvider, + StatsLogger statsLogger, + BookieAddressResolver bookieAddressResolver) + throws IOException { + EnsemblePlacementPolicy ensemblePlacementPolicy = null; + if (ZoneawareEnsemblePlacementPolicy.class == placementPolicyClass) { + ensemblePlacementPolicy = buildZoneAwareEnsemblePlacementPolicy(firstThreeBookies); + } else if (RackawareEnsemblePlacementPolicy.class == placementPolicyClass) { + ensemblePlacementPolicy = buildRackAwareEnsemblePlacementPolicy(firstThreeBookies); + } + ensemblePlacementPolicy.initialize(conf, Optional.ofNullable(dnsResolver), timer, + featureProvider, statsLogger, bookieAddressResolver); + return ensemblePlacementPolicy; + } + }; + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(REPLICATION_SCOPE); + ReplicationWorker rw = new ReplicationWorker(baseConf, bookKeeper, false, statsLogger); + + if (checkReplicationStats != null) { + checkReplicationStats.accept(true, rw); + } else { + rw.start(); + } + + //start new bookie, the rack is /rack2 + BookieId newBookieId = startNewBookieAndReturnBookieId(); + + if (checkReplicationStats != null) { + checkReplicationStats.accept(false, rw); + } + + Awaitility.await().untilAsserted(() -> { + LedgerMetadata metadata = bkc.getLedgerManager().readLedgerMetadata(lh.getId()).get().getValue(); + List newBookies = metadata.getAllEnsembles().get(0L); + assertTrue(newBookies.contains(newBookieId)); + }); + + Awaitility.await().untilAsserted(() -> { + Stat stat1 = zk.exists("/ledgers/underreplication/ledgers/0000/0000/0000/0000/urL0000000000", false); + assertNull(stat1); + }); + + for (BookieId rack1Book : firstThreeBookies) { + killBookie(rack1Book); + } + + verifyRecoveredLedgers(lh, 0, entrySize - 1); + + if (checkReplicationStats == null) { + rw.shutdown(); + } + baseConf.setRepairedPlacementPolicyNotAdheringBookieEnable(false); + bookKeeper.close(); + } + + private EnsemblePlacementPolicy buildRackAwareEnsemblePlacementPolicy(List bookieIds) { + return new RackawareEnsemblePlacementPolicy() { + @Override + public String resolveNetworkLocation(BookieId addr) { + if (bookieIds.contains(addr)) { + return "/rack1"; + } + //The other bookie is /rack2 + return "/rack2"; + } + }; + } + + private EnsemblePlacementPolicy buildZoneAwareEnsemblePlacementPolicy(List firstThreeBookies) { + return new ZoneawareEnsemblePlacementPolicy() { + @Override + protected String resolveNetworkLocation(BookieId addr) { + //The first three bookie 1 is /zone1/ud1 + //The first three bookie 2,3 is /zone1/ud2 + if (firstThreeBookies.get(0).equals(addr)) { + return "/zone1/ud1"; + } else if (firstThreeBookies.contains(addr)) { + return "/zone1/ud2"; + } + //The other bookie is /zone2/ud1 + return "/zone2/ud1"; + } + }; + } + + private TestStatsLogger startAuditorAndWaitForPlacementPolicyCheck(ServerConfiguration servConf, + MutableObject auditorRef) + throws MetadataException, CompatibilityException, KeeperException, + InterruptedException, ReplicationException.UnavailableException, UnknownHostException { + LedgerManagerFactory mFactory = driver.getLedgerManagerFactory(); + LedgerUnderreplicationManager urm = mFactory.newLedgerUnderreplicationManager(); + TestStatsProvider statsProvider = new TestStatsProvider(); + TestStatsLogger statsLogger = statsProvider.getStatsLogger(AUDITOR_SCOPE); + TestStatsProvider.TestOpStatsLogger placementPolicyCheckStatsLogger = + (TestStatsProvider.TestOpStatsLogger) statsLogger + .getOpStatsLogger(ReplicationStats.PLACEMENT_POLICY_CHECK_TIME); + + final AuditorPeriodicCheckTest.TestAuditor auditor = new AuditorPeriodicCheckTest.TestAuditor( + BookieImpl.getBookieId(servConf).toString(), servConf, bkc, false, statsLogger, null); + auditorRef.setValue(auditor); + CountDownLatch latch = auditor.getLatch(); + assertEquals("PLACEMENT_POLICY_CHECK_TIME SuccessCount", 0, + placementPolicyCheckStatsLogger.getSuccessCount()); + urm.setPlacementPolicyCheckCTime(-1); + auditor.start(); + /* + * since placementPolicyCheckCTime is set to -1, placementPolicyCheck should be + * scheduled to run with no initialdelay + */ + assertTrue("placementPolicyCheck should have executed", latch.await(20, TimeUnit.SECONDS)); + for (int i = 0; i < 20; i++) { + Thread.sleep(100); + if (placementPolicyCheckStatsLogger.getSuccessCount() >= 1) { + break; + } + } + assertEquals("PLACEMENT_POLICY_CHECK_TIME SuccessCount", 1, + placementPolicyCheckStatsLogger.getSuccessCount()); + return statsLogger; + } + + private ZooKeeper getZk(PulsarMetadataClientDriver pulsarMetadataClientDriver) throws Exception { + PulsarLedgerManagerFactory pulsarLedgerManagerFactory = + (PulsarLedgerManagerFactory) pulsarMetadataClientDriver.getLedgerManagerFactory(); + Field field = pulsarLedgerManagerFactory.getClass().getDeclaredField("store"); + field.setAccessible(true); + ZKMetadataStore zkMetadataStore = (ZKMetadataStore) field.get(pulsarLedgerManagerFactory); + return zkMetadataStore.getZkClient(); + } +} From eca14b3b7cd0b3e6c872b961a05d494e3db31a57 Mon Sep 17 00:00:00 2001 From: Yan Zhao Date: Sun, 8 Oct 2023 16:11:33 +0800 Subject: [PATCH 111/699] [fix] [metadata] Fix zookeeper related flacky test (#21310) Fix zookeeper related flacky test (cherry picked from commit 9ab7417edc20b6618bac8f66921815c7b9d5e5b8) --- pulsar-metadata/pom.xml | 7 +- .../replication/AuditorBookieTest.java | 1 - .../AuditorCheckAllLedgersTaskTest.java | 1 - .../replication/AuditorLedgerCheckerTest.java | 1 - .../AuditorPeriodicBookieCheckTest.java | 1 - .../replication/AuditorPeriodicCheckTest.java | 1 - .../AuditorPlacementPolicyCheckTaskTest.java | 1 - .../AuditorPlacementPolicyCheckTest.java | 1 - .../AuditorReplicasCheckTaskTest.java | 1 - .../replication/AuditorReplicasCheckTest.java | 1 - .../AuditorRollingRestartTest.java | 1 - .../replication/AuthAutoRecoveryTest.java | 1 - .../replication/AutoRecoveryMainTest.java | 6 - .../BookKeeperClusterTestCase.java | 851 ++++++++++++++++++ .../replication/BookieAutoRecoveryTest.java | 1 - .../replication/BookieLedgerIndexTest.java | 1 - ...estAutoRecoveryAlongWithBookieServers.java | 1 - .../replication/TestReplicationWorker.java | 1 - .../bookkeeper/replication/ZooKeeperUtil.java | 215 +++++ .../test/BookKeeperClusterTestCase.java | 3 +- .../client/BookKeeperTestClient.java | 1 - 21 files changed, 1073 insertions(+), 25 deletions(-) create mode 100644 pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookKeeperClusterTestCase.java create mode 100644 pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/ZooKeeperUtil.java diff --git a/pulsar-metadata/pom.xml b/pulsar-metadata/pom.xml index a868fa2bd9774..8b1496c66fea7 100644 --- a/pulsar-metadata/pom.xml +++ b/pulsar-metadata/pom.xml @@ -83,7 +83,12 @@ - + + ${project.groupId} + testmocks + ${project.version} + test + org.xerial.snappy snappy-java diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java index 9750fb52d41a3..14cdf3e1fc29c 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorBookieTest.java @@ -31,7 +31,6 @@ import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieServer; -import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.pulsar.metadata.bookkeeper.PulsarLedgerAuditorManager; import org.apache.zookeeper.ZooKeeper; import org.slf4j.Logger; diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorCheckAllLedgersTaskTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorCheckAllLedgersTaskTest.java index 5c0a3f39325e5..6b58c72af0766 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorCheckAllLedgersTaskTest.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorCheckAllLedgersTaskTest.java @@ -30,7 +30,6 @@ import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.stats.NullStatsLogger; -import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.test.TestStatsProvider; import org.apache.pulsar.metadata.api.MetadataStoreConfig; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java index ffd71f9311f42..ec5f77f79464b 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java @@ -67,7 +67,6 @@ import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.stats.NullStatsLogger; -import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.pulsar.metadata.api.MetadataStoreConfig; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java index 4acb207570a2d..9e8c5a54a5d91 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicBookieCheckTest.java @@ -30,7 +30,6 @@ import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.net.BookieSocketAddress; -import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.annotations.AfterMethod; diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java index 8eb04ce7185d7..9c5805dc536d6 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPeriodicCheckTest.java @@ -66,7 +66,6 @@ import org.apache.bookkeeper.stats.Counter; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.test.TestStatsProvider; import org.apache.bookkeeper.test.TestStatsProvider.TestOpStatsLogger; import org.apache.bookkeeper.test.TestStatsProvider.TestStatsLogger; diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTaskTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTaskTest.java index 1bafb8589d91a..8b9c0b143028a 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTaskTest.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTaskTest.java @@ -31,7 +31,6 @@ import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.stats.NullStatsLogger; -import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.test.TestStatsProvider; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java index 159a4e88a33bd..5637819a9275b 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorPlacementPolicyCheckTest.java @@ -53,7 +53,6 @@ import org.apache.bookkeeper.replication.ReplicationException.UnavailableException; import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.NullStatsLogger; -import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.test.TestStatsProvider; import org.apache.bookkeeper.test.TestStatsProvider.TestOpStatsLogger; import org.apache.bookkeeper.test.TestStatsProvider.TestStatsLogger; diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTaskTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTaskTest.java index 21dd2807b75d3..62162bd25f427 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTaskTest.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTaskTest.java @@ -31,7 +31,6 @@ import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.stats.NullStatsLogger; -import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.test.TestStatsProvider; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java index a4d6d86deced2..2e9dbc158597d 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorReplicasCheckTest.java @@ -57,7 +57,6 @@ import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.test.TestStatsProvider; import org.apache.bookkeeper.test.TestStatsProvider.TestOpStatsLogger; import org.apache.bookkeeper.test.TestStatsProvider.TestStatsLogger; diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorRollingRestartTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorRollingRestartTest.java index 2c458d635f528..3e5081ed0ef9d 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorRollingRestartTest.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorRollingRestartTest.java @@ -29,7 +29,6 @@ import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.LedgerUnderreplicationManager; import org.apache.bookkeeper.net.BookieId; -import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.test.TestCallbacks; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuthAutoRecoveryTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuthAutoRecoveryTest.java index db338d1bb4b39..41e159b77714f 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuthAutoRecoveryTest.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuthAutoRecoveryTest.java @@ -27,7 +27,6 @@ import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.proto.ClientConnectionPeer; -import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.annotations.AfterMethod; diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java index 62416968142b1..1d741c551ddb9 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AutoRecoveryMainTest.java @@ -25,10 +25,8 @@ import java.io.IOException; import java.lang.reflect.Field; import java.util.concurrent.TimeUnit; -import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.bookie.BookieImpl; import org.apache.bookkeeper.net.BookieId; -import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.util.TestUtils; import org.apache.pulsar.metadata.bookkeeper.PulsarLedgerManagerFactory; import org.apache.pulsar.metadata.bookkeeper.PulsarMetadataClientDriver; @@ -42,7 +40,6 @@ /** * Test the AuditorPeer. */ -@Slf4j public class AutoRecoveryMainTest extends BookKeeperClusterTestCase { public AutoRecoveryMainTest() throws Exception { @@ -68,7 +65,6 @@ public void tearDown() throws Exception { */ @Test public void testStartup() throws Exception { - log.info("testStartup()"); confByIndex(0).setMetadataServiceUri( zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); AutoRecoveryMain main = new AutoRecoveryMain(confByIndex(0)); @@ -89,7 +85,6 @@ public void testStartup() throws Exception { */ @Test public void testShutdown() throws Exception { - log.info("testShutdown()"); confByIndex(0).setMetadataServiceUri( zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); AutoRecoveryMain main = new AutoRecoveryMain(confByIndex(0)); @@ -113,7 +108,6 @@ public void testShutdown() throws Exception { */ @Test public void testAutoRecoverySessionLoss() throws Exception { - log.info("testAutoRecoverySessionLoss()"); confByIndex(0).setMetadataServiceUri( zkUtil.getMetadataServiceUri().replaceAll("zk://", "metadata-store:").replaceAll("/ledgers", "")); confByIndex(1).setMetadataServiceUri( 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 new file mode 100644 index 0000000000000..c681a1f0764ee --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookKeeperClusterTestCase.java @@ -0,0 +1,851 @@ +/* + * 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. + */ +/** + * This file is derived from BookKeeperClusterTestCase from Apache BookKeeper + * http://bookkeeper.apache.org + */ + +package org.apache.bookkeeper.replication; + +import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE; +import static org.apache.pulsar.common.util.PortManager.nextLockedFreePort; +import static org.testng.Assert.assertFalse; +import com.google.common.base.Stopwatch; +import java.io.File; +import java.io.IOException; +import java.lang.reflect.Method; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.bookkeeper.bookie.Bookie; +import org.apache.bookkeeper.bookie.BookieException; +import org.apache.bookkeeper.client.BookKeeperTestClient; +import org.apache.bookkeeper.client.TestStatsProvider; +import org.apache.bookkeeper.common.allocator.PoolingPolicy; +import org.apache.bookkeeper.conf.AbstractConfiguration; +import org.apache.bookkeeper.conf.ClientConfiguration; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.conf.TestBKConfiguration; +import org.apache.bookkeeper.metastore.InMemoryMetaStore; +import org.apache.bookkeeper.net.BookieId; +import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.proto.BookieServer; +import org.apache.bookkeeper.test.ServerTester; +import org.apache.bookkeeper.test.TmpDirs; +import org.apache.bookkeeper.test.ZooKeeperCluster; +import org.apache.bookkeeper.test.ZooKeeperClusterUtil; +import org.apache.pulsar.common.util.PortManager; +import org.apache.pulsar.metadata.api.MetadataStoreConfig; +import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; +import org.apache.pulsar.metadata.impl.FaultInjectionMetadataStore; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooKeeper; +import org.awaitility.Awaitility; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.AfterTest; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.BeforeTest; + +/** + * A class runs several bookie servers for testing. + */ +public abstract class BookKeeperClusterTestCase { + + static final Logger LOG = LoggerFactory.getLogger(BookKeeperClusterTestCase.class); + + protected String testName; + + @BeforeMethod + public void handleTestMethodName(Method method) { + testName = method.getName(); + } + + // Metadata service related variables + protected final ZooKeeperCluster zkUtil; + protected ZooKeeper zkc; + protected String metadataServiceUri; + protected FaultInjectionMetadataStore metadataStore; + + // BookKeeper related variables + protected final TmpDirs tmpDirs = new TmpDirs(); + protected final List servers = new LinkedList<>(); + + protected int numBookies; + protected BookKeeperTestClient bkc; + protected boolean useUUIDasBookieId = true; + + /* + * Loopback interface is set as the listening interface and allowloopback is + * set to true in this server config. So bookies in this test process would + * bind to loopback address. + */ + protected final ServerConfiguration baseConf = TestBKConfiguration.newServerConfiguration(); + protected final ClientConfiguration baseClientConf = TestBKConfiguration.newClientConfiguration(); + + private boolean isAutoRecoveryEnabled; + protected ExecutorService executor; + private final List bookiePorts = new ArrayList<>(); + + SynchronousQueue asyncExceptions = new SynchronousQueue<>(); + protected void captureThrowable(Runnable c) { + try { + c.run(); + } catch (Throwable e) { + LOG.error("Captured error: ", e); + asyncExceptions.add(e); + } + } + + public BookKeeperClusterTestCase(int numBookies) { + this(numBookies, 120); + } + + public BookKeeperClusterTestCase(int numBookies, int testTimeoutSecs) { + this(numBookies, 1, testTimeoutSecs); + } + + public BookKeeperClusterTestCase(int numBookies, int numOfZKNodes, int testTimeoutSecs) { + this.numBookies = numBookies; + if (numOfZKNodes == 1) { + zkUtil = new ZooKeeperUtil(getLedgersRootPath()); + } else { + try { + zkUtil = new ZooKeeperClusterUtil(numOfZKNodes); + } catch (IOException | KeeperException | InterruptedException e) { + throw new RuntimeException(e); + } + } + } + + @BeforeTest + public void setUp() throws Exception { + setUp(getLedgersRootPath()); + } + + protected void setUp(String ledgersRootPath) throws Exception { + LOG.info("Setting up test {}", getClass()); + InMemoryMetaStore.reset(); + setMetastoreImplClass(baseConf); + setMetastoreImplClass(baseClientConf); + executor = Executors.newCachedThreadPool(); + + Stopwatch sw = Stopwatch.createStarted(); + try { + // start zookeeper service + startZKCluster(); + // start bookkeeper service + this.metadataServiceUri = getMetadataServiceUri(ledgersRootPath); + startBKCluster(metadataServiceUri); + LOG.info("Setup testcase {} @ metadata service {} in {} ms.", + testName, metadataServiceUri, sw.elapsed(TimeUnit.MILLISECONDS)); + } catch (Exception e) { + LOG.error("Error setting up", e); + throw e; + } + } + + protected String getMetadataServiceUri(String ledgersRootPath) { + return zkUtil.getMetadataServiceUri(ledgersRootPath); + } + + private String getLedgersRootPath() { + return changeLedgerPath() + "/ledgers"; + } + + protected String changeLedgerPath() { + return ""; + } + + @AfterTest(alwaysRun = true) + public void tearDown() throws Exception { + boolean failed = false; + for (Throwable e : asyncExceptions) { + LOG.error("Got async exception: ", e); + failed = true; + } + assertFalse(failed, "Async failure"); + Stopwatch sw = Stopwatch.createStarted(); + LOG.info("TearDown"); + Exception tearDownException = null; + // stop bookkeeper service + try { + stopBKCluster(); + } catch (Exception e) { + LOG.error("Got Exception while trying to stop BKCluster", e); + tearDownException = e; + } + // stop zookeeper service + try { + // cleanup for metrics. + metadataStore.close(); + stopZKCluster(); + } catch (Exception e) { + LOG.error("Got Exception while trying to stop ZKCluster", e); + tearDownException = e; + } + // cleanup temp dirs + try { + tmpDirs.cleanup(); + } catch (Exception e) { + LOG.error("Got Exception while trying to cleanupTempDirs", e); + tearDownException = e; + } + + executor.shutdownNow(); + + LOG.info("Tearing down test {} in {} ms.", testName, sw.elapsed(TimeUnit.MILLISECONDS)); + if (tearDownException != null) { + throw tearDownException; + } + } + + /** + * Start zookeeper cluster. + * + * @throws Exception + */ + protected void startZKCluster() throws Exception { + zkUtil.startCluster(); + zkc = zkUtil.getZooKeeperClient(); + metadataStore = new FaultInjectionMetadataStore( + MetadataStoreExtended.create(zkUtil.getZooKeeperConnectString(), + MetadataStoreConfig.builder().build())); + } + + /** + * Stop zookeeper cluster. + * + * @throws Exception + */ + protected void stopZKCluster() throws Exception { + zkUtil.killCluster(); + } + + /** + * Start cluster. Also, starts the auto recovery process for each bookie, if + * isAutoRecoveryEnabled is true. + * + * @throws Exception + */ + protected void startBKCluster(String metadataServiceUri) throws Exception { + baseConf.setMetadataServiceUri(metadataServiceUri); + baseClientConf.setMetadataServiceUri(metadataServiceUri); + baseClientConf.setAllocatorPoolingPolicy(PoolingPolicy.UnpooledHeap); + + if (numBookies > 0) { + bkc = new BookKeeperTestClient(baseClientConf, new TestStatsProvider()); + } + + // Create Bookie Servers (B1, B2, B3) + for (int i = 0; i < numBookies; i++) { + bookiePorts.add(startNewBookie()); + } + } + + /** + * Stop cluster. Also, stops all the auto recovery processes for the bookie + * cluster, if isAutoRecoveryEnabled is true. + * + * @throws Exception + */ + protected void stopBKCluster() throws Exception { + if (bkc != null) { + bkc.close(); + } + + for (ServerTester t : servers) { + t.shutdown(); + } + servers.clear(); + bookiePorts.removeIf(PortManager::releaseLockedPort); + } + + protected ServerConfiguration newServerConfiguration() throws Exception { + File f = tmpDirs.createNew("bookie", "test"); + + int port; + if (baseConf.isEnableLocalTransport() || !baseConf.getAllowEphemeralPorts()) { + port = nextLockedFreePort(); + } else { + port = 0; + } + return newServerConfiguration(port, f, new File[] { f }); + } + + protected ClientConfiguration newClientConfiguration() { + return new ClientConfiguration(baseConf); + } + + protected ServerConfiguration newServerConfiguration(int port, File journalDir, File[] ledgerDirs) { + ServerConfiguration conf = new ServerConfiguration(baseConf); + conf.setBookiePort(port); + conf.setJournalDirName(journalDir.getPath()); + String[] ledgerDirNames = new String[ledgerDirs.length]; + for (int i = 0; i < ledgerDirs.length; i++) { + ledgerDirNames[i] = ledgerDirs[i].getPath(); + } + conf.setLedgerDirNames(ledgerDirNames); + conf.setEnableTaskExecutionStats(true); + conf.setAllocatorPoolingPolicy(PoolingPolicy.UnpooledHeap); + return conf; + } + + protected void stopAllBookies() throws Exception { + stopAllBookies(true); + } + + protected void stopAllBookies(boolean shutdownClient) throws Exception { + for (ServerTester t : servers) { + t.shutdown(); + } + servers.clear(); + if (shutdownClient && bkc != null) { + bkc.close(); + bkc = null; + } + } + + protected String newMetadataServiceUri(String ledgersRootPath) { + return zkUtil.getMetadataServiceUri(ledgersRootPath); + } + + protected String newMetadataServiceUri(String ledgersRootPath, String type) { + return zkUtil.getMetadataServiceUri(ledgersRootPath, type); + } + + /** + * Get bookie address for bookie at index. + */ + public BookieId getBookie(int index) throws Exception { + return servers.get(index).getServer().getBookieId(); + } + + protected List bookieAddresses() throws Exception { + List bookieIds = new ArrayList<>(); + for (ServerTester a : servers) { + bookieIds.add(a.getServer().getBookieId()); + } + return bookieIds; + } + + protected List bookieLedgerDirs() throws Exception { + return servers.stream() + .flatMap(t -> Arrays.stream(t.getConfiguration().getLedgerDirs())) + .collect(Collectors.toList()); + } + + protected List bookieJournalDirs() throws Exception { + return servers.stream() + .flatMap(t -> Arrays.stream(t.getConfiguration().getJournalDirs())) + .collect(Collectors.toList()); + } + + protected BookieId addressByIndex(int index) throws Exception { + return servers.get(index).getServer().getBookieId(); + } + + protected BookieServer serverByIndex(int index) throws Exception { + return servers.get(index).getServer(); + } + + protected ServerConfiguration confByIndex(int index) throws Exception { + return servers.get(index).getConfiguration(); + } + + private Optional byAddress(BookieId addr) throws UnknownHostException { + for (ServerTester s : servers) { + if (s.getServer().getBookieId().equals(addr)) { + return Optional.of(s); + } + } + return Optional.empty(); + } + + protected int indexOfServer(BookieServer b) throws Exception { + for (int i = 0; i < servers.size(); i++) { + if (servers.get(i).getServer().equals(b)) { + return i; + } + } + return -1; + } + + protected int lastBookieIndex() { + return servers.size() - 1; + } + + protected int bookieCount() { + return servers.size(); + } + + private OptionalInt indexByAddress(BookieId addr) throws UnknownHostException { + for (int i = 0; i < servers.size(); i++) { + if (addr.equals(servers.get(i).getServer().getBookieId())) { + return OptionalInt.of(i); + } + } + return OptionalInt.empty(); + } + + /** + * Get bookie configuration for bookie. + */ + public ServerConfiguration getBkConf(BookieId addr) throws Exception { + return byAddress(addr).get().getConfiguration(); + } + + /** + * Kill a bookie by its socket address. Also, stops the autorecovery process + * for the corresponding bookie server, if isAutoRecoveryEnabled is true. + * + * @param addr + * Socket Address + * @return the configuration of killed bookie + * @throws InterruptedException + */ + public ServerConfiguration killBookie(BookieId addr) throws Exception { + Optional tester = byAddress(addr); + if (tester.isPresent()) { + if (tester.get().autoRecovery != null + && tester.get().autoRecovery.getAuditor() != null + && tester.get().autoRecovery.getAuditor().isRunning()) { + LOG.warn("Killing bookie {} who is the current Auditor", addr); + } + servers.remove(tester.get()); + tester.get().shutdown(); + return tester.get().getConfiguration(); + } + return null; + } + + /** + * Set the bookie identified by its socket address to readonly. + * + * @param addr + * Socket Address + * @throws InterruptedException + */ + public void setBookieToReadOnly(BookieId addr) throws Exception { + Optional tester = byAddress(addr); + if (tester.isPresent()) { + tester.get().getServer().getBookie().getStateManager().transitionToReadOnlyMode().get(); + } + } + + /** + * Kill a bookie by index. Also, stops the respective auto recovery process + * for this bookie, if isAutoRecoveryEnabled is true. + * + * @param index + * Bookie Index + * @return the configuration of killed bookie + * @throws InterruptedException + * @throws IOException + */ + public ServerConfiguration killBookie(int index) throws Exception { + ServerTester tester = servers.remove(index); + tester.shutdown(); + return tester.getConfiguration(); + } + + /** + * Kill bookie by index and verify that it's stopped. + * + * @param index index of bookie to kill + * + * @return configuration of killed bookie + */ + public ServerConfiguration killBookieAndWaitForZK(int index) throws Exception { + ServerTester tester = servers.get(index); // IKTODO: this method is awful + ServerConfiguration ret = killBookie(index); + while (zkc.exists("/ledgers/" + AVAILABLE_NODE + "/" + + tester.getServer().getBookieId().toString(), false) != null) { + Thread.sleep(500); + } + return ret; + } + + /** + * Sleep a bookie. + * + * @param addr + * Socket Address + * @param seconds + * Sleep seconds + * @return Count Down latch which will be counted down just after sleep begins + * @throws InterruptedException + * @throws IOException + */ + public CountDownLatch sleepBookie(BookieId addr, final int seconds) + throws Exception { + Optional tester = byAddress(addr); + if (tester.isPresent()) { + CountDownLatch latch = new CountDownLatch(1); + Thread sleeper = new Thread() { + @Override + public void run() { + try { + tester.get().getServer().suspendProcessing(); + LOG.info("bookie {} is asleep", tester.get().getAddress()); + latch.countDown(); + Thread.sleep(seconds * 1000); + tester.get().getServer().resumeProcessing(); + LOG.info("bookie {} is awake", tester.get().getAddress()); + } catch (Exception e) { + LOG.error("Error suspending bookie", e); + } + } + }; + sleeper.start(); + return latch; + } else { + throw new IOException("Bookie not found"); + } + } + + /** + * Sleep a bookie until I count down the latch. + * + * @param addr + * Socket Address + * @param l + * Latch to wait on + * @throws InterruptedException + * @throws IOException + */ + public void sleepBookie(BookieId addr, final CountDownLatch l) + throws InterruptedException, IOException { + final CountDownLatch suspendLatch = new CountDownLatch(1); + sleepBookie(addr, l, suspendLatch); + suspendLatch.await(); + } + + public void sleepBookie(BookieId addr, final CountDownLatch l, final CountDownLatch suspendLatch) + throws InterruptedException, IOException { + Optional tester = byAddress(addr); + if (tester.isPresent()) { + BookieServer bookie = tester.get().getServer(); + LOG.info("Sleep bookie {}.", addr); + Thread sleeper = new Thread() { + @Override + public void run() { + try { + bookie.suspendProcessing(); + if (null != suspendLatch) { + suspendLatch.countDown(); + } + l.await(); + bookie.resumeProcessing(); + } catch (Exception e) { + LOG.error("Error suspending bookie", e); + } + } + }; + sleeper.start(); + } else { + throw new IOException("Bookie not found"); + } + } + + /** + * Restart bookie servers. Also restarts all the respective auto recovery + * process, if isAutoRecoveryEnabled is true. + * + * @throws InterruptedException + * @throws IOException + * @throws KeeperException + * @throws BookieException + */ + public void restartBookies() + throws Exception { + restartBookies(c -> c); + } + + /** + * Restart a bookie. Also restart the respective auto recovery process, + * if isAutoRecoveryEnabled is true. + * + * @param addr + * @throws InterruptedException + * @throws IOException + * @throws KeeperException + * @throws BookieException + */ + public void restartBookie(BookieId addr) throws Exception { + OptionalInt toRemove = indexByAddress(addr); + if (toRemove.isPresent()) { + ServerConfiguration newConfig = killBookie(toRemove.getAsInt()); + Thread.sleep(1000); + startAndAddBookie(newConfig); + } else { + throw new IOException("Bookie not found"); + } + } + + public void restartBookies(Function reconfFunction) + throws Exception { + // shut down bookie server + List confs = new ArrayList<>(); + for (ServerTester server : servers) { + server.shutdown(); + confs.add(server.getConfiguration()); + } + servers.clear(); + Thread.sleep(1000); + // restart them to ensure we can't + for (ServerConfiguration conf : confs) { + // ensure the bookie port is loaded correctly + startAndAddBookie(reconfFunction.apply(conf)); + } + } + + /** + * Helper method to startup a new bookie server with the indicated port + * number. Also, starts the auto recovery process, if the + * isAutoRecoveryEnabled is set true. + * + * @throws IOException + */ + public int startNewBookie() + throws Exception { + return startNewBookieAndReturnAddress().getPort(); + } + + public BookieSocketAddress startNewBookieAndReturnAddress() + throws Exception { + ServerConfiguration conf = newServerConfiguration(); + LOG.info("Starting new bookie on port: {}", conf.getBookiePort()); + return startAndAddBookie(conf).getServer().getLocalAddress(); + } + + public BookieId startNewBookieAndReturnBookieId() + throws Exception { + ServerConfiguration conf = newServerConfiguration(); + LOG.info("Starting new bookie on port: {}", conf.getBookiePort()); + return startAndAddBookie(conf).getServer().getBookieId(); + } + + protected ServerTester startAndAddBookie(ServerConfiguration conf) throws Exception { + ServerTester server = startBookie(conf); + servers.add(server); + return server; + } + + protected ServerTester startAndAddBookie(ServerConfiguration conf, Bookie b) throws Exception { + ServerTester server = startBookie(conf, b); + servers.add(server); + return server; + } + /** + * Helper method to startup a bookie server using a configuration object. + * Also, starts the auto recovery process if isAutoRecoveryEnabled is true. + * + * @param conf + * Server Configuration Object + * + */ + protected ServerTester startBookie(ServerConfiguration conf) + throws Exception { + ServerTester tester = new ServerTester(conf); + + if (bkc == null) { + bkc = new BookKeeperTestClient(baseClientConf, new TestStatsProvider()); + } + + BookieId address = tester.getServer().getBookieId(); + Future waitForBookie = conf.isForceReadOnlyBookie() + ? bkc.waitForReadOnlyBookie(address) + : bkc.waitForWritableBookie(address); + + tester.getServer().start(); + + waitForBookie.get(30, TimeUnit.SECONDS); + LOG.info("New bookie '{}' has been created.", address); + + if (isAutoRecoveryEnabled()) { + tester.startAutoRecovery(); + } + + int port = conf.getBookiePort(); + + Awaitility.await().atMost(30, TimeUnit.SECONDS).until(() -> { + while (zkc.exists("/ledgers/" + AVAILABLE_NODE + "/" + + tester.getServer().getBookieId().toString(), false) == null) { + Thread.sleep(100); + } + return true; + }); + bkc.readBookiesBlocking(); + + LOG.info("New bookie on port " + port + " has been created."); + + return tester; + } + + /** + * Start a bookie with the given bookie instance. Also, starts the auto + * recovery for this bookie, if isAutoRecoveryEnabled is true. + */ + protected ServerTester startBookie(ServerConfiguration conf, final Bookie b) + throws Exception { + ServerTester tester = new ServerTester(conf, b); + if (bkc == null) { + bkc = new BookKeeperTestClient(baseClientConf, new TestStatsProvider()); + } + BookieId address = tester.getServer().getBookieId(); + Future waitForBookie = conf.isForceReadOnlyBookie() + ? bkc.waitForReadOnlyBookie(address) + : bkc.waitForWritableBookie(address); + + tester.getServer().start(); + + waitForBookie.get(30, TimeUnit.SECONDS); + + if (isAutoRecoveryEnabled()) { + tester.startAutoRecovery(); + } + + int port = conf.getBookiePort(); + Awaitility.await().atMost(30, TimeUnit.SECONDS).until(() -> + metadataStore.exists( + getLedgersRootPath() + "/available/" + address).join() + ); + bkc.readBookiesBlocking(); + + LOG.info("New bookie '{}' has been created.", address); + return tester; + } + + public void setMetastoreImplClass(AbstractConfiguration conf) { + conf.setMetastoreImplClass(InMemoryMetaStore.class.getName()); + } + + /** + * Flags used to enable/disable the auto recovery process. If it is enabled, + * starting the bookie server will starts the auto recovery process for that + * bookie. Also, stopping bookie will stops the respective auto recovery + * process. + * + * @param isAutoRecoveryEnabled + * Value true will enable the auto recovery process. Value false + * will disable the auto recovery process + */ + public void setAutoRecoveryEnabled(boolean isAutoRecoveryEnabled) { + this.isAutoRecoveryEnabled = isAutoRecoveryEnabled; + } + + /** + * Flag used to check whether auto recovery process is enabled/disabled. By + * default the flag is false. + * + * @return true, if the auto recovery is enabled. Otherwise return false. + */ + public boolean isAutoRecoveryEnabled() { + return isAutoRecoveryEnabled; + } + + /** + * Will starts the auto recovery process for the bookie servers. One auto + * recovery process per each bookie server, if isAutoRecoveryEnabled is + * enabled. + */ + public void startReplicationService() throws Exception { + for (ServerTester t : servers) { + t.startAutoRecovery(); + } + } + + /** + * Will stops all the auto recovery processes for the bookie cluster, if + * isAutoRecoveryEnabled is true. + */ + public void stopReplicationService() throws Exception{ + for (ServerTester t : servers) { + t.stopAutoRecovery(); + } + } + + public Auditor getAuditor(int timeout, TimeUnit unit) throws Exception { + final long timeoutAt = System.nanoTime() + TimeUnit.NANOSECONDS.convert(timeout, unit); + while (System.nanoTime() < timeoutAt) { + for (ServerTester t : servers) { + Auditor a = t.getAuditor(); + ReplicationWorker replicationWorker = t.getReplicationWorker(); + + // found a candidate Auditor + ReplicationWorker + if (a != null && a.isRunning() + && replicationWorker != null && replicationWorker.isRunning()) { + int deathWatchInterval = t.getConfiguration().getDeathWatchInterval(); + Thread.sleep(deathWatchInterval + 1000); + } + + // double check, because in the meantime AutoRecoveryDeathWatcher may have killed the + // AutoRecovery daemon + if (a != null && a.isRunning() + && replicationWorker != null && replicationWorker.isRunning()) { + LOG.info("Found Auditor Bookie {}", t.getServer().getBookieId()); + return a; + } + } + Thread.sleep(100); + } + throw new Exception("No auditor found"); + } + + /** + * Check whether the InetSocketAddress was created using a hostname or an IP + * address. Represent as 'hostname/IPaddress' if the InetSocketAddress was + * created using hostname. Represent as '/IPaddress' if the + * InetSocketAddress was created using an IPaddress + * + * @param bookieId id + * @return true if the address was created using an IP address, false if the + * address was created using a hostname + */ + public boolean isCreatedFromIp(BookieId bookieId) { + BookieSocketAddress addr = bkc.getBookieAddressResolver().resolve(bookieId); + return addr.getSocketAddress().toString().startsWith("/"); + } + + public void resetBookieOpLoggers() { + servers.forEach(t -> t.getStatsProvider().clear()); + } + + public TestStatsProvider getStatsProvider(BookieId addr) throws UnknownHostException { + return byAddress(addr).get().getStatsProvider(); + } + + public TestStatsProvider getStatsProvider(int index) throws Exception { + return servers.get(index).getStatsProvider(); + } + +} diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java index c8c76302b89e1..888303d3e665c 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookieAutoRecoveryTest.java @@ -44,7 +44,6 @@ import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.proto.BookieServer; import org.apache.bookkeeper.stats.NullStatsLogger; -import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookieLedgerIndexTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookieLedgerIndexTest.java index eb9f95ffdf7a5..1d5cf868cce65 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookieLedgerIndexTest.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/BookieLedgerIndexTest.java @@ -35,7 +35,6 @@ import org.apache.bookkeeper.meta.LayoutManager; import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; -import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.pulsar.metadata.api.MetadataStoreConfig; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; import org.apache.pulsar.metadata.bookkeeper.PulsarLayoutManager; diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java index 8a2e7f2747a22..11797c8373715 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/TestAutoRecoveryAlongWithBookieServers.java @@ -27,7 +27,6 @@ import org.apache.bookkeeper.client.LedgerEntry; import org.apache.bookkeeper.client.LedgerHandle; import org.apache.bookkeeper.net.BookieId; -import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.util.BookKeeperConstants; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java index ca02f91d1de36..7938feaba19fe 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/TestReplicationWorker.java @@ -78,7 +78,6 @@ import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; -import org.apache.bookkeeper.test.BookKeeperClusterTestCase; import org.apache.bookkeeper.test.TestStatsProvider; import org.apache.bookkeeper.test.TestStatsProvider.TestStatsLogger; import org.apache.bookkeeper.util.BookKeeperConstants; diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/ZooKeeperUtil.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/ZooKeeperUtil.java new file mode 100644 index 0000000000000..5113edb72c49a --- /dev/null +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/ZooKeeperUtil.java @@ -0,0 +1,215 @@ +/* + * 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. + */ +/** + * This file is derived from ZooKeeperUtil from Apache BookKeeper + * http://bookkeeper.apache.org + */ + +package org.apache.bookkeeper.replication; + +import static org.testng.Assert.assertTrue; +import java.io.File; +import java.io.IOException; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import org.apache.bookkeeper.test.ZooKeeperCluster; +import org.apache.bookkeeper.util.IOUtils; +import org.apache.bookkeeper.util.ZkUtils; +import org.apache.bookkeeper.zookeeper.ZooKeeperClient; +import org.apache.commons.io.FileUtils; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.server.NIOServerCnxnFactory; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.apache.zookeeper.test.ClientBase; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Test the zookeeper utilities. + */ +public class ZooKeeperUtil implements ZooKeeperCluster { + + static { + // org.apache.zookeeper.test.ClientBase uses FourLetterWordMain, from 3.5.3 four letter words + // are disabled by default due to security reasons + System.setProperty("zookeeper.4lw.commands.whitelist", "*"); + } + static final Logger LOG = LoggerFactory.getLogger(ZooKeeperUtil.class); + + // ZooKeeper related variables + protected Integer zooKeeperPort = 0; + private InetSocketAddress zkaddr; + + protected ZooKeeperServer zks; + protected ZooKeeper zkc; // zookeeper client + protected NIOServerCnxnFactory serverFactory; + protected File zkTmpDir; + private String connectString; + private String ledgersRootPath; + + public ZooKeeperUtil(String ledgersRootPath) { + this.ledgersRootPath = ledgersRootPath; + String loopbackIPAddr = InetAddress.getLoopbackAddress().getHostAddress(); + zkaddr = new InetSocketAddress(loopbackIPAddr, 0); + connectString = loopbackIPAddr + ":" + zooKeeperPort; + } + + @Override + public ZooKeeper getZooKeeperClient() { + return zkc; + } + + @Override + public String getZooKeeperConnectString() { + return connectString; + } + + @Override + public String getMetadataServiceUri() { + return getMetadataServiceUri("/ledgers"); + } + + @Override + public String getMetadataServiceUri(String zkLedgersRootPath) { + return "zk://" + connectString + zkLedgersRootPath; + } + + @Override + public String getMetadataServiceUri(String zkLedgersRootPath, String type) { + return "zk+" + type + "://" + connectString + zkLedgersRootPath; + } + + @Override + public void startCluster() throws Exception { + // create a ZooKeeper server(dataDir, dataLogDir, port) + LOG.debug("Running ZK server"); + ClientBase.setupTestEnv(); + zkTmpDir = IOUtils.createTempDir("zookeeper", "test"); + + // start the server and client. + restartCluster(); + + // create default bk ensemble + createBKEnsemble(ledgersRootPath); + } + + @Override + public void createBKEnsemble(String ledgersPath) throws KeeperException, InterruptedException { + int last = ledgersPath.lastIndexOf('/'); + if (last > 0) { + String pathToCreate = ledgersPath.substring(0, last); + CompletableFuture future = new CompletableFuture<>(); + if (zkc.exists(pathToCreate, false) == null) { + ZkUtils.asyncCreateFullPathOptimistic(zkc, + pathToCreate, + new byte[0], + ZooDefs.Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT, (i, s, o, s1) -> { + future.complete(null); + }, null); + } + future.join(); + } + + ZooKeeperCluster.super.createBKEnsemble(ledgersPath); + } + @Override + public void restartCluster() throws Exception { + zks = new ZooKeeperServer(zkTmpDir, zkTmpDir, + ZooKeeperServer.DEFAULT_TICK_TIME); + serverFactory = new NIOServerCnxnFactory(); + serverFactory.configure(zkaddr, 100); + serverFactory.startup(zks); + + if (0 == zooKeeperPort) { + zooKeeperPort = serverFactory.getLocalPort(); + zkaddr = new InetSocketAddress(zkaddr.getAddress().getHostAddress(), zooKeeperPort); + connectString = zkaddr.getAddress().getHostAddress() + ":" + zooKeeperPort; + } + + boolean b = ClientBase.waitForServerUp(getZooKeeperConnectString(), + ClientBase.CONNECTION_TIMEOUT); + LOG.debug("Server up: " + b); + + // create a zookeeper client + LOG.debug("Instantiate ZK Client"); + zkc = ZooKeeperClient.newBuilder() + .connectString(getZooKeeperConnectString()) + .sessionTimeoutMs(10000) + .build(); + } + + @Override + public void sleepCluster(final int time, + final TimeUnit timeUnit, + final CountDownLatch l) + throws InterruptedException, IOException { + Thread[] allthreads = new Thread[Thread.activeCount()]; + Thread.enumerate(allthreads); + for (final Thread t : allthreads) { + if (t.getName().contains("SyncThread:0")) { + Thread sleeper = new Thread() { + @SuppressWarnings("deprecation") + public void run() { + try { + t.suspend(); + l.countDown(); + timeUnit.sleep(time); + t.resume(); + } catch (Exception e) { + LOG.error("Error suspending thread", e); + } + } + }; + sleeper.start(); + return; + } + } + throw new IOException("ZooKeeper thread not found"); + } + + @Override + public void stopCluster() throws Exception { + if (zkc != null) { + zkc.close(); + } + + // shutdown ZK server + if (serverFactory != null) { + serverFactory.shutdown(); + assertTrue(ClientBase.waitForServerDown(getZooKeeperConnectString(), ClientBase.CONNECTION_TIMEOUT), + "waiting for server down"); + } + if (zks != null) { + zks.getTxnLogFactory().close(); + } + } + + @Override + public void killCluster() throws Exception { + stopCluster(); + FileUtils.deleteDirectory(zkTmpDir); + } +} diff --git a/pulsar-package-management/bookkeeper-storage/src/test/java/org/apache/pulsar/packages/management/storage/bookkeeper/bookkeeper/test/BookKeeperClusterTestCase.java b/pulsar-package-management/bookkeeper-storage/src/test/java/org/apache/pulsar/packages/management/storage/bookkeeper/bookkeeper/test/BookKeeperClusterTestCase.java index 40c2041d4e6c4..43db5ad4ba845 100644 --- a/pulsar-package-management/bookkeeper-storage/src/test/java/org/apache/pulsar/packages/management/storage/bookkeeper/bookkeeper/test/BookKeeperClusterTestCase.java +++ b/pulsar-package-management/bookkeeper-storage/src/test/java/org/apache/pulsar/packages/management/storage/bookkeeper/bookkeeper/test/BookKeeperClusterTestCase.java @@ -70,7 +70,6 @@ import org.apache.bookkeeper.meta.LedgerManager; import org.apache.bookkeeper.meta.LedgerManagerFactory; import org.apache.bookkeeper.meta.MetadataBookieDriver; -import org.apache.bookkeeper.meta.zk.ZKMetadataDriverBase; import org.apache.bookkeeper.metastore.InMemoryMetaStore; import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieSocketAddress; @@ -486,7 +485,7 @@ public ServerConfiguration killBookie(int index) throws Exception { public ServerConfiguration killBookieAndWaitForZK(int index) throws Exception { ServerTester tester = servers.get(index); // IKTODO: this method is awful ServerConfiguration ret = killBookie(index); - while (zkc.exists(ZKMetadataDriverBase.resolveZkLedgersRootPath(baseConf) + "/" + AVAILABLE_NODE + "/" + while (zkc.exists("/ledgers/" + AVAILABLE_NODE + "/" + tester.getServer().getBookieId().toString(), false) != null) { Thread.sleep(500); } 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 d023427e3be31..dd33c2c4532bf 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/BookKeeperTestClient.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/BookKeeperTestClient.java @@ -52,7 +52,6 @@ public BookKeeperTestClient(ClientConfiguration conf, ZooKeeper zkc) throws IOException, InterruptedException, BKException { super(conf, zkc, null, new UnpooledByteBufAllocator(false), NullStatsLogger.INSTANCE, null, null, null); - this.statsProvider = statsProvider; } public BookKeeperTestClient(ClientConfiguration conf) From 763596965740e5929fb9ca8deedc64ca230ddb3c Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Sat, 14 Oct 2023 09:45:05 +0800 Subject: [PATCH 112/699] [fix][broker][branch-3.1] Fix inconsistent topic policy (#21255) --- .../ProxySaslAuthenticationTest.java | 13 +- .../authentication/SaslAuthenticateTest.java | 11 +- .../pulsar/broker/service/BrokerService.java | 257 +++++++++--------- .../SystemTopicBasedTopicPoliciesService.java | 100 +++++-- .../broker/service/TopicPoliciesService.java | 41 +++ .../broker/admin/PersistentTopicsTest.java | 1 + .../broker/admin/TopicPoliciesTest.java | 2 +- .../TopicPoliciesWithBrokerRestartTest.java | 104 +++++++ .../pulsar/broker/admin/TopicsAuthTest.java | 2 + .../pulsar/broker/auth/AuthLogsTest.java | 2 + .../broker/auth/MockAuthentication.java | 6 +- .../auth/MockedPulsarServiceBaseTest.java | 19 ++ .../service/BrokerBookieIsolationTest.java | 3 +- ...temTopicBasedTopicPoliciesServiceTest.java | 6 +- .../persistent/PersistentTopicTest.java | 3 +- ...enticationTlsHostnameVerificationTest.java | 3 + .../AuthorizationProducerConsumerTest.java | 5 + .../client/api/MutualAuthenticationTest.java | 2 +- ...okenAuthenticatedProducerConsumerTest.java | 3 + ...uth2AuthenticatedProducerConsumerTest.java | 14 +- ...reTlsProducerConsumerTestWithAuthTest.java | 22 ++ .../PatternTopicsConsumerImplAuthTest.java | 1 + .../standalone_no_client_auth.conf | 3 +- .../proxy/server/ProxyAuthenticationTest.java | 2 +- .../server/ProxyForwardAuthDataTest.java | 2 +- .../server/ProxyRolesEnforcementTest.java | 2 +- .../server/ProxyWithAuthorizationNegTest.java | 2 + .../pulsar/sql/presto/TestPulsarAuth.java | 4 + .../ExtensibleLoadManagerTest.java | 1 + .../integration/presto/TestPulsarSQLAuth.java | 1 + 30 files changed, 459 insertions(+), 178 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesWithBrokerRestartTest.java diff --git a/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/ProxySaslAuthenticationTest.java b/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/ProxySaslAuthenticationTest.java index 261efe680f862..f0e45aa734afb 100644 --- a/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/ProxySaslAuthenticationTest.java +++ b/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/ProxySaslAuthenticationTest.java @@ -49,6 +49,7 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.auth.AuthenticationSasl; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; +import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.proxy.server.ProxyConfiguration; import org.apache.pulsar.proxy.server.ProxyService; import org.slf4j.Logger; @@ -193,15 +194,17 @@ protected void setup() throws Exception { conf.setAuthenticationProviders(providers); conf.setClusterName("test"); conf.setSuperUserRoles(ImmutableSet.of("client/" + localHostname + "@" + kdc.getRealm())); - - super.init(); - - lookupUrl = new URI(pulsar.getBrokerServiceUrl()); - // set admin auth, to verify admin web resources Map clientSaslConfig = new HashMap<>(); clientSaslConfig.put("saslJaasClientSectionName", "PulsarClient"); clientSaslConfig.put("serverType", "broker"); + conf.setBrokerClientAuthenticationPlugin(AuthenticationSasl.class.getName()); + conf.setBrokerClientAuthenticationParameters(ObjectMapperFactory + .getMapper().getObjectMapper().writeValueAsString(clientSaslConfig)); + + super.init(); + + lookupUrl = new URI(pulsar.getBrokerServiceUrl()); log.info("set client jaas section name: PulsarClient"); admin = PulsarAdmin.builder() .serviceHttpUrl(brokerUrl.toString()) diff --git a/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/SaslAuthenticateTest.java b/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/SaslAuthenticateTest.java index 5cace2221dea8..230c2ad787de4 100644 --- a/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/SaslAuthenticateTest.java +++ b/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/SaslAuthenticateTest.java @@ -58,6 +58,7 @@ import org.apache.pulsar.client.impl.auth.AuthenticationSasl; import org.apache.pulsar.common.api.AuthData; import org.apache.pulsar.common.sasl.SaslConstants; +import org.apache.pulsar.common.util.ObjectMapperFactory; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.AfterMethod; @@ -186,7 +187,12 @@ protected void setup() throws Exception { conf.setAuthenticationProviders(providers); conf.setClusterName("test"); conf.setSuperUserRoles(ImmutableSet.of("client" + "@" + kdc.getRealm())); - + Map clientSaslConfig = new HashMap<>(); + clientSaslConfig.put("saslJaasClientSectionName", "PulsarClient"); + clientSaslConfig.put("serverType", "broker"); + conf.setBrokerClientAuthenticationPlugin(AuthenticationSasl.class.getName()); + conf.setBrokerClientAuthenticationParameters(ObjectMapperFactory + .getMapper().getObjectMapper().writeValueAsString(clientSaslConfig)); super.init(); lookupUrl = new URI(pulsar.getWebServiceAddress()); @@ -197,9 +203,6 @@ protected void setup() throws Exception { .authentication(authSasl)); // set admin auth, to verify admin web resources - Map clientSaslConfig = new HashMap<>(); - clientSaslConfig.put("saslJaasClientSectionName", "PulsarClient"); - clientSaslConfig.put("serverType", "broker"); log.info("set client jaas section name: PulsarClient"); admin = PulsarAdmin.builder() .serviceHttpUrl(brokerUrl.toString()) 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 1702a23ef3db6..05d6bca962b1d 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 @@ -1760,116 +1760,122 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { }); } - public CompletableFuture getManagedLedgerConfig(TopicName topicName) { + public CompletableFuture getManagedLedgerConfig(@Nonnull TopicName topicName) { + requireNonNull(topicName); NamespaceName namespace = topicName.getNamespaceObject(); ServiceConfiguration serviceConfig = pulsar.getConfiguration(); NamespaceResources nsr = pulsar.getPulsarResources().getNamespaceResources(); LocalPoliciesResources lpr = pulsar.getPulsarResources().getLocalPolicies(); - return nsr.getPoliciesAsync(namespace) - .thenCombine(lpr.getLocalPoliciesAsync(namespace), (policies, localPolicies) -> { - PersistencePolicies persistencePolicies = null; - RetentionPolicies retentionPolicies = null; - OffloadPoliciesImpl topicLevelOffloadPolicies = null; - - if (pulsar.getConfig().isTopicLevelPoliciesEnabled() - && !NamespaceService.isSystemServiceNamespace(namespace.toString())) { - final TopicPolicies topicPolicies = pulsar.getTopicPoliciesService() - .getTopicPoliciesIfExists(topicName); - if (topicPolicies != null) { - persistencePolicies = topicPolicies.getPersistence(); - retentionPolicies = topicPolicies.getRetentionPolicies(); - topicLevelOffloadPolicies = topicPolicies.getOffloadPolicies(); - } - } - - if (persistencePolicies == null) { - persistencePolicies = policies.map(p -> p.persistence).orElseGet( - () -> new PersistencePolicies(serviceConfig.getManagedLedgerDefaultEnsembleSize(), - serviceConfig.getManagedLedgerDefaultWriteQuorum(), - serviceConfig.getManagedLedgerDefaultAckQuorum(), - serviceConfig.getManagedLedgerDefaultMarkDeleteRateLimit())); - } + final CompletableFuture> topicPoliciesFuture; + if (pulsar.getConfig().isTopicLevelPoliciesEnabled() + && !NamespaceService.isSystemServiceNamespace(namespace.toString()) + && !SystemTopicNames.isTopicPoliciesSystemTopic(topicName.toString())) { + topicPoliciesFuture = pulsar.getTopicPoliciesService().getTopicPoliciesAsync(topicName); + } else { + topicPoliciesFuture = CompletableFuture.completedFuture(Optional.empty()); + } + return topicPoliciesFuture.thenCompose(topicPoliciesOptional -> { + final CompletableFuture> nsPolicies = nsr.getPoliciesAsync(namespace); + final CompletableFuture> lcPolicies = lpr.getLocalPoliciesAsync(namespace); + return nsPolicies.thenCombine(lcPolicies, (policies, localPolicies) -> { + PersistencePolicies persistencePolicies = null; + RetentionPolicies retentionPolicies = null; + OffloadPoliciesImpl topicLevelOffloadPolicies = null; + if (topicPoliciesOptional.isPresent()) { + final TopicPolicies topicPolicies = topicPoliciesOptional.get(); + persistencePolicies = topicPolicies.getPersistence(); + retentionPolicies = topicPolicies.getRetentionPolicies(); + topicLevelOffloadPolicies = topicPolicies.getOffloadPolicies(); + } - if (retentionPolicies == null) { - retentionPolicies = policies.map(p -> p.retention_policies).orElseGet( - () -> new RetentionPolicies(serviceConfig.getDefaultRetentionTimeInMinutes(), - serviceConfig.getDefaultRetentionSizeInMB()) - ); - } + if (persistencePolicies == null) { + persistencePolicies = policies.map(p -> p.persistence).orElseGet( + () -> new PersistencePolicies(serviceConfig.getManagedLedgerDefaultEnsembleSize(), + serviceConfig.getManagedLedgerDefaultWriteQuorum(), + serviceConfig.getManagedLedgerDefaultAckQuorum(), + serviceConfig.getManagedLedgerDefaultMarkDeleteRateLimit())); + } - ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); - managedLedgerConfig.setEnsembleSize(persistencePolicies.getBookkeeperEnsemble()); - managedLedgerConfig.setWriteQuorumSize(persistencePolicies.getBookkeeperWriteQuorum()); - managedLedgerConfig.setAckQuorumSize(persistencePolicies.getBookkeeperAckQuorum()); + if (retentionPolicies == null) { + retentionPolicies = policies.map(p -> p.retention_policies).orElseGet( + () -> new RetentionPolicies(serviceConfig.getDefaultRetentionTimeInMinutes(), + serviceConfig.getDefaultRetentionSizeInMB()) + ); + } - if (serviceConfig.isStrictBookieAffinityEnabled()) { + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + managedLedgerConfig.setEnsembleSize(persistencePolicies.getBookkeeperEnsemble()); + managedLedgerConfig.setWriteQuorumSize(persistencePolicies.getBookkeeperWriteQuorum()); + managedLedgerConfig.setAckQuorumSize(persistencePolicies.getBookkeeperAckQuorum()); + + if (serviceConfig.isStrictBookieAffinityEnabled()) { + managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyClassName( + IsolatedBookieEnsemblePlacementPolicy.class); + if (localPolicies.isPresent() && localPolicies.get().bookieAffinityGroup != null) { + Map properties = new HashMap<>(); + properties.put(IsolatedBookieEnsemblePlacementPolicy.ISOLATION_BOOKIE_GROUPS, + localPolicies.get().bookieAffinityGroup.getBookkeeperAffinityGroupPrimary()); + properties.put(IsolatedBookieEnsemblePlacementPolicy.SECONDARY_ISOLATION_BOOKIE_GROUPS, + localPolicies.get().bookieAffinityGroup.getBookkeeperAffinityGroupSecondary()); + managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyProperties(properties); + } else if (isSystemTopic(topicName)) { + Map properties = new HashMap<>(); + properties.put(IsolatedBookieEnsemblePlacementPolicy.ISOLATION_BOOKIE_GROUPS, "*"); + properties.put(IsolatedBookieEnsemblePlacementPolicy + .SECONDARY_ISOLATION_BOOKIE_GROUPS, "*"); + managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyProperties(properties); + } else { + Map properties = new HashMap<>(); + properties.put(IsolatedBookieEnsemblePlacementPolicy.ISOLATION_BOOKIE_GROUPS, ""); + properties.put(IsolatedBookieEnsemblePlacementPolicy.SECONDARY_ISOLATION_BOOKIE_GROUPS, ""); + managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyProperties(properties); + } + } else { + if (localPolicies.isPresent() && localPolicies.get().bookieAffinityGroup != null) { managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyClassName( IsolatedBookieEnsemblePlacementPolicy.class); - if (localPolicies.isPresent() && localPolicies.get().bookieAffinityGroup != null) { - Map properties = new HashMap<>(); - properties.put(IsolatedBookieEnsemblePlacementPolicy.ISOLATION_BOOKIE_GROUPS, - localPolicies.get().bookieAffinityGroup.getBookkeeperAffinityGroupPrimary()); - properties.put(IsolatedBookieEnsemblePlacementPolicy.SECONDARY_ISOLATION_BOOKIE_GROUPS, - localPolicies.get().bookieAffinityGroup.getBookkeeperAffinityGroupSecondary()); - managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyProperties(properties); - } else if (isSystemTopic(topicName)) { - Map properties = new HashMap<>(); - properties.put(IsolatedBookieEnsemblePlacementPolicy.ISOLATION_BOOKIE_GROUPS, "*"); - properties.put(IsolatedBookieEnsemblePlacementPolicy - .SECONDARY_ISOLATION_BOOKIE_GROUPS, "*"); - managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyProperties(properties); - } else { - Map properties = new HashMap<>(); - properties.put(IsolatedBookieEnsemblePlacementPolicy.ISOLATION_BOOKIE_GROUPS, ""); - properties.put(IsolatedBookieEnsemblePlacementPolicy.SECONDARY_ISOLATION_BOOKIE_GROUPS, ""); - managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyProperties(properties); - } - } else { - if (localPolicies.isPresent() && localPolicies.get().bookieAffinityGroup != null) { - managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyClassName( - IsolatedBookieEnsemblePlacementPolicy.class); - Map properties = new HashMap<>(); - properties.put(IsolatedBookieEnsemblePlacementPolicy.ISOLATION_BOOKIE_GROUPS, - localPolicies.get().bookieAffinityGroup.getBookkeeperAffinityGroupPrimary()); - properties.put(IsolatedBookieEnsemblePlacementPolicy.SECONDARY_ISOLATION_BOOKIE_GROUPS, - localPolicies.get().bookieAffinityGroup.getBookkeeperAffinityGroupSecondary()); - managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyProperties(properties); - } + Map properties = new HashMap<>(); + properties.put(IsolatedBookieEnsemblePlacementPolicy.ISOLATION_BOOKIE_GROUPS, + localPolicies.get().bookieAffinityGroup.getBookkeeperAffinityGroupPrimary()); + properties.put(IsolatedBookieEnsemblePlacementPolicy.SECONDARY_ISOLATION_BOOKIE_GROUPS, + localPolicies.get().bookieAffinityGroup.getBookkeeperAffinityGroupSecondary()); + managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyProperties(properties); } + } - managedLedgerConfig.setThrottleMarkDelete(persistencePolicies.getManagedLedgerMaxMarkDeleteRate()); - managedLedgerConfig.setDigestType(serviceConfig.getManagedLedgerDigestType()); - managedLedgerConfig.setPassword(serviceConfig.getManagedLedgerPassword()); - - managedLedgerConfig - .setMaxUnackedRangesToPersist(serviceConfig.getManagedLedgerMaxUnackedRangesToPersist()); - managedLedgerConfig.setPersistentUnackedRangesWithMultipleEntriesEnabled( - serviceConfig.isPersistentUnackedRangesWithMultipleEntriesEnabled()); - managedLedgerConfig.setMaxUnackedRangesToPersistInMetadataStore( - serviceConfig.getManagedLedgerMaxUnackedRangesToPersistInMetadataStore()); - managedLedgerConfig.setMaxEntriesPerLedger(serviceConfig.getManagedLedgerMaxEntriesPerLedger()); - managedLedgerConfig - .setMinimumRolloverTime(serviceConfig.getManagedLedgerMinLedgerRolloverTimeMinutes(), - TimeUnit.MINUTES); - managedLedgerConfig - .setMaximumRolloverTime(serviceConfig.getManagedLedgerMaxLedgerRolloverTimeMinutes(), - TimeUnit.MINUTES); - managedLedgerConfig.setMaxSizePerLedgerMb(serviceConfig.getManagedLedgerMaxSizePerLedgerMbytes()); - - managedLedgerConfig.setMetadataOperationsTimeoutSeconds( - serviceConfig.getManagedLedgerMetadataOperationsTimeoutSeconds()); - managedLedgerConfig - .setReadEntryTimeoutSeconds(serviceConfig.getManagedLedgerReadEntryTimeoutSeconds()); - managedLedgerConfig - .setAddEntryTimeoutSeconds(serviceConfig.getManagedLedgerAddEntryTimeoutSeconds()); - managedLedgerConfig.setMetadataEnsembleSize(serviceConfig.getManagedLedgerDefaultEnsembleSize()); - managedLedgerConfig.setUnackedRangesOpenCacheSetEnabled( - serviceConfig.isManagedLedgerUnackedRangesOpenCacheSetEnabled()); - managedLedgerConfig.setMetadataWriteQuorumSize(serviceConfig.getManagedLedgerDefaultWriteQuorum()); - managedLedgerConfig.setMetadataAckQuorumSize(serviceConfig.getManagedLedgerDefaultAckQuorum()); - managedLedgerConfig - .setMetadataMaxEntriesPerLedger(serviceConfig.getManagedLedgerCursorMaxEntriesPerLedger()); + managedLedgerConfig.setThrottleMarkDelete(persistencePolicies.getManagedLedgerMaxMarkDeleteRate()); + managedLedgerConfig.setDigestType(serviceConfig.getManagedLedgerDigestType()); + managedLedgerConfig.setPassword(serviceConfig.getManagedLedgerPassword()); + + managedLedgerConfig + .setMaxUnackedRangesToPersist(serviceConfig.getManagedLedgerMaxUnackedRangesToPersist()); + managedLedgerConfig.setPersistentUnackedRangesWithMultipleEntriesEnabled( + serviceConfig.isPersistentUnackedRangesWithMultipleEntriesEnabled()); + managedLedgerConfig.setMaxUnackedRangesToPersistInMetadataStore( + serviceConfig.getManagedLedgerMaxUnackedRangesToPersistInMetadataStore()); + managedLedgerConfig.setMaxEntriesPerLedger(serviceConfig.getManagedLedgerMaxEntriesPerLedger()); + managedLedgerConfig + .setMinimumRolloverTime(serviceConfig.getManagedLedgerMinLedgerRolloverTimeMinutes(), + TimeUnit.MINUTES); + managedLedgerConfig + .setMaximumRolloverTime(serviceConfig.getManagedLedgerMaxLedgerRolloverTimeMinutes(), + TimeUnit.MINUTES); + managedLedgerConfig.setMaxSizePerLedgerMb(serviceConfig.getManagedLedgerMaxSizePerLedgerMbytes()); + + managedLedgerConfig.setMetadataOperationsTimeoutSeconds( + serviceConfig.getManagedLedgerMetadataOperationsTimeoutSeconds()); + managedLedgerConfig + .setReadEntryTimeoutSeconds(serviceConfig.getManagedLedgerReadEntryTimeoutSeconds()); + managedLedgerConfig + .setAddEntryTimeoutSeconds(serviceConfig.getManagedLedgerAddEntryTimeoutSeconds()); + managedLedgerConfig.setMetadataEnsembleSize(serviceConfig.getManagedLedgerDefaultEnsembleSize()); + managedLedgerConfig.setUnackedRangesOpenCacheSetEnabled( + serviceConfig.isManagedLedgerUnackedRangesOpenCacheSetEnabled()); + managedLedgerConfig.setMetadataWriteQuorumSize(serviceConfig.getManagedLedgerDefaultWriteQuorum()); + managedLedgerConfig.setMetadataAckQuorumSize(serviceConfig.getManagedLedgerDefaultAckQuorum()); + managedLedgerConfig + .setMetadataMaxEntriesPerLedger(serviceConfig.getManagedLedgerCursorMaxEntriesPerLedger()); managedLedgerConfig .setLedgerRolloverTimeout(serviceConfig.getManagedLedgerCursorRolloverTimeInSeconds()); @@ -1893,36 +1899,37 @@ public CompletableFuture getManagedLedgerConfig(TopicName t managedLedgerConfig.setMaxBacklogBetweenCursorsForCaching( serviceConfig.getManagedLedgerMaxBacklogBetweenCursorsForCaching()); - OffloadPoliciesImpl nsLevelOffloadPolicies = - (OffloadPoliciesImpl) policies.map(p -> p.offload_policies).orElse(null); - OffloadPoliciesImpl offloadPolicies = OffloadPoliciesImpl.mergeConfiguration( - topicLevelOffloadPolicies, - OffloadPoliciesImpl.oldPoliciesCompatible(nsLevelOffloadPolicies, policies.orElse(null)), - getPulsar().getConfig().getProperties()); - if (NamespaceService.isSystemServiceNamespace(namespace.toString())) { - managedLedgerConfig.setLedgerOffloader(NullLedgerOffloader.INSTANCE); - } else { - if (topicLevelOffloadPolicies != null) { - try { - LedgerOffloader topicLevelLedgerOffLoader = - pulsar().createManagedLedgerOffloader(offloadPolicies); - managedLedgerConfig.setLedgerOffloader(topicLevelLedgerOffLoader); - } catch (PulsarServerException e) { - throw new RuntimeException(e); - } - } else { - //If the topic level policy is null, use the namespace level - managedLedgerConfig - .setLedgerOffloader(pulsar.getManagedLedgerOffloader(namespace, offloadPolicies)); + OffloadPoliciesImpl nsLevelOffloadPolicies = + (OffloadPoliciesImpl) policies.map(p -> p.offload_policies).orElse(null); + OffloadPoliciesImpl offloadPolicies = OffloadPoliciesImpl.mergeConfiguration( + topicLevelOffloadPolicies, + OffloadPoliciesImpl.oldPoliciesCompatible(nsLevelOffloadPolicies, policies.orElse(null)), + getPulsar().getConfig().getProperties()); + if (NamespaceService.isSystemServiceNamespace(namespace.toString())) { + managedLedgerConfig.setLedgerOffloader(NullLedgerOffloader.INSTANCE); + } else { + if (topicLevelOffloadPolicies != null) { + try { + LedgerOffloader topicLevelLedgerOffLoader = + pulsar().createManagedLedgerOffloader(offloadPolicies); + managedLedgerConfig.setLedgerOffloader(topicLevelLedgerOffLoader); + } catch (PulsarServerException e) { + throw new RuntimeException(e); } + } else { + //If the topic level policy is null, use the namespace level + managedLedgerConfig + .setLedgerOffloader(pulsar.getManagedLedgerOffloader(namespace, offloadPolicies)); } + } - managedLedgerConfig.setDeletionAtBatchIndexLevelEnabled( - serviceConfig.isAcknowledgmentAtBatchIndexLevelEnabled()); - managedLedgerConfig.setNewEntriesCheckDelayInMillis( - serviceConfig.getManagedLedgerNewEntriesCheckDelayInMillis()); - return managedLedgerConfig; - }); + managedLedgerConfig.setDeletionAtBatchIndexLevelEnabled( + serviceConfig.isAcknowledgmentAtBatchIndexLevelEnabled()); + managedLedgerConfig.setNewEntriesCheckDelayInMillis( + serviceConfig.getManagedLedgerNewEntriesCheckDelayInMillis()); + return managedLedgerConfig; + }); + }); } private void addTopicToStatsMaps(TopicName topicName, Topic topic) { 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 09f8de818db0a..ed76d37ae2536 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 @@ -18,12 +18,14 @@ */ package org.apache.pulsar.broker.service; +import static java.util.Objects.requireNonNull; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Sets; import java.util.HashSet; 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.CopyOnWriteArrayList; @@ -54,6 +56,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TopicPolicies; import org.apache.pulsar.common.util.FutureUtil; +import org.jetbrains.annotations.NotNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -78,8 +81,8 @@ public class SystemTopicBasedTopicPoliciesService implements TopicPoliciesServic private final Map>> readerCaches = new ConcurrentHashMap<>(); - @VisibleForTesting - final Map policyCacheInitMap = new ConcurrentHashMap<>(); + + final Map> policyCacheInitMap = new ConcurrentHashMap<>(); @VisibleForTesting final Map>> listeners = new ConcurrentHashMap<>(); @@ -219,12 +222,12 @@ public TopicPolicies getTopicPolicies(TopicName topicName, boolean isGlobal) throws TopicPoliciesCacheNotInitException { if (!policyCacheInitMap.containsKey(topicName.getNamespaceObject())) { NamespaceName namespace = topicName.getNamespaceObject(); - prepareInitPoliciesCache(namespace, new CompletableFuture<>()); + prepareInitPoliciesCacheAsync(namespace); } MutablePair result = new MutablePair<>(); policyCacheInitMap.compute(topicName.getNamespaceObject(), (k, initialized) -> { - if (initialized == null || !initialized) { + if (initialized == null || !initialized.isDone()) { result.setLeft(new TopicPoliciesCacheNotInitException()); } else { TopicPolicies topicPolicies = @@ -242,6 +245,34 @@ public TopicPolicies getTopicPolicies(TopicName topicName, } } + @NotNull + @Override + public CompletableFuture> getTopicPoliciesAsync(@NotNull TopicName topicName, + boolean isGlobal) { + requireNonNull(topicName); + final CompletableFuture preparedFuture = prepareInitPoliciesCacheAsync(topicName.getNamespaceObject()); + return preparedFuture.thenApply(__ -> { + final TopicPolicies candidatePolicies = isGlobal + ? globalPoliciesCache.get(TopicName.get(topicName.getPartitionedTopicName())) + : policiesCache.get(TopicName.get(topicName.getPartitionedTopicName())); + return Optional.ofNullable(candidatePolicies); + }); + } + + @NotNull + @Override + public CompletableFuture> getTopicPoliciesAsync(@NotNull TopicName topicName) { + requireNonNull(topicName); + final CompletableFuture preparedFuture = prepareInitPoliciesCacheAsync(topicName.getNamespaceObject()); + return preparedFuture.thenApply(__ -> { + final TopicPolicies localPolicies = policiesCache.get(TopicName.get(topicName.getPartitionedTopicName())); + if (localPolicies != null) { + return Optional.of(localPolicies); + } + return Optional.ofNullable(globalPoliciesCache.get(TopicName.get(topicName.getPartitionedTopicName()))); + }); + } + @Override public TopicPolicies getTopicPoliciesIfExists(TopicName topicName) { return policiesCache.get(TopicName.get(topicName.getPartitionedTopicName())); @@ -265,39 +296,48 @@ public CompletableFuture getTopicPoliciesBypassCacheAsync(TopicNa @Override public CompletableFuture addOwnedNamespaceBundleAsync(NamespaceBundle namespaceBundle) { - CompletableFuture result = new CompletableFuture<>(); NamespaceName namespace = namespaceBundle.getNamespaceObject(); if (NamespaceService.isHeartbeatNamespace(namespace)) { - result.complete(null); - return result; + return CompletableFuture.completedFuture(null); } synchronized (this) { if (readerCaches.get(namespace) != null) { ownedBundlesCountPerNamespace.get(namespace).incrementAndGet(); - result.complete(null); + return CompletableFuture.completedFuture(null); } else { - prepareInitPoliciesCache(namespace, result); + return prepareInitPoliciesCacheAsync(namespace); } } - return result; } - private void prepareInitPoliciesCache(@Nonnull NamespaceName namespace, CompletableFuture result) { - if (policyCacheInitMap.putIfAbsent(namespace, false) == null) { - CompletableFuture> readerCompletableFuture = + private @Nonnull CompletableFuture prepareInitPoliciesCacheAsync(@Nonnull NamespaceName namespace) { + requireNonNull(namespace); + return policyCacheInitMap.computeIfAbsent(namespace, (k) -> { + final CompletableFuture> readerCompletableFuture = createSystemTopicClientWithRetry(namespace); readerCaches.put(namespace, readerCompletableFuture); ownedBundlesCountPerNamespace.putIfAbsent(namespace, new AtomicInteger(1)); - readerCompletableFuture.thenAccept(reader -> { - initPolicesCache(reader, result); - result.thenRun(() -> readMorePolicies(reader)); - }).exceptionally(ex -> { - log.error("[{}] Failed to create reader on __change_events topic", namespace, ex); - cleanCacheAndCloseReader(namespace, false); - result.completeExceptionally(ex); + final CompletableFuture initFuture = readerCompletableFuture + .thenCompose(reader -> { + final CompletableFuture stageFuture = new CompletableFuture<>(); + initPolicesCache(reader, stageFuture); + return stageFuture + // Read policies in background + .thenAccept(__ -> readMorePoliciesAsync(reader)); + }); + initFuture.exceptionally(ex -> { + try { + log.error("[{}] Failed to create reader on __change_events topic", namespace, ex); + cleanCacheAndCloseReader(namespace, false); + } catch (Throwable cleanupEx) { + // Adding this catch to avoid break callback chain + log.error("[{}] Failed to cleanup reader on __change_events topic", namespace, cleanupEx); + } return null; }); - } + // let caller know we've got an exception. + return initFuture; + }); } protected CompletableFuture> createSystemTopicClientWithRetry( @@ -381,8 +421,7 @@ private void initPolicesCache(SystemTopicClient.Reader reader, Comp if (log.isDebugEnabled()) { log.debug("[{}] Reach the end of the system topic.", reader.getSystemTopic().getTopicName()); } - policyCacheInitMap.computeIfPresent( - reader.getSystemTopic().getTopicName().getNamespaceObject(), (k, v) -> true); + // replay policy message policiesCache.forEach(((topicName, topicPolicies) -> { if (listeners.get(topicName) != null) { @@ -395,6 +434,7 @@ private void initPolicesCache(SystemTopicClient.Reader reader, Comp } } })); + future.complete(null); } }); @@ -420,7 +460,13 @@ private void cleanCacheAndCloseReader(@Nonnull NamespaceName namespace, boolean }); } - private void readMorePolicies(SystemTopicClient.Reader reader) { + /** + * This is an async method for the background reader to continue syncing new messages. + * + * Note: You should not do any blocking call here. because it will affect + * #{@link SystemTopicBasedTopicPoliciesService#getTopicPoliciesAsync(TopicName)} method to block loading topic. + */ + private void readMorePoliciesAsync(SystemTopicClient.Reader reader) { reader.readNextAsync() .thenAccept(msg -> { refreshTopicPoliciesCache(msg); @@ -428,7 +474,7 @@ private void readMorePolicies(SystemTopicClient.Reader reader) { }) .whenComplete((__, ex) -> { if (ex == null) { - readMorePolicies(reader); + readMorePoliciesAsync(reader); } else { Throwable cause = FutureUtil.unwrapCompletionException(ex); if (cause instanceof PulsarClientException.AlreadyClosedException) { @@ -437,7 +483,7 @@ private void readMorePolicies(SystemTopicClient.Reader reader) { reader.getSystemTopic().getTopicName().getNamespaceObject(), false); } else { log.warn("Read more topic polices exception, read again.", ex); - readMorePolicies(reader); + readMorePoliciesAsync(reader); } } }); @@ -605,7 +651,7 @@ boolean checkReaderIsCached(NamespaceName namespaceName) { } @VisibleForTesting - public Boolean getPoliciesCacheInit(NamespaceName namespaceName) { + public CompletableFuture getPoliciesCacheInit(NamespaceName namespaceName) { return policyCacheInitMap.get(namespaceName); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java index c4bcc0c39353c..aa3a6aaeff29f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicPoliciesService.java @@ -22,6 +22,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import javax.annotation.Nonnull; import org.apache.pulsar.broker.service.BrokerServiceException.TopicPoliciesCacheNotInitException; import org.apache.pulsar.client.impl.Backoff; import org.apache.pulsar.client.impl.BackoffBuilder; @@ -31,6 +32,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TopicPolicies; import org.apache.pulsar.common.util.FutureUtil; +import org.jetbrains.annotations.NotNull; /** * Topic policies service. @@ -109,6 +111,32 @@ default CompletableFuture> getTopicPoliciesAsyncWithRetr return response; } + /** + * Asynchronously retrieves topic policies. + * This triggers the Pulsar broker's internal client to load policies from the + * system topic `persistent://tenant/namespace/__change_event`. + * + * @param topicName The name of the topic. + * @param isGlobal Indicates if the policies are global. + * @return A CompletableFuture containing an Optional of TopicPolicies. + * @throws NullPointerException If the topicName is null. + */ + @Nonnull + CompletableFuture> getTopicPoliciesAsync(@Nonnull TopicName topicName, boolean isGlobal); + + /** + * Asynchronously retrieves topic policies. + * This triggers the Pulsar broker's internal client to load policies from the + * system topic `persistent://tenant/namespace/__change_event`. + * + * NOTE: If local policies are not available, it will fallback to using topic global policies. + * @param topicName The name of the topic. + * @return A CompletableFuture containing an Optional of TopicPolicies. + * @throws NullPointerException If the topicName is null. + */ + @Nonnull + CompletableFuture> getTopicPoliciesAsync(@Nonnull TopicName topicName); + /** * Get policies for a topic without cache async. * @param topicName topic name @@ -162,6 +190,19 @@ public TopicPolicies getTopicPolicies(TopicName topicName, boolean isGlobal) return null; } + @NotNull + @Override + public CompletableFuture> getTopicPoliciesAsync(@NotNull TopicName topicName, + boolean isGlobal) { + return CompletableFuture.completedFuture(Optional.empty()); + } + + @NotNull + @Override + public CompletableFuture> getTopicPoliciesAsync(@NotNull TopicName topicName) { + return CompletableFuture.completedFuture(Optional.empty()); + } + @Override public TopicPolicies getTopicPoliciesIfExists(TopicName topicName) { return null; 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 a4f6bd4650f7c..25ad6cab94272 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 @@ -127,6 +127,7 @@ public void initPersistentTopics() throws Exception { @Override @BeforeMethod protected void setup() throws Exception { + conf.setTopicLevelPoliciesEnabled(false); super.internalSetup(); persistentTopics = spy(PersistentTopics.class); persistentTopics.setServletContext(new MockServletContext()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java index 87471f4972f8d..faf141a5d1cf4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java @@ -180,7 +180,7 @@ public void testTopicPolicyInitialValueWithNamespaceAlreadyLoaded() throws Excep assertFalse(pulsar.getBrokerService().getTopics().containsKey(topic)); //make sure namespace policy reader is fully started. Awaitility.await().untilAsserted(()-> { - assertTrue(policyService.getPoliciesCacheInit(topicName.getNamespaceObject())); + assertTrue(policyService.getPoliciesCacheInit(topicName.getNamespaceObject()).isDone()); }); //load the topic. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesWithBrokerRestartTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesWithBrokerRestartTest.java new file mode 100644 index 0000000000000..672fc2c95f890 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesWithBrokerRestartTest.java @@ -0,0 +1,104 @@ +/* + * 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 lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.awaitility.Awaitility; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; +import java.nio.charset.StandardCharsets; +import java.util.concurrent.TimeUnit; + +@Slf4j +@Test(groups = "broker-admin") +public class TopicPoliciesWithBrokerRestartTest extends MockedPulsarServiceBaseTest { + + @Override + @BeforeClass(alwaysRun = true) + protected void setup() throws Exception { + super.internalSetup(); + setupDefaultTenantAndNamespace(); + } + + @Override + @AfterClass(alwaysRun = true) + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + + @Test + public void testRetentionWithBrokerRestart() throws Exception { + final int messages = 1_000; + final int topicNum = 500; + // (1) Init topic + admin.namespaces().createNamespace("public/retention"); + final String topicName = "persistent://public/retention/retention_with_broker_restart"; + admin.topics().createNonPartitionedTopic(topicName); + for (int i = 0; i < topicNum; i++) { + final String shadowTopicNames = topicName + "_" + i; + admin.topics().createNonPartitionedTopic(shadowTopicNames); + } + // (2) Set retention + final RetentionPolicies retentionPolicies = new RetentionPolicies(20, 20); + for (int i = 0; i < topicNum; i++) { + final String shadowTopicNames = topicName + "_" + i; + admin.topicPolicies().setRetention(shadowTopicNames, retentionPolicies); + } + admin.topicPolicies().setRetention(topicName, retentionPolicies); + // (3) Send messages + @Cleanup + final Producer publisher = pulsarClient.newProducer() + .topic(topicName) + .create(); + for (int i = 0; i < messages; i++) { + publisher.send((i + "").getBytes(StandardCharsets.UTF_8)); + } + // (4) Check configuration + Awaitility.await().untilAsserted(() -> { + final PersistentTopic persistentTopic1 = (PersistentTopic) + pulsar.getBrokerService().getTopic(topicName, true).join().get(); + final ManagedLedgerImpl managedLedger1 = (ManagedLedgerImpl) persistentTopic1.getManagedLedger(); + Assert.assertEquals(managedLedger1.getConfig().getRetentionSizeInMB(), 20); + Assert.assertEquals(managedLedger1.getConfig().getRetentionTimeMillis(), + TimeUnit.MINUTES.toMillis(20)); + }); + // (5) Restart broker + restartBroker(); + // (6) Check configuration again + for (int i = 0; i < topicNum; i++) { + final String shadowTopicNames = topicName + "_" + i; + admin.lookups().lookupTopic(shadowTopicNames); + final PersistentTopic persistentTopicTmp = (PersistentTopic) + pulsar.getBrokerService().getTopic(shadowTopicNames, true).join().get(); + final ManagedLedgerImpl managedLedgerTemp = (ManagedLedgerImpl) persistentTopicTmp.getManagedLedger(); + Assert.assertEquals(managedLedgerTemp.getConfig().getRetentionSizeInMB(), 20); + Assert.assertEquals(managedLedgerTemp.getConfig().getRetentionTimeMillis(), + TimeUnit.MINUTES.toMillis(20)); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicsAuthTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicsAuthTest.java index efd8b66d754ac..234af7afa8d09 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicsAuthTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicsAuthTest.java @@ -84,6 +84,8 @@ protected void setup() throws Exception { Set providers = new HashSet<>(); providers.add(AuthenticationProviderToken.class.getName()); conf.setAuthenticationProviders(providers); + conf.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName()); + conf.setBrokerClientAuthenticationParameters("token:" + ADMIN_TOKEN); super.internalSetup(); PulsarAdminBuilder pulsarAdminBuilder = PulsarAdmin.builder().serviceHttpUrl(brokerUrl != null ? brokerUrl.toString() : brokerUrlTls.toString()) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthLogsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthLogsTest.java index 6ffcecbeb9f8b..942a42fa7aaa1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthLogsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/AuthLogsTest.java @@ -60,6 +60,8 @@ public void setup() throws Exception { conf.setAuthorizationEnabled(true); conf.setAuthorizationAllowWildcardsMatching(true); conf.setSuperUserRoles(Sets.newHashSet("super")); + conf.setBrokerClientAuthenticationPlugin(MockAuthentication.class.getName()); + conf.setBrokerClientAuthenticationParameters("user:pass.pass"); internalSetup(); try (PulsarAdmin admin = PulsarAdmin.builder() diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockAuthentication.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockAuthentication.java index 0b1726617f71f..25ac59796b02c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockAuthentication.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockAuthentication.java @@ -29,7 +29,10 @@ public class MockAuthentication implements Authentication { private static final Logger log = LoggerFactory.getLogger(MockAuthentication.class); - private final String user; + private String user; + + public MockAuthentication() { + } public MockAuthentication(String user) { this.user = user; @@ -67,6 +70,7 @@ public String getCommandData() { @Override public void configure(Map authParams) { + this.user = authParams.get("user"); } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java index c32d3fc3b0b27..53f6af82bee8d 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 @@ -39,6 +39,7 @@ import javax.ws.rs.container.TimeoutHandler; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.authentication.AuthenticationProviderTls; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.broker.testcontext.PulsarTestContext; @@ -48,6 +49,8 @@ import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.impl.auth.AuthenticationDisabled; +import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -231,6 +234,22 @@ protected void doInitConf() throws Exception { protected final void init() throws Exception { doInitConf(); + // trying to config the broker internal client + if (conf.getWebServicePortTls().isPresent() + && conf.getAuthenticationProviders().contains(AuthenticationProviderTls.class.getName()) + && !conf.isTlsEnabledWithKeyStore()) { + // enabled TLS + if (conf.getBrokerClientAuthenticationPlugin() == null + || conf.getBrokerClientAuthenticationPlugin().equals(AuthenticationDisabled.class.getName())) { + conf.setBrokerClientAuthenticationPlugin(AuthenticationTls.class.getName()); + conf.setBrokerClientAuthenticationParameters("tlsCertFile:" + BROKER_CERT_FILE_PATH + + ",tlsKeyFile:" + BROKER_KEY_FILE_PATH); + conf.setBrokerClientTlsEnabled(true); + conf.setBrokerClientTrustCertsFilePath(CA_CERT_FILE_PATH); + conf.setBrokerClientCertificateFilePath(BROKER_CERT_FILE_PATH); + conf.setBrokerClientKeyFilePath(BROKER_KEY_FILE_PATH); + } + } startBroker(); } 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 951892f4ebfbc..5252407892eea 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 @@ -304,6 +304,7 @@ public void testSetRackInfoAndAffinityGroupDuringProduce() throws Exception { bookies[3].getBookieId()); ServiceConfiguration config = new ServiceConfiguration(); + config.setTopicLevelPoliciesEnabled(false); config.setLoadManagerClassName(ModularLoadManagerImpl.class.getName()); config.setClusterName(cluster); config.setWebServicePort(Optional.of(0)); @@ -612,9 +613,9 @@ public void testBookieIsolationWithSecondaryGroup() throws Exception { config.setBrokerShutdownTimeoutMs(0L); config.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); config.setBrokerServicePort(Optional.of(0)); + config.setTopicLevelPoliciesEnabled(false); config.setAdvertisedAddress("localhost"); config.setBookkeeperClientIsolationGroups(brokerBookkeeperClientIsolationGroups); - config.setManagedLedgerDefaultEnsembleSize(2); config.setManagedLedgerDefaultWriteQuorum(2); config.setManagedLedgerDefaultAckQuorum(2); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java index 31b5bcb23cd98..5b70ff996756e 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java @@ -141,7 +141,7 @@ public void testGetPolicy() throws ExecutionException, InterruptedException, Top // Wait for all topic policies updated. Awaitility.await().untilAsserted(() -> Assert.assertTrue(systemTopicBasedTopicPoliciesService - .getPoliciesCacheInit(TOPIC1.getNamespaceObject()))); + .getPoliciesCacheInit(TOPIC1.getNamespaceObject()).isDone())); // Assert broker is cache all topic policies Awaitility.await().untilAsserted(() -> @@ -304,8 +304,8 @@ private void prepareData() throws PulsarAdminException { @Test public void testGetPolicyTimeout() throws Exception { SystemTopicBasedTopicPoliciesService service = (SystemTopicBasedTopicPoliciesService) pulsar.getTopicPoliciesService(); - Awaitility.await().untilAsserted(() -> assertTrue(service.policyCacheInitMap.get(TOPIC1.getNamespaceObject()))); - service.policyCacheInitMap.put(TOPIC1.getNamespaceObject(), false); + Awaitility.await().untilAsserted(() -> assertTrue(service.policyCacheInitMap.get(TOPIC1.getNamespaceObject()).isDone())); + service.policyCacheInitMap.put(TOPIC1.getNamespaceObject(), new CompletableFuture<>()); long start = System.currentTimeMillis(); Backoff backoff = new BackoffBuilder() .setInitialTime(500, TimeUnit.MILLISECONDS) 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 e29d015c45db7..9995b6a28a903 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 @@ -45,6 +45,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -620,7 +621,7 @@ public void testCheckPersistencePolicies() throws Exception { doReturn(policiesService).when(pulsar).getTopicPoliciesService(); TopicPolicies policies = new TopicPolicies(); policies.setRetentionPolicies(retentionPolicies); - doReturn(policies).when(policiesService).getTopicPoliciesIfExists(TopicName.get(topic)); + doReturn(CompletableFuture.completedFuture(Optional.of(policies))).when(policiesService).getTopicPoliciesAsync(TopicName.get(topic)); persistentTopic.onUpdate(policies); verify(persistentTopic, times(1)).checkPersistencePolicies(); Awaitility.await().untilAsserted(() -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticationTlsHostnameVerificationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticationTlsHostnameVerificationTest.java index e3bd321d76332..042c9b328d58b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticationTlsHostnameVerificationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticationTlsHostnameVerificationTest.java @@ -30,6 +30,7 @@ import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.common.tls.PublicSuffixMatcher; import org.apache.pulsar.common.tls.TlsHostnameVerifier; +import org.assertj.core.util.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; @@ -141,6 +142,7 @@ public void testTlsSyncProducerAndConsumerWithInvalidBrokerHost(boolean hostname // setup broker cert which has CN = "pulsar" different than broker's hostname="localhost" conf.setBrokerServicePortTls(Optional.of(0)); conf.setWebServicePortTls(Optional.of(0)); + conf.setAuthenticationProviders(Sets.newTreeSet(AuthenticationProviderTls.class.getName())); conf.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); conf.setTlsCertificateFilePath(TLS_MIM_SERVER_CERT_FILE_PATH); conf.setTlsKeyFilePath(TLS_MIM_SERVER_KEY_FILE_PATH); @@ -182,6 +184,7 @@ public void testTlsSyncProducerAndConsumerCorrectBrokerHost() throws Exception { // setup broker cert which has CN = "localhost" conf.setBrokerServicePortTls(Optional.of(0)); conf.setWebServicePortTls(Optional.of(0)); + conf.setAuthenticationProviders(Sets.newTreeSet(AuthenticationProviderTls.class.getName())); conf.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); conf.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); conf.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java index bdbe8efc8e6ef..9a36e0683b422 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthorizationProducerConsumerTest.java @@ -119,6 +119,7 @@ protected void cleanup() throws Exception { public void testProducerAndConsumerAuthorization() throws Exception { log.info("-- Starting {} test --", methodName); cleanup(); + conf.setTopicLevelPoliciesEnabled(false); conf.setAuthorizationProvider(TestAuthorizationProvider.class.getName()); setup(); @@ -179,6 +180,7 @@ public void testProducerAndConsumerAuthorization() throws Exception { public void testSubscriberPermission() throws Exception { log.info("-- Starting {} test --", methodName); cleanup(); + conf.setTopicLevelPoliciesEnabled(false); conf.setEnablePackagesManagement(true); conf.setPackagesManagementStorageProvider(MockedPackagesStorageProvider.class.getName()); conf.setAuthorizationProvider(PulsarAuthorizationProvider.class.getName()); @@ -369,6 +371,7 @@ public void testSubscriberPermission() throws Exception { public void testClearBacklogPermission() throws Exception { log.info("-- Starting {} test --", methodName); cleanup(); + conf.setTopicLevelPoliciesEnabled(false); conf.setAuthorizationProvider(PulsarAuthorizationProvider.class.getName()); setup(); @@ -610,6 +613,7 @@ public void testUpdateTopicPropertiesAuthorization() throws Exception { public void testSubscriptionPrefixAuthorization() throws Exception { log.info("-- Starting {} test --", methodName); cleanup(); + conf.setTopicLevelPoliciesEnabled(false); conf.setAuthorizationProvider(TestAuthorizationProviderWithSubscriptionPrefix.class.getName()); setup(); @@ -749,6 +753,7 @@ public void testAuthData() throws Exception { public void testPermissionForProducerCreateInitialSubscription() throws Exception { log.info("-- Starting {} test --", methodName); cleanup(); + conf.setTopicLevelPoliciesEnabled(false); conf.setAuthorizationProvider(PulsarAuthorizationProvider.class.getName()); setup(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MutualAuthenticationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MutualAuthenticationTest.java index 2fc8aebf64a4a..81d65b192049b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MutualAuthenticationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MutualAuthenticationTest.java @@ -195,7 +195,7 @@ protected void setup() throws Exception { Set superUserRoles = new HashSet<>(); superUserRoles.add("admin"); conf.setSuperUserRoles(superUserRoles); - + conf.setTopicLevelPoliciesEnabled(false); conf.setAuthorizationEnabled(true); conf.setAuthenticationEnabled(true); Set providersClassNames = Sets.newHashSet(MutualAuthenticationProvider.class.getName()); 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 87f12e6acdcb2..4d5e7deaf7d99 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 @@ -37,6 +37,7 @@ import java.util.concurrent.TimeUnit; import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.impl.auth.AuthenticationToken; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.slf4j.Logger; @@ -92,6 +93,8 @@ protected void setup() throws Exception { Set providers = new HashSet<>(); providers.add(AuthenticationProviderToken.class.getName()); conf.setAuthenticationProviders(providers); + conf.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName()); + conf.setBrokerClientAuthenticationParameters("token:" + ADMIN_TOKEN); conf.setClusterName("test"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenOauth2AuthenticatedProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenOauth2AuthenticatedProducerConsumerTest.java index fdf41c4a6ada1..ba43ee6d6a2dd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenOauth2AuthenticatedProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenOauth2AuthenticatedProducerConsumerTest.java @@ -27,7 +27,9 @@ import java.nio.file.Path; import java.nio.file.Paths; import java.time.Duration; +import java.util.HashMap; import java.util.HashSet; +import java.util.Map; import java.util.Properties; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -39,6 +41,7 @@ import org.apache.pulsar.client.impl.auth.oauth2.AuthenticationOAuth2; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.util.ObjectMapperFactory; import org.awaitility.Awaitility; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -87,11 +90,12 @@ protected void setup() throws Exception { conf.setAuthenticationProviders(providers); conf.setBrokerClientAuthenticationPlugin(AuthenticationOAuth2.class.getName()); - conf.setBrokerClientAuthenticationParameters("{\n" - + " \"privateKey\": \"" + CREDENTIALS_FILE + "\",\n" - + " \"issuerUrl\": \"" + server.getIssuer() + "\",\n" - + " \"audience\": \"" + audience + "\",\n" - + "}\n"); + final Map oauth2Param = new HashMap<>(); + oauth2Param.put("privateKey", CREDENTIALS_FILE); + oauth2Param.put("issuerUrl", server.getIssuer()); + oauth2Param.put("audience", audience); + conf.setBrokerClientAuthenticationParameters(ObjectMapperFactory + .getMapper().getObjectMapper().writeValueAsString(oauth2Param)); conf.setClusterName("test"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsProducerConsumerTestWithAuthTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsProducerConsumerTestWithAuthTest.java index 8e508b6cf2068..77405e142013a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsProducerConsumerTestWithAuthTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/KeyStoreTlsProducerConsumerTestWithAuthTest.java @@ -32,6 +32,7 @@ import io.jsonwebtoken.SignatureAlgorithm; import lombok.Cleanup; +import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.authentication.AuthenticationProviderTls; import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; @@ -49,6 +50,7 @@ import org.apache.pulsar.client.impl.auth.AuthenticationToken; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.util.ObjectMapperFactory; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -83,6 +85,7 @@ protected void cleanup() throws Exception { super.internalCleanup(); } + @SneakyThrows protected void internalSetUpForBroker() { conf.setBrokerServicePortTls(Optional.of(0)); conf.setWebServicePortTls(Optional.of(0)); @@ -114,6 +117,25 @@ protected void internalSetUpForBroker() { conf.setAuthenticationProviders(providers); conf.setNumExecutorThreadPoolSize(5); + Set tlsProtocols = Sets.newConcurrentHashSet(); + tlsProtocols.add("TLSv1.3"); + tlsProtocols.add("TLSv1.2"); + conf.setBrokerClientAuthenticationPlugin(AuthenticationKeyStoreTls.class.getName()); + Map authParams = new HashMap<>(); + authParams.put(AuthenticationKeyStoreTls.KEYSTORE_TYPE, KEYSTORE_TYPE); + authParams.put(AuthenticationKeyStoreTls.KEYSTORE_PATH, CLIENT_KEYSTORE_FILE_PATH); + authParams.put(AuthenticationKeyStoreTls.KEYSTORE_PW, CLIENT_KEYSTORE_PW); + conf.setBrokerClientAuthenticationParameters(ObjectMapperFactory.getMapper() + .getObjectMapper().writeValueAsString(authParams)); + conf.setBrokerClientTlsEnabled(true); + conf.setBrokerClientTlsEnabledWithKeyStore(true); + conf.setBrokerClientTlsTrustStore(BROKER_TRUSTSTORE_FILE_PATH); + conf.setBrokerClientTlsTrustStorePassword(BROKER_TRUSTSTORE_PW); + conf.setBrokerClientTlsKeyStore(CLIENT_KEYSTORE_FILE_PATH); + conf.setBrokerClientTlsKeyStoreType(KEYSTORE_TYPE); + conf.setBrokerClientTlsKeyStorePassword(CLIENT_KEYSTORE_PW); + conf.setBrokerClientTlsProtocols(tlsProtocols); + } protected void internalSetUpForClient(boolean addCertificates, String lookupUrl) throws Exception { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplAuthTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplAuthTest.java index 76936334eb0ba..b9139dabdf021 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplAuthTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplAuthTest.java @@ -85,6 +85,7 @@ public void setup() throws Exception { // set isTcpLookup = true, to use BinaryProtoLookupService to get topics for a pattern. isTcpLookup = true; + conf.setTopicLevelPoliciesEnabled(false); conf.setAuthenticationEnabled(true); conf.setAuthorizationEnabled(true); diff --git a/pulsar-broker/src/test/resources/configurations/standalone_no_client_auth.conf b/pulsar-broker/src/test/resources/configurations/standalone_no_client_auth.conf index d9411e655ad5b..4e2fd40298354 100644 --- a/pulsar-broker/src/test/resources/configurations/standalone_no_client_auth.conf +++ b/pulsar-broker/src/test/resources/configurations/standalone_no_client_auth.conf @@ -29,4 +29,5 @@ authenticationEnabled=true authenticationProviders=org.apache.pulsar.MockTokenAuthenticationProvider brokerClientAuthenticationPlugin= brokerClientAuthenticationParameters= -loadBalancerOverrideBrokerNicSpeedGbps=2 \ No newline at end of file +loadBalancerOverrideBrokerNicSpeedGbps=2 +topicLevelPoliciesEnabled=false \ No newline at end of file diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticationTest.java index 8229d929ee5e3..9c8e5197adf1a 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticationTest.java @@ -168,7 +168,7 @@ protected void setup() throws Exception { conf.setBrokerClientAuthenticationPlugin(BasicAuthentication.class.getName()); // Expires after an hour conf.setBrokerClientAuthenticationParameters( - "entityType:broker,expiryTime:" + (System.currentTimeMillis() + 3600 * 1000)); + "entityType:admin,expiryTime:" + (System.currentTimeMillis() + 3600 * 1000)); Set superUserRoles = new HashSet<>(); superUserRoles.add("admin"); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java index 99af3b1cf6abe..b7cfb87474707 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java @@ -53,7 +53,7 @@ protected void setup() throws Exception { conf.setAuthenticationEnabled(true); conf.setAuthorizationEnabled(true); conf.setBrokerClientAuthenticationPlugin(BasicAuthentication.class.getName()); - conf.setBrokerClientAuthenticationParameters("authParam:broker"); + conf.setBrokerClientAuthenticationParameters("authParam:admin"); conf.setAuthenticateOriginalAuthData(true); Set superUserRoles = new HashSet(); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRolesEnforcementTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRolesEnforcementTest.java index 2c8c382b6a5ef..3259cfd95c741 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRolesEnforcementTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRolesEnforcementTest.java @@ -144,7 +144,7 @@ protected void setup() throws Exception { conf.setAuthenticationEnabled(true); conf.setAuthorizationEnabled(true); conf.setBrokerClientAuthenticationPlugin(BasicAuthentication.class.getName()); - conf.setBrokerClientAuthenticationParameters("authParam:broker"); + conf.setBrokerClientAuthenticationParameters("authParam:admin"); Set superUserRoles = new HashSet<>(); superUserRoles.add("admin"); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java index e8bb128c8c190..2d97a4b06a856 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java @@ -78,6 +78,8 @@ public class ProxyWithAuthorizationNegTest extends ProducerConsumerBase { protected void setup() throws Exception { // enable tls and auth&auth at broker + conf.setTopicLevelPoliciesEnabled(false); + conf.setAuthenticationEnabled(true); conf.setAuthorizationEnabled(true); diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarAuth.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarAuth.java index 9119ffed4e28f..7b550b7270f37 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarAuth.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/TestPulsarAuth.java @@ -38,6 +38,7 @@ import org.apache.pulsar.client.admin.PulsarAdminBuilder; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.AuthenticationFactory; +import org.apache.pulsar.client.impl.auth.AuthenticationToken; import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -63,6 +64,9 @@ public void setup() throws Exception { conf.setProperties(properties); conf.setSuperUserRoles(Sets.newHashSet(SUPER_USER_ROLE)); conf.setClusterName("c1"); + conf.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName()); + conf.setBrokerClientAuthenticationParameters("token:" + AuthTokenUtils + .createToken(secretKey, SUPER_USER_ROLE, Optional.empty())); internalSetup(); admin.clusters().createCluster("c1", ClusterData.builder().build()); diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java index 057039edc3be2..49e5ae378342d 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java @@ -89,6 +89,7 @@ public void setup() throws Exception { "org.apache.pulsar.broker.loadbalance.extensions.scheduler.TransferShedder"); brokerEnvs.put("forceDeleteNamespaceAllowed", "true"); brokerEnvs.put("loadBalancerDebugModeEnabled", "true"); + brokerEnvs.put("topicLevelPoliciesEnabled", "false"); brokerEnvs.put("PULSAR_MEM", "-Xmx512M"); spec.brokerEnvs(brokerEnvs); pulsarCluster = PulsarCluster.forSpec(spec); diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestPulsarSQLAuth.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestPulsarSQLAuth.java index 0a9bb5e19592a..87db46f2bb625 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestPulsarSQLAuth.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/presto/TestPulsarSQLAuth.java @@ -68,6 +68,7 @@ protected void beforeStartCluster() { envMap.put("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(secretKey)); envMap.put("superUserRoles", "admin"); envMap.put("brokerDeleteInactiveTopicsEnabled", "false"); + envMap.put("topicLevelPoliciesEnabled", "false"); for (BrokerContainer brokerContainer : pulsarCluster.getBrokers()) { brokerContainer.withEnv(envMap); From ec8e5b4dc75f504b7efdd8c9a446ab0071333705 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 14 Oct 2023 10:45:10 +0300 Subject: [PATCH 113/699] [fix][ci] Fix docker image building by releasing more disk space before building (#21365) (cherry picked from commit 421c98a1a5ec08941e794698dfc43a1a08d6e782) --- .github/actions/clean-disk/action.yml | 2 +- .github/workflows/pulsar-ci.yaml | 12 ++++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/.github/actions/clean-disk/action.yml b/.github/actions/clean-disk/action.yml index 8bcc5f1396802..d74c3f25fc64c 100644 --- a/.github/actions/clean-disk/action.yml +++ b/.github/actions/clean-disk/action.yml @@ -31,7 +31,7 @@ runs: directories=(/usr/local/lib/android /opt/ghc) if [[ "${{ inputs.mode }}" == "full" ]]; then # remove these directories only when mode is 'full' - directories+=(/usr/share/dotnet) + directories+=(/usr/share/dotnet /opt/hostedtoolcache/CodeQL) fi emptydir=/tmp/empty$$/ mkdir $emptydir diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 64708d940a9fb..b8d498c27be93 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -747,6 +747,8 @@ jobs: - name: Clean Disk uses: ./.github/actions/clean-disk + with: + mode: full - name: Cache local Maven repository uses: actions/cache@v3 @@ -862,6 +864,7 @@ jobs: - name: Pulsar IO group: PULSAR_IO + clean_disk: true - name: Sql group: SQL @@ -873,6 +876,10 @@ jobs: - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm + - name: Clean Disk when needed + if: ${{ matrix.clean_disk }} + uses: ./.github/actions/clean-disk + - name: Setup ssh access to build runner VM # ssh access is enabled for builds in own forks if: ${{ github.repository != 'apache/pulsar' && github.event_name == 'pull_request' }} @@ -1073,6 +1080,7 @@ jobs: - name: Pulsar IO - Oracle group: PULSAR_IO_ORA + clean_disk: true steps: - name: checkout @@ -1081,6 +1089,10 @@ jobs: - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm + - name: Clean Disk when needed + if: ${{ matrix.clean_disk }} + uses: ./.github/actions/clean-disk + - name: Setup ssh access to build runner VM # ssh access is enabled for builds in own forks if: ${{ github.repository != 'apache/pulsar' && github.event_name == 'pull_request' }} From cbaad0a69e5866b3d3f2e04ce3fc83d5229b1ff7 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 14 Sep 2023 00:17:10 +0800 Subject: [PATCH 114/699] [fix] [broker] Make specified producer could override the previous one (#21155) The client assumed the connection was inactive, but the Broker assumed the connection was fine. The Client tried to use a new connection to reconnect a producer, then got an error `Producer with name 'st-0-5' is already connected to topic`. - In a connection, the second connection waits for the first connection to complete\. But there is a bug that causes this mechanism to fail\. - If a producer uses a default name, the second registration will override the first one. But it can not override the first one if it uses a specified producer name\. I think this mechanism is to prevent a client from creating two producers with the same name. However, method `Producer.isSuccessorTo` has checked the `producer-id`, and the `producer-id` of multiple producers created by the same client are different. So this mechanism can be deleted. - For `issue 1`: If a producer with the same name tries to use a new connection, async checks the old connection is available. The producers related to the connection that is not available are automatically cleaned up. - For `issue 2`: - Fix the bug that causes a complete producer future will be removed from `ServerCnx`. - Remove the mechanism that prevents a producer with a specified name from overriding the previous producer. (cherry picked from commit bda16b6f5b715942f7ed996052f6cbd8026fbbf0) --- .../pulsar/broker/service/AbstractTopic.java | 8 +- .../pulsar/broker/service/ServerCnx.java | 40 ++--- .../auth/MockedPulsarServiceBaseTest.java | 24 +++ .../pulsar/broker/service/ServerCnxTest.java | 162 +++++++++++++++++- .../impl/ProducerConsumerInternalTest.java | 57 ++++++ 5 files changed, 268 insertions(+), 23 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 cef2dd2080cf0..31e37d0f176d7 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 @@ -982,8 +982,7 @@ protected void internalAddProducer(Producer producer) throws BrokerServiceExcept private void tryOverwriteOldProducer(Producer oldProducer, Producer newProducer) throws BrokerServiceException { - if (newProducer.isSuccessorTo(oldProducer) && !isUserProvidedProducerName(oldProducer) - && !isUserProvidedProducerName(newProducer)) { + if (newProducer.isSuccessorTo(oldProducer)) { oldProducer.close(false); if (!producers.replace(newProducer.getProducerName(), oldProducer, newProducer)) { // Met concurrent update, throw exception here so that client can try reconnect later. @@ -993,6 +992,11 @@ private void tryOverwriteOldProducer(Producer oldProducer, Producer newProducer) handleProducerRemoved(oldProducer); } } else { + // If a producer with the same name tries to use a new connection, async check the old connection is + // available. The producers related the connection that not available are automatically cleaned up. + if (!Objects.equals(oldProducer.getCnx(), newProducer.getCnx())) { + oldProducer.getCnx().checkConnectionLiveness(); + } throw new BrokerServiceException.NamingException( "Producer with name '" + newProducer.getProducerName() + "' is already connected to 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 74c2cee3ec7be..b819a99309365 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 @@ -1402,36 +1402,36 @@ protected void handleProducer(final CommandProducer cmdProducer) { CompletableFuture existingProducerFuture = producers.putIfAbsent(producerId, producerFuture); if (existingProducerFuture != null) { - if (existingProducerFuture.isDone() && !existingProducerFuture.isCompletedExceptionally()) { - Producer producer = existingProducerFuture.getNow(null); - log.info("[{}] Producer with the same id is already created:" - + " producerId={}, producer={}", remoteAddress, producerId, producer); - commandSender.sendProducerSuccessResponse(requestId, producer.getProducerName(), - producer.getSchemaVersion()); - return null; - } else { + if (!existingProducerFuture.isDone()) { // There was an early request to create a producer with same producerId. // This can happen when client timeout is lower than the broker timeouts. // We need to wait until the previous producer creation request // either complete or fails. - ServerError error = null; - if (!existingProducerFuture.isDone()) { - error = ServerError.ServiceNotReady; - } else { - error = getErrorCode(existingProducerFuture); - // remove producer with producerId as it's already completed with exception - producers.remove(producerId, existingProducerFuture); - } log.warn("[{}][{}] Producer with id is already present on the connection, producerId={}", remoteAddress, topicName, producerId); - commandSender.sendErrorResponse(requestId, error, "Producer is already present on the connection"); - return null; + commandSender.sendErrorResponse(requestId, ServerError.ServiceNotReady, + "Producer is already present on the connection"); + } else if (existingProducerFuture.isCompletedExceptionally()) { + // remove producer with producerId as it's already completed with exception + log.warn("[{}][{}] Producer with id is failed to register present on the connection, producerId={}", + remoteAddress, topicName, producerId); + ServerError error = getErrorCode(existingProducerFuture); + producers.remove(producerId, existingProducerFuture); + commandSender.sendErrorResponse(requestId, error, + "Producer is already failed to register present on the connection"); + } else { + Producer producer = existingProducerFuture.getNow(null); + log.info("[{}] [{}] Producer with the same id is already created:" + + " producerId={}, producer={}", remoteAddress, topicName, producerId, producer); + commandSender.sendProducerSuccessResponse(requestId, producer.getProducerName(), + producer.getSchemaVersion()); } + return null; } if (log.isDebugEnabled()) { - log.debug("[{}][{}] Creating producer. producerId={}, schema is {}", remoteAddress, topicName, - producerId, schema == null ? "absent" : "present"); + log.debug("[{}][{}] Creating producer. producerId={}, producerName={}, schema is {}", remoteAddress, + topicName, producerId, producerName, schema == null ? "absent" : "present"); } service.getOrCreateTopic(topicName.toString()).thenCompose((Topic topic) -> { 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 53f6af82bee8d..fa99885488134 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 @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.auth; import static org.apache.pulsar.broker.BrokerTestUtil.spyWithoutRecordingInvocations; +import static org.testng.Assert.assertEquals; import com.google.common.collect.Sets; import java.lang.reflect.Field; import java.net.InetSocketAddress; @@ -37,11 +38,14 @@ import java.util.function.Predicate; import javax.ws.rs.container.AsyncResponse; import javax.ws.rs.container.TimeoutHandler; +import lombok.AllArgsConstructor; +import lombok.Data; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationProviderTls; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerTestBase; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminBuilder; @@ -51,6 +55,7 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.auth.AuthenticationDisabled; import org.apache.pulsar.client.impl.auth.AuthenticationTls; +import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -59,6 +64,7 @@ import org.apache.pulsar.utils.ResourceUtils; import org.apache.zookeeper.MockZooKeeper; import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; import org.mockito.Mockito; import org.mockito.internal.util.MockUtil; import org.slf4j.Logger; @@ -663,5 +669,23 @@ public Object[][] incorrectPersistentPolicies() { }; } + protected ServiceProducer getServiceProducer(ProducerImpl clientProducer, String topicName) { + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).join().get(); + org.apache.pulsar.broker.service.Producer serviceProducer = + persistentTopic.getProducers().get(clientProducer.getProducerName()); + long clientProducerId = WhiteboxImpl.getInternalState(clientProducer, "producerId"); + assertEquals(serviceProducer.getProducerId(), clientProducerId); + assertEquals(serviceProducer.getEpoch(), clientProducer.getConnectionHandler().getEpoch()); + return new ServiceProducer(serviceProducer, persistentTopic); + } + + @Data + @AllArgsConstructor + public static class ServiceProducer { + private org.apache.pulsar.broker.service.Producer serviceProducer; + private PersistentTopic persistentTopic; + } + private static final Logger log = LoggerFactory.getLogger(MockedPulsarServiceBaseTest.class); } 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 e66abc34e3650..5fd4881981365 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 @@ -47,6 +47,8 @@ import io.netty.channel.ChannelHandler; import io.netty.channel.embedded.EmbeddedChannel; import io.netty.handler.codec.LengthFieldBasedFrameDecoder; +import io.vertx.core.impl.ConcurrentHashSet; +import java.io.Closeable; import java.io.IOException; import java.lang.reflect.Field; import java.nio.charset.StandardCharsets; @@ -64,6 +66,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; import java.util.function.Supplier; +import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.CloseCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCursorCallback; @@ -74,6 +77,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.commons.lang3.mutable.MutableInt; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockAlwaysExpiredAuthenticationProvider; @@ -93,6 +97,7 @@ import org.apache.pulsar.broker.service.ServerCnx.State; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.utils.ClientChannelHelper; +import org.apache.pulsar.client.api.ProducerAccessMode; import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.common.api.AuthData; import org.apache.pulsar.common.api.proto.AuthMethod; @@ -113,6 +118,7 @@ import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespaceResponse; import org.apache.pulsar.common.api.proto.CommandLookupTopicResponse; import org.apache.pulsar.common.api.proto.CommandPartitionedTopicMetadataResponse; +import org.apache.pulsar.common.api.proto.CommandPing; import org.apache.pulsar.common.api.proto.CommandProducerSuccess; import org.apache.pulsar.common.api.proto.CommandSendError; import org.apache.pulsar.common.api.proto.CommandSendReceipt; @@ -135,6 +141,7 @@ import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.Commands.ChecksumType; import org.apache.pulsar.common.protocol.PulsarHandler; +import org.apache.pulsar.common.protocol.schema.EmptyVersion; import org.apache.pulsar.common.topics.TopicList; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.ConcurrentLongHashMap; @@ -149,6 +156,7 @@ import org.testng.annotations.DataProvider; import org.testng.annotations.Test; +@Slf4j @SuppressWarnings("unchecked") @Test(groups = "broker") public class ServerCnxTest { @@ -184,10 +192,12 @@ public class ServerCnxTest { private ManagedLedger ledgerMock; private ManagedCursor cursorMock; + private ConcurrentHashSet channelsStoppedAnswerHealthCheck = new ConcurrentHashSet<>(); @BeforeMethod(alwaysRun = true) public void setup() throws Exception { + channelsStoppedAnswerHealthCheck.clear(); svcConfig = new ServiceConfiguration(); svcConfig.setBrokerShutdownTimeoutMs(0L); svcConfig.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); @@ -962,6 +972,134 @@ public void testVerifyOriginalPrincipalWithAuthDataForwardedFromProxy() throws E })); } + @Test + public void testHandleProducerAfterClientChannelInactive() throws Exception { + final String tName = successTopicName; + final long producerId = 1; + final MutableInt requestId = new MutableInt(1); + final MutableInt epoch = new MutableInt(1); + final Map metadata = Collections.emptyMap(); + final String pName = "p1"; + resetChannel(); + setChannelConnected(); + + // The producer register using the first connection. + ByteBuf cmdProducer1 = Commands.newProducer(tName, producerId, requestId.incrementAndGet(), + pName, false, metadata, null, epoch.incrementAndGet(), false, + ProducerAccessMode.Shared, Optional.empty(), false); + channel.writeInbound(cmdProducer1); + assertTrue(getResponse() instanceof CommandProducerSuccess); + PersistentTopic topicRef = (PersistentTopic) brokerService.getTopicReference(tName).get(); + assertNotNull(topicRef); + assertEquals(topicRef.getProducers().size(), 1); + + // Verify the second producer using a new connection will override the producer who using a stopped channel. + channelsStoppedAnswerHealthCheck.add(channel); + ClientChannel channel2 = new ClientChannel(); + setChannelConnected(channel2.serverCnx); + Awaitility.await().untilAsserted(() -> { + ByteBuf cmdProducer2 = Commands.newProducer(tName, producerId, requestId.incrementAndGet(), + pName, false, metadata, null, epoch.incrementAndGet(), false, + ProducerAccessMode.Shared, Optional.empty(), false); + channel2.channel.writeInbound(cmdProducer2); + assertTrue(getResponse(channel2.channel, channel2.clientChannelHelper) instanceof CommandProducerSuccess); + assertEquals(topicRef.getProducers().size(), 1); + }); + + // cleanup. + channel.finish(); + channel2.close(); + } + + private class ClientChannel implements Closeable { + private ClientChannelHelper clientChannelHelper = new ClientChannelHelper(); + private ServerCnx serverCnx = new ServerCnx(pulsar); + private EmbeddedChannel channel = new EmbeddedChannel(new LengthFieldBasedFrameDecoder( + 5 * 1024 * 1024, + 0, + 4, + 0, + 4), + serverCnx); + public ClientChannel() { + serverCnx.setAuthRole(""); + } + public void close(){ + if (channel != null && channel.isActive()) { + serverCnx.close(); + channel.close(); + } + } + } + + @Test + public void testHandleProducer() throws Exception { + final String tName = "persistent://public/default/test-topic"; + final long producerId = 1; + final MutableInt requestId = new MutableInt(1); + final MutableInt epoch = new MutableInt(1); + final Map metadata = Collections.emptyMap(); + final String pName = "p1"; + resetChannel(); + assertTrue(channel.isActive()); + assertEquals(serverCnx.getState(), State.Start); + + // connect. + ByteBuf cConnect = Commands.newConnect("none", "", null); + channel.writeInbound(cConnect); + assertEquals(serverCnx.getState(), State.Connected); + assertTrue(getResponse() instanceof CommandConnected); + + // There is an in-progress producer registration. + ByteBuf cProducer1 = Commands.newProducer(tName, producerId, requestId.incrementAndGet(), + pName, false, metadata, null, epoch.incrementAndGet(), false, + ProducerAccessMode.Shared, Optional.empty(), false); + CompletableFuture existingFuture1 = new CompletableFuture(); + serverCnx.getProducers().put(producerId, existingFuture1); + channel.writeInbound(cProducer1); + Object response1 = getResponse(); + assertTrue(response1 instanceof CommandError); + CommandError error1 = (CommandError) response1; + assertEquals(error1.getError().toString(), ServerError.ServiceNotReady.toString()); + assertTrue(error1.getMessage().contains("already present on the connection")); + + // There is a failed registration. + ByteBuf cProducer2 = Commands.newProducer(tName, producerId, requestId.incrementAndGet(), + pName, false, metadata, null, epoch.incrementAndGet(), false, + ProducerAccessMode.Shared, Optional.empty(), false); + CompletableFuture existingFuture2 = new CompletableFuture(); + existingFuture2.completeExceptionally(new BrokerServiceException.ProducerBusyException("123")); + serverCnx.getProducers().put(producerId, existingFuture2); + + channel.writeInbound(cProducer2); + Object response2 = getResponse(); + assertTrue(response2 instanceof CommandError); + CommandError error2 = (CommandError) response2; + assertEquals(error2.getError().toString(), ServerError.ProducerBusy.toString()); + assertTrue(error2.getMessage().contains("already failed to register present on the connection")); + + // There is an successful registration. + ByteBuf cProducer3 = Commands.newProducer(tName, producerId, requestId.incrementAndGet(), + pName, false, metadata, null, epoch.incrementAndGet(), false, + ProducerAccessMode.Shared, Optional.empty(), false); + CompletableFuture existingFuture3 = new CompletableFuture(); + org.apache.pulsar.broker.service.Producer serviceProducer = + mock(org.apache.pulsar.broker.service.Producer.class); + when(serviceProducer.getProducerName()).thenReturn(pName); + when(serviceProducer.getSchemaVersion()).thenReturn(new EmptyVersion()); + existingFuture3.complete(serviceProducer); + serverCnx.getProducers().put(producerId, existingFuture3); + + channel.writeInbound(cProducer3); + Object response3 = getResponse(); + assertTrue(response3 instanceof CommandProducerSuccess); + CommandProducerSuccess cProducerSuccess = (CommandProducerSuccess) response3; + assertEquals(cProducerSuccess.getProducerName(), pName); + + // cleanup. + channel.finish(); + } + // This test used to be in the ServerCnxAuthorizationTest class, but it was migrated here because the mocking // in that class was too extensive. There is some overlap with this test and other tests in this class. The primary // role of this test is verifying that the correct role and AuthenticationDataSource are passed to the @@ -2506,6 +2644,10 @@ protected void resetChannel() throws Exception { } protected void setChannelConnected() throws Exception { + setChannelConnected(serverCnx); + } + + protected void setChannelConnected(ServerCnx serverCnx) throws Exception { Field channelState = ServerCnx.class.getDeclaredField("state"); channelState.setAccessible(true); channelState.set(serverCnx, State.Connected); @@ -2519,13 +2661,31 @@ private void setConnectionVersion(int version) throws Exception { } protected Object getResponse() throws Exception { + return getResponse(channel, clientChannelHelper); + } + + protected Object getResponse(EmbeddedChannel channel, ClientChannelHelper clientChannelHelper) throws Exception { // Wait at most for 10s to get a response final long sleepTimeMs = 10; final long iterations = TimeUnit.SECONDS.toMillis(10) / sleepTimeMs; for (int i = 0; i < iterations; i++) { if (!channel.outboundMessages().isEmpty()) { Object outObject = channel.outboundMessages().remove(); - return clientChannelHelper.getCommand(outObject); + Object cmd = clientChannelHelper.getCommand(outObject); + if (cmd instanceof CommandPing) { + if (channelsStoppedAnswerHealthCheck.contains(channel)) { + continue; + } + channel.writeAndFlush(Commands.newPong()).addListener(future -> { + if (!future.isSuccess()) { + log.warn("[{}] Forcing connection to close since cannot send a pong message.", + channel, future.cause()); + channel.close(); + } + }); + continue; + } + return cmd; } else { Thread.sleep(sleepTimeMs); } 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 5f242d4495480..f05f735635746 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 @@ -19,11 +19,13 @@ package org.apache.pulsar.client.impl; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; import java.util.concurrent.CountDownLatch; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.ServerCnx; import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.common.api.proto.CommandCloseProducer; import org.awaitility.Awaitility; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -49,6 +51,61 @@ protected void cleanup() throws Exception { super.internalCleanup(); } + @Test + public void testSameProducerRegisterTwice() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/tp_"); + admin.topics().createNonPartitionedTopic(topicName); + + // Create producer using default producerName. + ProducerImpl producer = (ProducerImpl) pulsarClient.newProducer().topic(topicName).create(); + ServiceProducer serviceProducer = getServiceProducer(producer, topicName); + + // Remove producer maintained by server cnx. To make it can register the second time. + removeServiceProducerMaintainedByServerCnx(serviceProducer); + + // Trigger the client producer reconnect. + CommandCloseProducer commandCloseProducer = new CommandCloseProducer(); + commandCloseProducer.setProducerId(producer.producerId); + producer.getClientCnx().handleCloseProducer(commandCloseProducer); + + // Verify the reconnection will be success. + Awaitility.await().untilAsserted(() -> { + assertEquals(producer.getState().toString(), "Ready"); + }); + } + + @Test + public void testSameProducerRegisterTwiceWithSpecifiedProducerName() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/tp_"); + final String pName = "p1"; + admin.topics().createNonPartitionedTopic(topicName); + + // Create producer using default producerName. + ProducerImpl producer = (ProducerImpl) pulsarClient.newProducer().producerName(pName).topic(topicName).create(); + ServiceProducer serviceProducer = getServiceProducer(producer, topicName); + + // Remove producer maintained by server cnx. To make it can register the second time. + removeServiceProducerMaintainedByServerCnx(serviceProducer); + + // Trigger the client producer reconnect. + CommandCloseProducer commandCloseProducer = new CommandCloseProducer(); + commandCloseProducer.setProducerId(producer.producerId); + producer.getClientCnx().handleCloseProducer(commandCloseProducer); + + // Verify the reconnection will be success. + Awaitility.await().untilAsserted(() -> { + assertEquals(producer.getState().toString(), "Ready", "The producer registration failed"); + }); + } + + private void removeServiceProducerMaintainedByServerCnx(ServiceProducer serviceProducer) { + ServerCnx serverCnx = (ServerCnx) serviceProducer.getServiceProducer().getCnx(); + serverCnx.removedProducer(serviceProducer.getServiceProducer()); + Awaitility.await().untilAsserted(() -> { + assertFalse(serverCnx.getProducers().containsKey(serviceProducer.getServiceProducer().getProducerId())); + }); + } + @Test public void testExclusiveConsumerWillAlwaysRetryEvenIfReceivedConsumerBusyError() throws Exception { final String topicName = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/tp_"); From 4e22df8e5c0db0bdd1fb3c68485ae4aef7ef27ad Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 16 Oct 2023 15:47:16 +0300 Subject: [PATCH 115/699] [fix][test] Fix a resource leak in ClusterMigrationTest (#21366) (cherry picked from commit 39235edcb45a615627dcd0471ed0872e568790ff) # Conflicts: # pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ClusterMigrationTest.java --- .../broker/service/ClusterMigrationTest.java | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) 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 469e155d409b3..f6a8a03527eb4 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 @@ -25,11 +25,11 @@ 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 java.lang.reflect.Method; import java.net.URL; import java.util.concurrent.TimeUnit; - +import lombok.Cleanup; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; @@ -50,10 +50,6 @@ import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import com.google.common.collect.Sets; - -import lombok.Cleanup; - @Test(groups = "broker") public class ClusterMigrationTest { @@ -202,9 +198,13 @@ public void setup() throws Exception { protected void cleanup() throws Exception { log.info("--- Shutting down ---"); broker1.cleanup(); + admin1.close(); broker2.cleanup(); + admin2.close(); broker3.cleanup(); + admin3.close(); broker4.cleanup(); + admin4.close(); } @BeforeMethod(alwaysRun = true) @@ -399,7 +399,7 @@ public void testClusterMigrationWithReplicationBacklog(boolean persistent, Subsc assertEquals(topic1.getReplicators().size(), 1); // stop service in the replication cluster to build replication backlog - broker3.cleanup(); + broker3.stop(); retryStrategically((test) -> broker3.getPulsarService() == null, 10, 1000); assertNull(pulsar3.getBrokerService()); @@ -485,9 +485,13 @@ public String getClusterName() { return configClusterName; } + public void stop() throws Exception { + stopBroker(); + } + @Override protected void cleanup() throws Exception { - stopBroker(); + internalCleanup(); } public void restart() throws Exception { From 433431818a22bf788deba5692755ddd817ea65fa Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Mon, 9 Oct 2023 12:07:01 +0800 Subject: [PATCH 116/699] [feat][sql] Support UUID for json and avro (#21267) ### Motivation As https://pulsar.apache.org/docs/3.1.x/sql-overview/, Pulsar SQL is based on [Trino (formerly Presto SQL)](https://trino.io/), which supports UUID type. But now, the UUID field in Avro or JSON schema will be interpreted as VARCHAR. ### Modifications Support decoding UUID form AVRO or JSON schema. (cherry picked from commit 8c7094328e03b11bf57e8f9d1022047961b75481) --- .../sql/presto/decoder/avro/PulsarAvroColumnDecoder.java | 9 ++++++++- .../presto/decoder/avro/PulsarAvroRowDecoderFactory.java | 5 +++++ .../sql/presto/decoder/json/PulsarJsonFieldDecoder.java | 4 +++- .../presto/decoder/json/PulsarJsonRowDecoderFactory.java | 5 +++++ .../pulsar/sql/presto/decoder/DecoderTestMessage.java | 4 ++++ .../pulsar/sql/presto/decoder/avro/TestAvroDecoder.java | 7 +++++++ .../pulsar/sql/presto/decoder/json/TestJsonDecoder.java | 8 ++++++++ 7 files changed, 40 insertions(+), 2 deletions(-) diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroColumnDecoder.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroColumnDecoder.java index 73081f8948a51..1672d5f144817 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroColumnDecoder.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroColumnDecoder.java @@ -54,6 +54,7 @@ import io.trino.spi.type.Timestamps; import io.trino.spi.type.TinyintType; import io.trino.spi.type.Type; +import io.trino.spi.type.UuidType; import io.trino.spi.type.VarbinaryType; import io.trino.spi.type.VarcharType; import java.math.BigInteger; @@ -61,6 +62,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.UUID; import org.apache.avro.generic.GenericEnumSymbol; import org.apache.avro.generic.GenericFixed; import org.apache.avro.generic.GenericRecord; @@ -87,7 +89,8 @@ public class PulsarAvroColumnDecoder { TimestampType.TIMESTAMP_MILLIS, DateType.DATE, TimeType.TIME_MILLIS, - VarbinaryType.VARBINARY); + VarbinaryType.VARBINARY, + UuidType.UUID); private final Type columnType; private final String columnMapping; @@ -255,6 +258,10 @@ private static Slice getSlice(Object value, Type type, String columnName) { } } + if (type instanceof UuidType) { + return UuidType.javaUuidToTrinoUuid(UUID.fromString(value.toString())); + } + throw new TrinoException(DECODER_CONVERSION_NOT_SUPPORTED, format("cannot decode object of '%s' as '%s' for column '%s'", value.getClass(), type, columnName)); diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroRowDecoderFactory.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroRowDecoderFactory.java index 3072bf9441b2c..e6eb6b7f2f947 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroRowDecoderFactory.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/avro/PulsarAvroRowDecoderFactory.java @@ -44,6 +44,7 @@ import io.trino.spi.type.TypeManager; import io.trino.spi.type.TypeSignature; import io.trino.spi.type.TypeSignatureParameter; +import io.trino.spi.type.UuidType; import io.trino.spi.type.VarbinaryType; import io.trino.spi.type.VarcharType; import java.util.List; @@ -121,6 +122,10 @@ private Type parseAvroPrestoType(String fieldName, Schema schema) { LogicalType logicalType = schema.getLogicalType(); switch (type) { case STRING: + if (logicalType != null && logicalType.equals(LogicalTypes.uuid())) { + return UuidType.UUID; + } + return createUnboundedVarcharType(); case ENUM: return createUnboundedVarcharType(); case NULL: diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/PulsarJsonFieldDecoder.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/PulsarJsonFieldDecoder.java index 905e3bd6becb4..8e744e3b1229c 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/PulsarJsonFieldDecoder.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/PulsarJsonFieldDecoder.java @@ -58,6 +58,7 @@ import io.trino.spi.type.Timestamps; import io.trino.spi.type.TinyintType; import io.trino.spi.type.Type; +import io.trino.spi.type.UuidType; import io.trino.spi.type.VarbinaryType; import io.trino.spi.type.VarcharType; import java.util.Iterator; @@ -126,7 +127,8 @@ private boolean isSupportedType(Type type) { TimestampType.TIMESTAMP_MILLIS, DateType.DATE, TimeType.TIME_MILLIS, - RealType.REAL + RealType.REAL, + UuidType.UUID ).contains(type)) { return true; } diff --git a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/PulsarJsonRowDecoderFactory.java b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/PulsarJsonRowDecoderFactory.java index 0d5cc2d262dfe..737eb608d82d6 100644 --- a/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/PulsarJsonRowDecoderFactory.java +++ b/pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/decoder/json/PulsarJsonRowDecoderFactory.java @@ -44,6 +44,7 @@ import io.trino.spi.type.TypeManager; import io.trino.spi.type.TypeSignature; import io.trino.spi.type.TypeSignatureParameter; +import io.trino.spi.type.UuidType; import io.trino.spi.type.VarbinaryType; import io.trino.spi.type.VarcharType; import java.util.List; @@ -121,6 +122,10 @@ private Type parseJsonPrestoType(String fieldName, Schema schema) { LogicalType logicalType = schema.getLogicalType(); switch (type) { case STRING: + if (logicalType != null && logicalType.equals(LogicalTypes.uuid())) { + return UuidType.UUID; + } + return createUnboundedVarcharType(); case ENUM: return createUnboundedVarcharType(); case NULL: diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/DecoderTestMessage.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/DecoderTestMessage.java index 4561282c67196..0dec76b3d4dec 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/DecoderTestMessage.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/DecoderTestMessage.java @@ -19,6 +19,7 @@ package org.apache.pulsar.sql.presto.decoder; import java.math.BigDecimal; +import java.util.UUID; import lombok.Data; import java.util.List; @@ -55,6 +56,9 @@ public static enum TestEnum { public Map mapField; public CompositeRow compositeRow; + @org.apache.avro.reflect.AvroSchema("{\"type\":\"string\",\"logicalType\":\"uuid\"}") + public UUID uuidField; + public static class TestRow { public String stringField; public int intField; diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/avro/TestAvroDecoder.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/avro/TestAvroDecoder.java index c4e7009b9465b..5f9df96619b9f 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/avro/TestAvroDecoder.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/avro/TestAvroDecoder.java @@ -44,6 +44,7 @@ import io.trino.spi.type.Timestamps; import io.trino.spi.type.Type; import io.trino.spi.type.TypeSignatureParameter; +import io.trino.spi.type.UuidType; import io.trino.spi.type.VarcharType; import java.math.BigDecimal; import java.time.LocalDate; @@ -55,6 +56,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.UUID; import org.apache.pulsar.client.impl.schema.AvroSchema; import org.apache.pulsar.client.impl.schema.generic.GenericAvroRecord; import org.apache.pulsar.client.impl.schema.generic.GenericAvroSchema; @@ -90,6 +92,7 @@ public void testPrimitiveType() { message.longField = 222L; message.timestampField = System.currentTimeMillis(); message.enumField = DecoderTestMessage.TestEnum.TEST_ENUM_1; + message.uuidField = UUID.randomUUID(); LocalTime now = LocalTime.now(ZoneId.systemDefault()); message.timeField = now.toSecondOfDay() * 1000; @@ -137,6 +140,10 @@ public void testPrimitiveType() { PulsarColumnHandle timeFieldColumnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), "timeField", TIME_MILLIS, false, false, "timeField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); checkValue(decodedRow, timeFieldColumnHandle, (long) message.timeField * Timestamps.PICOSECONDS_PER_MILLISECOND); + + PulsarColumnHandle uuidHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), + "uuidField", UuidType.UUID, false, false, "uuidField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + checkValue(decodedRow, uuidHandle, UuidType.javaUuidToTrinoUuid(message.uuidField)); } @Test diff --git a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/json/TestJsonDecoder.java b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/json/TestJsonDecoder.java index 4afad9b318fc5..32e71a53444cf 100644 --- a/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/json/TestJsonDecoder.java +++ b/pulsar-sql/presto-pulsar/src/test/java/org/apache/pulsar/sql/presto/decoder/json/TestJsonDecoder.java @@ -44,6 +44,7 @@ import io.trino.spi.type.Timestamps; import io.trino.spi.type.Type; import io.trino.spi.type.TypeSignatureParameter; +import io.trino.spi.type.UuidType; import io.trino.spi.type.VarcharType; import java.math.BigDecimal; import java.time.LocalDate; @@ -55,6 +56,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.UUID; import org.apache.pulsar.client.impl.schema.JSONSchema; import org.apache.pulsar.client.impl.schema.generic.GenericJsonRecord; import org.apache.pulsar.client.impl.schema.generic.GenericJsonSchema; @@ -98,6 +100,8 @@ public void testPrimitiveType() { LocalDate epoch = LocalDate.ofEpochDay(0); message.dateField = Math.toIntExact(ChronoUnit.DAYS.between(epoch, localDate)); + message.uuidField = UUID.randomUUID(); + ByteBuf payload = io.netty.buffer.Unpooled .copiedBuffer(schema.encode(message)); Map decodedRow = pulsarRowDecoder.decodeRow(payload).get(); @@ -137,6 +141,10 @@ public void testPrimitiveType() { PulsarColumnHandle timeFieldColumnHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), "timeField", TIME_MILLIS, false, false, "timeField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); checkValue(decodedRow, timeFieldColumnHandle, (long) message.timeField * Timestamps.PICOSECONDS_PER_MILLISECOND); + + PulsarColumnHandle uuidHandle = new PulsarColumnHandle(getPulsarConnectorId().toString(), + "uuidField", UuidType.UUID, false, false, "uuidField", null, null, PulsarColumnHandle.HandleKeyValueType.NONE); + checkValue(decodedRow, uuidHandle, message.uuidField.toString()); } @Test From 4f06c87b0bd31d0330a8efba51e168c844e8cb9a Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 17 Oct 2023 13:17:58 +0300 Subject: [PATCH 117/699] [fix][test] Fix some resource leaks in compaction tests (#21374) (cherry picked from commit e6cd11fb49f2d8f3376f2218e60f8c5909f66348) --- .../java/org/apache/pulsar/compaction/CompactedTopicTest.java | 3 +++ .../pulsar/compaction/ServiceUnitStateCompactionTest.java | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicTest.java index 4a098fd0eb9f0..33e4e998ad6cf 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicTest.java @@ -161,6 +161,7 @@ public void cleanup() throws Exception { @Test public void testEntryLookup() throws Exception { + @Cleanup BookKeeper bk = pulsar.getBookKeeperClientFactory().create( this.conf, null, null, Optional.empty(), null); @@ -216,6 +217,7 @@ public void testEntryLookup() throws Exception { @Test public void testCleanupOldCompactedTopicLedger() throws Exception { + @Cleanup BookKeeper bk = pulsar.getBookKeeperClientFactory().create( this.conf, null, null, Optional.empty(), null); @@ -845,6 +847,7 @@ public void testReadCompactedLatestMessageWithInclusive() throws Exception { @Test public void testCompactWithConcurrentGetCompactionHorizonAndCompactedTopicContext() throws Exception { + @Cleanup BookKeeper bk0 = pulsar.getBookKeeperClientFactory().create( this.conf, null, null, Optional.empty(), null); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/ServiceUnitStateCompactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/ServiceUnitStateCompactionTest.java index e4f0750a981c9..09c3ebe419394 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/ServiceUnitStateCompactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/ServiceUnitStateCompactionTest.java @@ -168,7 +168,7 @@ public void setup() throws Exception { @Override public void cleanup() throws Exception { super.internalCleanup(); - + bk.close(); if (compactionScheduler != null) { compactionScheduler.shutdownNow(); } From ad2f3db877fe4d972be3c95671d7f81f230eb6cc Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 17 Oct 2023 15:34:16 +0300 Subject: [PATCH 118/699] [fix][test] Fix resource leaks with Pulsar Functions tests (#21378) (cherry picked from commit ffc083b5ea9da998c35161b9dcfbfb6e38e3917e) --- .../pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java | 1 - .../pulsar/functions/worker/PulsarFunctionPublishTest.java | 1 - .../pulsar/functions/worker/PulsarWorkerAssignmentTest.java | 1 - .../test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java | 1 - .../test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java | 1 - 5 files changed, 5 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java index 714c9d7269970..107aedd076691 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java @@ -268,7 +268,6 @@ private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration conf workerConfig.setAuthorizationProvider(config.getAuthorizationProvider()); PulsarWorkerService workerService = new PulsarWorkerService(); - workerService.init(workerConfig, null, false); return workerService; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java index c820f512a68de..7bcf1dec871e0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java @@ -269,7 +269,6 @@ private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration conf workerConfig.setAuthorizationEnabled(true); PulsarWorkerService workerService = new PulsarWorkerService(); - workerService.init(workerConfig, null, false); return workerService; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarWorkerAssignmentTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarWorkerAssignmentTest.java index 0821974bea506..6226fa904885c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarWorkerAssignmentTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarWorkerAssignmentTest.java @@ -174,7 +174,6 @@ private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration conf workerConfig.setTopicCompactionFrequencySec(1); PulsarWorkerService workerService = new PulsarWorkerService(); - workerService.init(workerConfig, null, false); return workerService; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java index 3a99cc647ed5c..f968315a7124c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java @@ -307,7 +307,6 @@ private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration conf workerConfig.setAuthorizationEnabled(true); PulsarWorkerService workerService = new PulsarWorkerService(); - workerService.init(workerConfig, null, false); return workerService; } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java index d31d0c66bdf93..22b9ad0df3a69 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/PulsarFunctionAdminTest.java @@ -211,7 +211,6 @@ private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration conf workerConfig.setTlsTrustCertsFilePath(TLS_CLIENT_CERT_FILE_PATH); PulsarWorkerService workerService = new PulsarWorkerService(); - workerService.init(workerConfig, null, false); return workerService; } } From f02785769358612275451df22c1c733cea567d11 Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Tue, 17 Oct 2023 20:04:51 -0500 Subject: [PATCH 119/699] [fix][broker][branch-3.1] Fix lookup heartbeat and sla namespace bundle when using extensible load manager (#21213) (#21314) --- .../apache/pulsar/broker/PulsarService.java | 2 +- .../pulsar/broker/loadbalance/LoadData.java | 2 +- .../extensions/ExtensibleLoadManagerImpl.java | 338 +++++++++++------- .../channel/ServiceUnitStateChannelImpl.java | 71 ++-- .../extensions/models/TopKBundles.java | 5 +- .../broker/namespace/NamespaceService.java | 38 +- .../ExtensibleLoadManagerImplTest.java | 70 +++- .../channel/ServiceUnitStateChannelTest.java | 57 +-- 8 files changed, 320 insertions(+), 263 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 4ffb5b77d5424..6747bbb916d93 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 @@ -1176,7 +1176,7 @@ protected void startLeaderElectionService() { protected void acquireSLANamespace() { try { // Namespace not created hence no need to unload it - NamespaceName nsName = NamespaceService.getSLAMonitorNamespace(getAdvertisedAddress(), config); + NamespaceName nsName = NamespaceService.getSLAMonitorNamespace(getLookupServiceAddress(), config); if (!this.pulsarResources.getNamespaceResources().namespaceExists(nsName)) { LOG.info("SLA Namespace = {} doesn't exist.", nsName); return; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadData.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadData.java index a632a47f05116..c1fe2a4930c34 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadData.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadData.java @@ -64,7 +64,7 @@ public Map getBundleData() { public Map getBundleDataForLoadShedding() { return bundleData.entrySet().stream() - .filter(e -> !NamespaceService.filterNamespaceForShedding( + .filter(e -> !NamespaceService.isSLAOrHeartbeatNamespace( NamespaceBundle.getBundleNamespace(e.getKey()))) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); } 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 cba499eb8eedb..85baf9ec4fbdf 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 @@ -44,6 +44,7 @@ import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang.StringUtils; +import org.apache.commons.lang3.mutable.MutableObject; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; @@ -86,6 +87,7 @@ import org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared; import org.apache.pulsar.broker.loadbalance.impl.SimpleResourceAllocationPolicies; import org.apache.pulsar.broker.namespace.NamespaceEphemeralData; +import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.NamespaceBundleSplitAlgorithm; @@ -95,7 +97,6 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.metadata.api.coordination.LeaderElectionState; import org.slf4j.Logger; @@ -152,6 +153,7 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager { @Getter private final List brokerFilterPipeline; + /** * The load data reporter. */ @@ -181,10 +183,8 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager { // record split metrics private final AtomicReference> splitMetrics = new AtomicReference<>(); - private final ConcurrentOpenHashMap>> - lookupRequests = ConcurrentOpenHashMap.>>newBuilder() - .build(); + private final ConcurrentHashMap>> + lookupRequests = new ConcurrentHashMap<>(); private final CountDownLatch initWaiter = new CountDownLatch(1); /** @@ -197,7 +197,7 @@ public Set getOwnedServiceUnits() { } Set> entrySet = serviceUnitStateChannel.getOwnershipEntrySet(); String brokerId = brokerRegistry.getBrokerId(); - return entrySet.stream() + Set ownedServiceUnits = entrySet.stream() .filter(entry -> { var stateData = entry.getValue(); return stateData.state() == ServiceUnitState.Owned @@ -207,6 +207,36 @@ public Set getOwnedServiceUnits() { var bundle = entry.getKey(); return getNamespaceBundle(pulsar, bundle); }).collect(Collectors.toSet()); + // Add heartbeat and SLA monitor namespace bundle. + NamespaceName heartbeatNamespace = NamespaceService.getHeartbeatNamespace(brokerId, pulsar.getConfiguration()); + try { + NamespaceBundle fullBundle = pulsar.getNamespaceService().getNamespaceBundleFactory() + .getFullBundle(heartbeatNamespace); + ownedServiceUnits.add(fullBundle); + } catch (Exception e) { + log.warn("Failed to get heartbeat namespace bundle.", e); + } + NamespaceName heartbeatNamespaceV2 = NamespaceService + .getHeartbeatNamespaceV2(brokerId, pulsar.getConfiguration()); + try { + NamespaceBundle fullBundle = pulsar.getNamespaceService().getNamespaceBundleFactory() + .getFullBundle(heartbeatNamespaceV2); + ownedServiceUnits.add(fullBundle); + } catch (Exception e) { + log.warn("Failed to get heartbeat namespace V2 bundle.", e); + } + + NamespaceName slaMonitorNamespace = NamespaceService + .getSLAMonitorNamespace(brokerId, pulsar.getConfiguration()); + try { + NamespaceBundle fullBundle = pulsar.getNamespaceService().getNamespaceBundleFactory() + .getFullBundle(slaMonitorNamespace); + ownedServiceUnits.add(fullBundle); + } catch (Exception e) { + log.warn("Failed to get SLA Monitor namespace bundle.", e); + } + + return ownedServiceUnits; } public enum Role { @@ -261,102 +291,108 @@ public void start() throws PulsarServerException { if (this.started) { return; } - this.brokerRegistry = new BrokerRegistryImpl(pulsar); - this.leaderElectionService = new LeaderElectionService( - pulsar.getCoordinationService(), pulsar.getSafeWebServiceAddress(), ELECTION_ROOT, - state -> { - pulsar.getLoadManagerExecutor().execute(() -> { - if (state == LeaderElectionState.Leading) { - playLeader(); - } else { - playFollower(); - } + try { + this.brokerRegistry = new BrokerRegistryImpl(pulsar); + this.leaderElectionService = new LeaderElectionService( + pulsar.getCoordinationService(), pulsar.getSafeWebServiceAddress(), ELECTION_ROOT, + state -> { + pulsar.getLoadManagerExecutor().execute(() -> { + if (state == LeaderElectionState.Leading) { + playLeader(); + } else { + playFollower(); + } + }); }); - }); - this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar); - this.brokerRegistry.start(); - this.splitManager = new SplitManager(splitCounter); - this.unloadManager = new UnloadManager(unloadCounter); - this.serviceUnitStateChannel.listen(unloadManager); - this.serviceUnitStateChannel.listen(splitManager); - this.leaderElectionService.start(); - this.serviceUnitStateChannel.start(); - this.antiAffinityGroupPolicyHelper = - new AntiAffinityGroupPolicyHelper(pulsar, serviceUnitStateChannel); - antiAffinityGroupPolicyHelper.listenFailureDomainUpdate(); - this.antiAffinityGroupPolicyFilter = new AntiAffinityGroupPolicyFilter(antiAffinityGroupPolicyHelper); - this.brokerFilterPipeline.add(antiAffinityGroupPolicyFilter); - SimpleResourceAllocationPolicies policies = new SimpleResourceAllocationPolicies(pulsar); - this.isolationPoliciesHelper = new IsolationPoliciesHelper(policies); - this.brokerFilterPipeline.add(new BrokerIsolationPoliciesFilter(isolationPoliciesHelper)); - - createSystemTopic(pulsar, BROKER_LOAD_DATA_STORE_TOPIC); - createSystemTopic(pulsar, TOP_BUNDLES_LOAD_DATA_STORE_TOPIC); + this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar); + this.brokerRegistry.start(); + this.splitManager = new SplitManager(splitCounter); + this.unloadManager = new UnloadManager(unloadCounter); + this.serviceUnitStateChannel.listen(unloadManager); + this.serviceUnitStateChannel.listen(splitManager); + this.leaderElectionService.start(); + this.serviceUnitStateChannel.start(); + this.antiAffinityGroupPolicyHelper = + new AntiAffinityGroupPolicyHelper(pulsar, serviceUnitStateChannel); + antiAffinityGroupPolicyHelper.listenFailureDomainUpdate(); + this.antiAffinityGroupPolicyFilter = new AntiAffinityGroupPolicyFilter(antiAffinityGroupPolicyHelper); + this.brokerFilterPipeline.add(antiAffinityGroupPolicyFilter); + SimpleResourceAllocationPolicies policies = new SimpleResourceAllocationPolicies(pulsar); + this.isolationPoliciesHelper = new IsolationPoliciesHelper(policies); + this.brokerFilterPipeline.add(new BrokerIsolationPoliciesFilter(isolationPoliciesHelper)); + + createSystemTopic(pulsar, BROKER_LOAD_DATA_STORE_TOPIC); + createSystemTopic(pulsar, TOP_BUNDLES_LOAD_DATA_STORE_TOPIC); - try { - this.brokerLoadDataStore = LoadDataStoreFactory - .create(pulsar.getClient(), BROKER_LOAD_DATA_STORE_TOPIC, BrokerLoadData.class); - this.brokerLoadDataStore.startTableView(); - this.topBundlesLoadDataStore = LoadDataStoreFactory - .create(pulsar.getClient(), TOP_BUNDLES_LOAD_DATA_STORE_TOPIC, TopBundlesLoadData.class); - } catch (LoadDataStoreException e) { - throw new PulsarServerException(e); - } + try { + this.brokerLoadDataStore = LoadDataStoreFactory + .create(pulsar.getClient(), BROKER_LOAD_DATA_STORE_TOPIC, BrokerLoadData.class); + this.brokerLoadDataStore.startTableView(); + this.topBundlesLoadDataStore = LoadDataStoreFactory + .create(pulsar.getClient(), TOP_BUNDLES_LOAD_DATA_STORE_TOPIC, TopBundlesLoadData.class); + } catch (LoadDataStoreException e) { + throw new PulsarServerException(e); + } - this.context = LoadManagerContextImpl.builder() - .configuration(conf) - .brokerRegistry(brokerRegistry) - .brokerLoadDataStore(brokerLoadDataStore) - .topBundleLoadDataStore(topBundlesLoadDataStore).build(); - - this.brokerLoadDataReporter = - new BrokerLoadDataReporter(pulsar, brokerRegistry.getBrokerId(), brokerLoadDataStore); - - this.topBundleLoadDataReporter = - new TopBundleLoadDataReporter(pulsar, brokerRegistry.getBrokerId(), topBundlesLoadDataStore); - this.serviceUnitStateChannel.listen(brokerLoadDataReporter); - this.serviceUnitStateChannel.listen(topBundleLoadDataReporter); - var interval = conf.getLoadBalancerReportUpdateMinIntervalMillis(); - this.brokerLoadDataReportTask = this.pulsar.getLoadManagerExecutor() - .scheduleAtFixedRate(() -> { - try { - brokerLoadDataReporter.reportAsync(false); - // TODO: update broker load metrics using getLocalData - } catch (Throwable e) { - log.error("Failed to run the broker load manager executor job.", e); - } - }, - interval, - interval, TimeUnit.MILLISECONDS); - - this.topBundlesLoadDataReportTask = this.pulsar.getLoadManagerExecutor() - .scheduleAtFixedRate(() -> { - try { - // TODO: consider excluding the bundles that are in the process of split. - topBundleLoadDataReporter.reportAsync(false); - } catch (Throwable e) { - log.error("Failed to run the top bundles load manager executor job.", e); - } - }, - interval, - interval, TimeUnit.MILLISECONDS); - - this.monitorTask = this.pulsar.getLoadManagerExecutor() - .scheduleAtFixedRate(() -> { - monitor(); - }, - MONITOR_INTERVAL_IN_MILLIS, - MONITOR_INTERVAL_IN_MILLIS, TimeUnit.MILLISECONDS); - - this.unloadScheduler = new UnloadScheduler( - pulsar, pulsar.getLoadManagerExecutor(), unloadManager, context, - serviceUnitStateChannel, unloadCounter, unloadMetrics); - this.unloadScheduler.start(); - this.splitScheduler = new SplitScheduler( - pulsar, serviceUnitStateChannel, splitManager, splitCounter, splitMetrics, context); - this.splitScheduler.start(); - this.initWaiter.countDown(); - this.started = true; + this.context = LoadManagerContextImpl.builder() + .configuration(conf) + .brokerRegistry(brokerRegistry) + .brokerLoadDataStore(brokerLoadDataStore) + .topBundleLoadDataStore(topBundlesLoadDataStore).build(); + + this.brokerLoadDataReporter = + new BrokerLoadDataReporter(pulsar, brokerRegistry.getBrokerId(), brokerLoadDataStore); + + this.topBundleLoadDataReporter = + new TopBundleLoadDataReporter(pulsar, brokerRegistry.getBrokerId(), topBundlesLoadDataStore); + this.serviceUnitStateChannel.listen(brokerLoadDataReporter); + this.serviceUnitStateChannel.listen(topBundleLoadDataReporter); + var interval = conf.getLoadBalancerReportUpdateMinIntervalMillis(); + this.brokerLoadDataReportTask = this.pulsar.getLoadManagerExecutor() + .scheduleAtFixedRate(() -> { + try { + brokerLoadDataReporter.reportAsync(false); + // TODO: update broker load metrics using getLocalData + } catch (Throwable e) { + log.error("Failed to run the broker load manager executor job.", e); + } + }, + interval, + interval, TimeUnit.MILLISECONDS); + + this.topBundlesLoadDataReportTask = this.pulsar.getLoadManagerExecutor() + .scheduleAtFixedRate(() -> { + try { + // TODO: consider excluding the bundles that are in the process of split. + topBundleLoadDataReporter.reportAsync(false); + } catch (Throwable e) { + log.error("Failed to run the top bundles load manager executor job.", e); + } + }, + interval, + interval, TimeUnit.MILLISECONDS); + + this.monitorTask = this.pulsar.getLoadManagerExecutor() + .scheduleAtFixedRate(() -> { + monitor(); + }, + MONITOR_INTERVAL_IN_MILLIS, + MONITOR_INTERVAL_IN_MILLIS, TimeUnit.MILLISECONDS); + + this.unloadScheduler = new UnloadScheduler( + pulsar, pulsar.getLoadManagerExecutor(), unloadManager, context, + serviceUnitStateChannel, unloadCounter, unloadMetrics); + this.unloadScheduler.start(); + this.splitScheduler = new SplitScheduler( + pulsar, serviceUnitStateChannel, splitManager, splitCounter, splitMetrics, context); + this.splitScheduler.start(); + this.initWaiter.countDown(); + this.started = true; + } catch (Exception ex) { + if (this.brokerRegistry != null) { + brokerRegistry.close(); + } + } } @Override @@ -377,25 +413,38 @@ public CompletableFuture> assign(Optional getOwnerAsync( - ServiceUnitId serviceUnit, String bundle, boolean ownByLocalBrokerIfAbsent) { + private String getHeartbeatOrSLAMonitorBrokerId(ServiceUnitId serviceUnit) { + // Check if this is Heartbeat or SLAMonitor namespace + String candidateBroker = NamespaceService.checkHeartbeatNamespace(serviceUnit); + if (candidateBroker == null) { + candidateBroker = NamespaceService.checkHeartbeatNamespaceV2(serviceUnit); + } + if (candidateBroker == null) { + candidateBroker = NamespaceService.getSLAMonitorBrokerName(serviceUnit); + } + if (candidateBroker != null) { + return candidateBroker.substring(candidateBroker.lastIndexOf('/') + 1); + } + return candidateBroker; + } + + private CompletableFuture getOrSelectOwnerAsync(ServiceUnitId serviceUnit, + String bundle) { return serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> { // If the bundle not assign yet, select and publish assign event to channel. if (broker.isEmpty()) { - CompletableFuture> selectedBroker; - if (ownByLocalBrokerIfAbsent) { - String brokerId = this.brokerRegistry.getBrokerId(); - selectedBroker = CompletableFuture.completedFuture(Optional.of(brokerId)); - } else { - selectedBroker = this.selectAsync(serviceUnit); - } - return selectedBroker.thenCompose(brokerOpt -> { + return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> { if (brokerOpt.isPresent()) { assignCounter.incrementSuccess(); log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle); @@ -425,7 +474,8 @@ private CompletableFuture> getBrokerLookupData( }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> { if (brokerLookupData.isEmpty()) { String errorMsg = String.format( - "Failed to look up a broker registry:%s for bundle:%s", broker, bundle); + "Failed to lookup broker:%s for bundle:%s, the broker has not been registered.", + broker, bundle); log.error(errorMsg); throw new IllegalStateException(errorMsg); } @@ -443,30 +493,37 @@ private CompletableFuture> getBrokerLookupData( public CompletableFuture tryAcquiringOwnership(NamespaceBundle namespaceBundle) { log.info("Try acquiring ownership for bundle: {} - {}.", namespaceBundle, brokerRegistry.getBrokerId()); final String bundle = namespaceBundle.toString(); - return dedupeLookupRequest(bundle, k -> { - final CompletableFuture owner = - this.getOwnerAsync(namespaceBundle, bundle, true); - return getBrokerLookupData(owner.thenApply(Optional::ofNullable), bundle); - }).thenApply(brokerLookupData -> { - if (brokerLookupData.isEmpty()) { - throw new IllegalStateException( - "Failed to get the broker lookup data for bundle: " + bundle); - } - return brokerLookupData.get().toNamespaceEphemeralData(); - }); + return assign(Optional.empty(), namespaceBundle) + .thenApply(brokerLookupData -> { + if (brokerLookupData.isEmpty()) { + String errorMsg = String.format( + "Failed to get the broker lookup data for bundle:%s", bundle); + log.error(errorMsg); + throw new IllegalStateException(errorMsg); + } + return brokerLookupData.get().toNamespaceEphemeralData(); + }); } private CompletableFuture> dedupeLookupRequest( String key, Function>> provider) { - CompletableFuture> future = lookupRequests.computeIfAbsent(key, provider); - future.whenComplete((r, t) -> { - if (t != null) { + final MutableObject>> newFutureCreated = new MutableObject<>(); + try { + return lookupRequests.computeIfAbsent(key, k -> { + CompletableFuture> future = provider.apply(k); + newFutureCreated.setValue(future); + return future; + }); + } finally { + if (newFutureCreated.getValue() != null) { + newFutureCreated.getValue().whenComplete((v, ex) -> { + if (ex != null) { assignCounter.incrementFailure(); } - lookupRequests.remove(key); - } - ); - return future; + lookupRequests.remove(key, newFutureCreated.getValue()); + }); + } + } } public CompletableFuture> selectAsync(ServiceUnitId bundle) { @@ -521,15 +578,16 @@ public CompletableFuture checkOwnershipAsync(Optional to } public CompletableFuture> getOwnershipAsync(Optional topic, - ServiceUnitId bundleUnit) { - final String bundle = bundleUnit.toString(); - CompletableFuture> owner; + ServiceUnitId serviceUnit) { + final String bundle = serviceUnit.toString(); if (topic.isPresent() && isInternalTopic(topic.get().toString())) { - owner = serviceUnitStateChannel.getChannelOwnerAsync(); - } else { - owner = serviceUnitStateChannel.getOwnerAsync(bundle); + return serviceUnitStateChannel.getChannelOwnerAsync(); } - return owner; + String candidateBroker = getHeartbeatOrSLAMonitorBrokerId(serviceUnit); + if (candidateBroker != null) { + return CompletableFuture.completedFuture(Optional.of(candidateBroker)); + } + return serviceUnitStateChannel.getOwnerAsync(bundle); } public CompletableFuture> getOwnershipWithLookupDataAsync(ServiceUnitId bundleUnit) { @@ -543,6 +601,10 @@ public CompletableFuture> getOwnershipWithLookupDataA public CompletableFuture unloadNamespaceBundleAsync(ServiceUnitId bundle, Optional destinationBroker) { + if (NamespaceService.isSLAOrHeartbeatNamespace(bundle.getNamespaceObject().toString())) { + log.info("Skip unloading namespace bundle: {}.", bundle); + return CompletableFuture.completedFuture(null); + } return getOwnershipAsync(Optional.empty(), bundle) .thenCompose(brokerOpt -> { if (brokerOpt.isEmpty()) { @@ -577,6 +639,10 @@ private CompletableFuture unloadAsync(UnloadDecision unloadDecision, public CompletableFuture splitNamespaceBundleAsync(ServiceUnitId bundle, NamespaceBundleSplitAlgorithm splitAlgorithm, List boundaries) { + if (NamespaceService.isSLAOrHeartbeatNamespace(bundle.getNamespaceObject().toString())) { + log.info("Skip split namespace bundle: {}.", bundle); + return CompletableFuture.completedFuture(null); + } final String namespaceName = LoadManagerShared.getNamespaceNameFromBundleName(bundle.toString()); final String bundleRange = LoadManagerShared.getBundleRangeFromBundleName(bundle.toString()); NamespaceBundle namespaceBundle = 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 98aa02d4e72b4..d71513652e9b8 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 @@ -41,8 +41,6 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.MetadataState.Stable; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.MetadataState.Unstable; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData.state; -import static org.apache.pulsar.broker.namespace.NamespaceService.HEARTBEAT_NAMESPACE_FMT; -import static org.apache.pulsar.broker.namespace.NamespaceService.HEARTBEAT_NAMESPACE_FMT_V2; import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; import static org.apache.pulsar.common.topics.TopicCompactionStrategy.TABLE_VIEW_TAG; import static org.apache.pulsar.metadata.api.extended.SessionEvent.SessionLost; @@ -94,7 +92,6 @@ 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.NamespaceName; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.stats.Metrics; @@ -1216,48 +1213,19 @@ private synchronized void doCleanup(String broker) { log.info("Started ownership cleanup for the inactive broker:{}", broker); int orphanServiceUnitCleanupCnt = 0; long totalCleanupErrorCntStart = totalCleanupErrorCnt.get(); - String heartbeatNamespace = - NamespaceName.get(String.format(HEARTBEAT_NAMESPACE_FMT, config.getClusterName(), broker)).toString(); - String heartbeatNamespaceV2 = - NamespaceName.get(String.format(HEARTBEAT_NAMESPACE_FMT_V2, broker)).toString(); - Map orphanSystemServiceUnits = new HashMap<>(); for (var etr : tableview.entrySet()) { var stateData = etr.getValue(); var serviceUnit = etr.getKey(); var state = state(stateData); - if (StringUtils.equals(broker, stateData.dstBroker())) { - if (isActiveState(state)) { - if (serviceUnit.startsWith(SYSTEM_NAMESPACE.toString())) { - orphanSystemServiceUnits.put(serviceUnit, stateData); - } else if (serviceUnit.startsWith(heartbeatNamespace) - || serviceUnit.startsWith(heartbeatNamespaceV2)) { - // Skip the heartbeat namespace - log.info("Skip override heartbeat namespace bundle" - + " serviceUnit:{}, stateData:{}", serviceUnit, stateData); - tombstoneAsync(serviceUnit).whenComplete((__, e) -> { - if (e != null) { - log.error("Failed cleaning the heartbeat namespace ownership serviceUnit:{}, " - + "stateData:{}, cleanupErrorCnt:{}.", - serviceUnit, stateData, - totalCleanupErrorCnt.incrementAndGet() - totalCleanupErrorCntStart, e); - } - }); - } else { - overrideOwnership(serviceUnit, stateData, broker); - } - orphanServiceUnitCleanupCnt++; - } - - } else if (StringUtils.equals(broker, stateData.sourceBroker())) { - if (isInFlightState(state)) { - if (serviceUnit.startsWith(SYSTEM_NAMESPACE.toString())) { - orphanSystemServiceUnits.put(serviceUnit, stateData); - } else { - overrideOwnership(serviceUnit, stateData, broker); - } - orphanServiceUnitCleanupCnt++; + if (StringUtils.equals(broker, stateData.dstBroker()) && isActiveState(state) + || StringUtils.equals(broker, stateData.sourceBroker()) && isInFlightState(state)) { + if (serviceUnit.startsWith(SYSTEM_NAMESPACE.toString())) { + orphanSystemServiceUnits.put(serviceUnit, stateData); + } else { + overrideOwnership(serviceUnit, stateData, broker); } + orphanServiceUnitCleanupCnt++; } } @@ -1401,16 +1369,21 @@ protected void monitorOwnerships(List brokers) { String srcBroker = stateData.sourceBroker(); var state = stateData.state(); - if (isActiveState(state)) { - if (StringUtils.isNotBlank(srcBroker) && !activeBrokers.contains(srcBroker)) { - inactiveBrokers.add(srcBroker); - } else if (StringUtils.isNotBlank(dstBroker) && !activeBrokers.contains(dstBroker)) { - inactiveBrokers.add(dstBroker); - } else if (isInFlightState(state) - && now - stateData.timestamp() > inFlightStateWaitingTimeInMillis) { - orphanServiceUnits.put(serviceUnit, stateData); - } - } else if (now - stateData.timestamp() > semiTerminalStateWaitingTimeInMillis) { + if (isActiveState(state) && StringUtils.isNotBlank(srcBroker) && !activeBrokers.contains(srcBroker)) { + inactiveBrokers.add(srcBroker); + continue; + } + if (isActiveState(state) && StringUtils.isNotBlank(dstBroker) && !activeBrokers.contains(dstBroker)) { + inactiveBrokers.add(dstBroker); + continue; + } + if (isActiveState(state) && isInFlightState(state) + && now - stateData.timestamp() > inFlightStateWaitingTimeInMillis) { + orphanServiceUnits.put(serviceUnit, stateData); + continue; + } + + if (now - stateData.timestamp() > semiTerminalStateWaitingTimeInMillis) { log.info("Found semi-terminal states to tombstone" + " serviceUnit:{}, stateData:{}", serviceUnit, stateData); tombstoneAsync(serviceUnit).whenComplete((__, e) -> { 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 2f5c32197c1fd..624546fdff837 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 @@ -30,6 +30,8 @@ import org.apache.pulsar.broker.loadbalance.extensions.data.TopBundlesLoadData; import org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared; import org.apache.pulsar.broker.loadbalance.impl.SimpleResourceAllocationPolicies; +import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats; @@ -70,7 +72,8 @@ public void update(Map bundleStats, int topk) { pulsar.getConfiguration().isLoadBalancerSheddingBundlesWithPoliciesEnabled(); for (var etr : bundleStats.entrySet()) { String bundle = etr.getKey(); - if (bundle.startsWith(NamespaceName.SYSTEM_NAMESPACE.toString())) { + // TODO: do not filter system topic while shedding + if (NamespaceService.isSystemServiceNamespace(NamespaceBundle.getBundleNamespace(bundle))) { continue; } if (!isLoadBalancerSheddingBundlesWithPoliciesEnabled && hasPolicies(bundle)) { 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 d66e3c3b65d76..57c0cc7c0464b 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 @@ -135,7 +135,7 @@ public class NamespaceService implements AutoCloseable { public static final Pattern SLA_NAMESPACE_PATTERN = Pattern.compile(SLA_NAMESPACE_PROPERTY + "/[^/]+/([^:]+:\\d+)"); public static final String HEARTBEAT_NAMESPACE_FMT = "pulsar/%s/%s"; public static final String HEARTBEAT_NAMESPACE_FMT_V2 = "pulsar/%s"; - public static final String SLA_NAMESPACE_FMT = SLA_NAMESPACE_PROPERTY + "/%s/%s:%s"; + public static final String SLA_NAMESPACE_FMT = SLA_NAMESPACE_PROPERTY + "/%s/%s"; private final ConcurrentOpenHashMap namespaceClients; @@ -189,7 +189,7 @@ public CompletableFuture> getBrokerServiceUrlAsync(TopicN CompletableFuture> future = getBundleAsync(topic) .thenCompose(bundle -> { // Do redirection if the cluster is in rollback or deploying. - return redirectManager.findRedirectLookupResultAsync().thenCompose(optResult -> { + return findRedirectLookupResultAsync(bundle).thenCompose(optResult -> { if (optResult.isPresent()) { LOG.info("[{}] Redirect lookup request to {} for topic {}", pulsar.getSafeWebServiceAddress(), optResult.get(), topic); @@ -221,6 +221,13 @@ public CompletableFuture> getBrokerServiceUrlAsync(TopicN return future; } + private CompletableFuture> findRedirectLookupResultAsync(ServiceUnitId bundle) { + if (isSLAOrHeartbeatNamespace(bundle.getNamespaceObject().toString())) { + return CompletableFuture.completedFuture(Optional.empty()); + } + return redirectManager.findRedirectLookupResultAsync(); + } + public CompletableFuture getBundleAsync(TopicName topic) { return bundleFactory.getBundlesAsync(topic.getNamespaceObject()) .thenApply(bundles -> bundles.findBundle(topic)); @@ -288,8 +295,7 @@ public Optional getWebServiceUrl(ServiceUnitId suName, LookupOptions option private CompletableFuture> internalGetWebServiceUrl(@Nullable ServiceUnitId topic, NamespaceBundle bundle, LookupOptions options) { - - return redirectManager.findRedirectLookupResultAsync().thenCompose(optResult -> { + return findRedirectLookupResultAsync(bundle).thenCompose(optResult -> { if (optResult.isPresent()) { LOG.info("[{}] Redirect lookup request to {} for topic {}", pulsar.getSafeWebServiceAddress(), optResult.get(), topic); @@ -695,7 +701,7 @@ public CompletableFuture createLookupResult(String candidateBroker return lookupFuture; } - private boolean isBrokerActive(String candidateBroker) { + public boolean isBrokerActive(String candidateBroker) { String candidateBrokerHostAndPort = parseHostAndPort(candidateBroker); Set availableBrokers = getAvailableBrokers(); if (availableBrokers.contains(candidateBrokerHostAndPort)) { @@ -1564,7 +1570,7 @@ public CompletableFuture checkOwnershipPresentAsync(NamespaceBundle bun } public void unloadSLANamespace() throws Exception { - NamespaceName namespaceName = getSLAMonitorNamespace(host, config); + NamespaceName namespaceName = getSLAMonitorNamespace(pulsar.getLookupServiceAddress(), config); LOG.info("Checking owner for SLA namespace {}", namespaceName); @@ -1589,14 +1595,8 @@ public static NamespaceName getHeartbeatNamespaceV2(String lookupBroker, Service return NamespaceName.get(String.format(HEARTBEAT_NAMESPACE_FMT_V2, lookupBroker)); } - public static NamespaceName getSLAMonitorNamespace(String host, ServiceConfiguration config) { - Integer port = null; - if (config.getWebServicePort().isPresent()) { - port = config.getWebServicePort().get(); - } else if (config.getWebServicePortTls().isPresent()) { - port = config.getWebServicePortTls().get(); - } - return NamespaceName.get(String.format(SLA_NAMESPACE_FMT, config.getClusterName(), host, port)); + public static NamespaceName getSLAMonitorNamespace(String lookupBroker, ServiceConfiguration config) { + return NamespaceName.get(String.format(SLA_NAMESPACE_FMT, config.getClusterName(), lookupBroker)); } public static String checkHeartbeatNamespace(ServiceUnitId ns) { @@ -1640,7 +1640,7 @@ public static boolean isSystemServiceNamespace(String namespace) { * @param namespace the namespace name * @return True if namespace is HEARTBEAT_NAMESPACE or SLA_NAMESPACE */ - public static boolean filterNamespaceForShedding(String namespace) { + public static boolean isSLAOrHeartbeatNamespace(String namespace) { return SLA_NAMESPACE_PATTERN.matcher(namespace).matches() || HEARTBEAT_NAMESPACE_PATTERN.matcher(namespace).matches() || HEARTBEAT_NAMESPACE_PATTERN_V2.matcher(namespace).matches(); @@ -1653,14 +1653,16 @@ public static boolean isHeartbeatNamespace(ServiceUnitId ns) { } public boolean registerSLANamespace() throws PulsarServerException { - boolean isNameSpaceRegistered = registerNamespace(getSLAMonitorNamespace(host, config), false); + String lookupServiceAddress = pulsar.getLookupServiceAddress(); + boolean isNameSpaceRegistered = registerNamespace(getSLAMonitorNamespace(lookupServiceAddress, config), false); if (isNameSpaceRegistered) { if (LOG.isDebugEnabled()) { LOG.debug("Added SLA Monitoring namespace name in local cache: ns={}", - getSLAMonitorNamespace(host, config)); + getSLAMonitorNamespace(lookupServiceAddress, config)); } } else if (LOG.isDebugEnabled()) { - LOG.debug("SLA Monitoring not owned by the broker: ns={}", getSLAMonitorNamespace(host, config)); + LOG.debug("SLA Monitoring not owned by the broker: ns={}", + getSLAMonitorNamespace(lookupServiceAddress, config)); } return isNameSpaceRegistered; } 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 9ce57a88540c3..011e7174cbec2 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 @@ -35,6 +35,9 @@ import static org.apache.pulsar.broker.loadbalance.extensions.models.UnloadDecision.Reason.Overloaded; import static org.apache.pulsar.broker.loadbalance.extensions.models.UnloadDecision.Reason.Underloaded; import static org.apache.pulsar.broker.loadbalance.extensions.models.UnloadDecision.Reason.Unknown; +import static org.apache.pulsar.broker.namespace.NamespaceService.getHeartbeatNamespace; +import static org.apache.pulsar.broker.namespace.NamespaceService.getHeartbeatNamespaceV2; +import static org.apache.pulsar.broker.namespace.NamespaceService.getSLAMonitorNamespace; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -594,6 +597,18 @@ public void testDeployAndRollbackLoadManager() throws Exception { assertTrue(webServiceUrl3.isPresent()); assertEquals(webServiceUrl3.get().toString(), webServiceUrl1.get().toString()); + List pulsarServices = List.of(pulsar1, pulsar2, pulsar3); + for (PulsarService pulsarService : pulsarServices) { + // Test lookup heartbeat namespace's topic + for (PulsarService pulsar : pulsarServices) { + assertLookupHeartbeatOwner(pulsarService, pulsar.getLookupServiceAddress(), pulsar.getBrokerServiceUrl()); + } + // Test lookup SLA namespace's topic + for (PulsarService pulsar : pulsarServices) { + assertLookupSLANamespaceOwner(pulsarService, pulsar.getLookupServiceAddress(), pulsar.getBrokerServiceUrl()); + } + } + // Test deploy new broker with new load manager ServiceConfiguration conf = getDefaultConf(); conf.setAllowAutoTopicCreation(true); @@ -642,10 +657,48 @@ public void testDeployAndRollbackLoadManager() throws Exception { assertTrue(webServiceUrl4.isPresent()); assertEquals(webServiceUrl4.get().toString(), webServiceUrl1.get().toString()); + pulsarServices = List.of(pulsar1, pulsar2, pulsar3, pulsar4); + for (PulsarService pulsarService : pulsarServices) { + // Test lookup heartbeat namespace's topic + for (PulsarService pulsar : pulsarServices) { + assertLookupHeartbeatOwner(pulsarService, pulsar.getLookupServiceAddress(), pulsar.getBrokerServiceUrl()); + } + // Test lookup SLA namespace's topic + for (PulsarService pulsar : pulsarServices) { + assertLookupSLANamespaceOwner(pulsarService, pulsar.getLookupServiceAddress(), pulsar.getBrokerServiceUrl()); + } + } } } } + private void assertLookupHeartbeatOwner(PulsarService pulsar, + String lookupServiceAddress, + String expectedBrokerServiceUrl) throws Exception { + NamespaceName heartbeatNamespaceV1 = + getHeartbeatNamespace(lookupServiceAddress, pulsar.getConfiguration()); + + String heartbeatV1Topic = heartbeatNamespaceV1.getPersistentTopicName("test"); + assertEquals(pulsar.getAdminClient().lookups().lookupTopic(heartbeatV1Topic), expectedBrokerServiceUrl); + + NamespaceName heartbeatNamespaceV2 = + getHeartbeatNamespaceV2(lookupServiceAddress, pulsar.getConfiguration()); + + String heartbeatV2Topic = heartbeatNamespaceV2.getPersistentTopicName("test"); + assertEquals(pulsar.getAdminClient().lookups().lookupTopic(heartbeatV2Topic), expectedBrokerServiceUrl); + } + + private void assertLookupSLANamespaceOwner(PulsarService pulsar, + String lookupServiceAddress, + String expectedBrokerServiceUrl) throws Exception { + NamespaceName slaMonitorNamespace = getSLAMonitorNamespace(lookupServiceAddress, pulsar.getConfiguration()); + String slaMonitorTopic = slaMonitorNamespace.getPersistentTopicName("test"); + String result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); + log.info("Topic {} Lookup result: {}", slaMonitorTopic, result); + assertNotNull(result); + assertEquals(result, expectedBrokerServiceUrl); + } + @Test public void testTopBundlesLoadDataStoreTableViewFromChannelOwner() throws Exception { var topBundlesLoadDataStorePrimary = @@ -1043,15 +1096,15 @@ public void testListTopic() throws Exception { admin.namespaces().deleteNamespace(namespace, true); } - @Test(timeOut = 30 * 1000) + @Test(timeOut = 30 * 1000, priority = -1) public void testGetOwnedServiceUnitsAndGetOwnedNamespaceStatus() throws Exception { NamespaceName heartbeatNamespacePulsar1V1 = - NamespaceService.getHeartbeatNamespace(pulsar1.getLookupServiceAddress(), pulsar1.getConfiguration()); + getHeartbeatNamespace(pulsar1.getLookupServiceAddress(), pulsar1.getConfiguration()); NamespaceName heartbeatNamespacePulsar1V2 = NamespaceService.getHeartbeatNamespaceV2(pulsar1.getLookupServiceAddress(), pulsar1.getConfiguration()); NamespaceName heartbeatNamespacePulsar2V1 = - NamespaceService.getHeartbeatNamespace(pulsar2.getLookupServiceAddress(), pulsar2.getConfiguration()); + getHeartbeatNamespace(pulsar2.getLookupServiceAddress(), pulsar2.getConfiguration()); NamespaceName heartbeatNamespacePulsar2V2 = NamespaceService.getHeartbeatNamespaceV2(pulsar2.getLookupServiceAddress(), pulsar2.getConfiguration()); @@ -1068,22 +1121,22 @@ public void testGetOwnedServiceUnitsAndGetOwnedNamespaceStatus() throws Exceptio Set ownedServiceUnitsByPulsar1 = primaryLoadManager.getOwnedServiceUnits(); log.info("Owned service units: {}", ownedServiceUnitsByPulsar1); // heartbeat namespace bundle will own by pulsar1 - assertEquals(ownedServiceUnitsByPulsar1.size(), 2); + assertEquals(ownedServiceUnitsByPulsar1.size(), 3); assertTrue(ownedServiceUnitsByPulsar1.contains(bundle1)); assertTrue(ownedServiceUnitsByPulsar1.contains(bundle2)); Set ownedServiceUnitsByPulsar2 = secondaryLoadManager.getOwnedServiceUnits(); log.info("Owned service units: {}", ownedServiceUnitsByPulsar2); - assertEquals(ownedServiceUnitsByPulsar2.size(), 2); + assertEquals(ownedServiceUnitsByPulsar2.size(), 3); assertTrue(ownedServiceUnitsByPulsar2.contains(bundle3)); assertTrue(ownedServiceUnitsByPulsar2.contains(bundle4)); Map ownedNamespacesByPulsar1 = admin.brokers().getOwnedNamespaces(conf.getClusterName(), pulsar1.getLookupServiceAddress()); Map ownedNamespacesByPulsar2 = admin.brokers().getOwnedNamespaces(conf.getClusterName(), pulsar2.getLookupServiceAddress()); - assertEquals(ownedNamespacesByPulsar1.size(), 2); + assertEquals(ownedNamespacesByPulsar1.size(), 3); assertTrue(ownedNamespacesByPulsar1.containsKey(bundle1.toString())); assertTrue(ownedNamespacesByPulsar1.containsKey(bundle2.toString())); - assertEquals(ownedNamespacesByPulsar2.size(), 2); + assertEquals(ownedNamespacesByPulsar2.size(), 3); assertTrue(ownedNamespacesByPulsar2.containsKey(bundle3.toString())); assertTrue(ownedNamespacesByPulsar2.containsKey(bundle4.toString())); @@ -1134,7 +1187,8 @@ public void testTryAcquiringOwnership() String topic = "persistent://" + namespace + "/test"; NamespaceBundle bundle = getBundleAsync(pulsar1, TopicName.get(topic)).get(); NamespaceEphemeralData namespaceEphemeralData = primaryLoadManager.tryAcquiringOwnership(bundle).get(); - assertEquals(namespaceEphemeralData.getNativeUrl(), pulsar1.getBrokerServiceUrl()); + assertTrue(Set.of(pulsar1.getBrokerServiceUrl(), pulsar2.getBrokerServiceUrl()) + .contains(namespaceEphemeralData.getNativeUrl())); admin.namespaces().deleteNamespace(namespace, true); } 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 de21842f4584c..a226df53e12f3 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 @@ -30,8 +30,6 @@ import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.EventType.Unload; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.MAX_CLEAN_UP_DELAY_TIME_IN_SECS; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData.state; -import static org.apache.pulsar.broker.namespace.NamespaceService.HEARTBEAT_NAMESPACE_FMT; -import static org.apache.pulsar.broker.namespace.NamespaceService.HEARTBEAT_NAMESPACE_FMT_V2; import static org.apache.pulsar.metadata.api.extended.SessionEvent.ConnectionLost; import static org.apache.pulsar.metadata.api.extended.SessionEvent.Reconnected; import static org.apache.pulsar.metadata.api.extended.SessionEvent.SessionLost; @@ -89,7 +87,6 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.client.impl.TableViewImpl; -import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.policies.data.TopicType; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.metadata.api.MetadataStoreException; @@ -639,7 +636,7 @@ public void splitAndRetryTest() throws Exception { var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; validateMonitorCounters(leader, 0, - 1, + 3, 0, 0, 0, @@ -756,34 +753,6 @@ public void handleBrokerDeletionEventTest() waitUntilNewOwner(channel1, bundle2, broker); waitUntilNewOwner(channel2, bundle2, broker); - // Register the broker-1 heartbeat namespace bundle. - String heartbeatNamespaceBroker1V1 = NamespaceName - .get(String.format(HEARTBEAT_NAMESPACE_FMT, conf.getClusterName(), broker)).toString(); - String heartbeatNamespaceBroker1V2 = NamespaceName - .get(String.format(HEARTBEAT_NAMESPACE_FMT_V2, broker)).toString(); - String heartbeatNamespaceBroker1V1Bundle = heartbeatNamespaceBroker1V1 + "/0x00000000_0xfffffff0"; - String heartbeatNamespaceBroker1V2Bundle = heartbeatNamespaceBroker1V2 + "/0x00000000_0xfffffff0"; - channel1.publishAssignEventAsync(heartbeatNamespaceBroker1V1Bundle, broker); - channel1.publishAssignEventAsync(heartbeatNamespaceBroker1V2Bundle, broker); - - // Register the broker-2 heartbeat namespace bundle. - String heartbeatNamespaceBroker2V1 = NamespaceName - .get(String.format(HEARTBEAT_NAMESPACE_FMT, conf.getClusterName(), lookupServiceAddress2)).toString(); - String heartbeatNamespaceBroker2V2 = NamespaceName - .get(String.format(HEARTBEAT_NAMESPACE_FMT_V2, lookupServiceAddress2)).toString(); - String heartbeatNamespaceBroker2V1Bundle = heartbeatNamespaceBroker2V1 + "/0x00000000_0xfffffff0"; - String heartbeatNamespaceBroker2V2Bundle = heartbeatNamespaceBroker2V2 + "/0x00000000_0xfffffff0"; - channel1.publishAssignEventAsync(heartbeatNamespaceBroker2V1Bundle, lookupServiceAddress2); - channel1.publishAssignEventAsync(heartbeatNamespaceBroker2V2Bundle, lookupServiceAddress2); - waitUntilNewOwner(channel1, heartbeatNamespaceBroker1V1Bundle, broker); - waitUntilNewOwner(channel1, heartbeatNamespaceBroker1V2Bundle, broker); - waitUntilNewOwner(channel2, heartbeatNamespaceBroker1V1Bundle, broker); - waitUntilNewOwner(channel2, heartbeatNamespaceBroker1V2Bundle, broker); - waitUntilNewOwner(channel1, heartbeatNamespaceBroker2V1Bundle, lookupServiceAddress2); - waitUntilNewOwner(channel1, heartbeatNamespaceBroker2V2Bundle, lookupServiceAddress2); - waitUntilNewOwner(channel2, heartbeatNamespaceBroker2V1Bundle, lookupServiceAddress2); - waitUntilNewOwner(channel2, heartbeatNamespaceBroker2V2Bundle, lookupServiceAddress2); - // Verify to transfer the ownership to the other broker. channel1.publishUnloadEventAsync(new Unload(broker, bundle1, Optional.of(lookupServiceAddress2))); waitUntilNewOwner(channel1, bundle1, lookupServiceAddress2); @@ -806,16 +775,6 @@ public void handleBrokerDeletionEventTest() waitUntilNewOwner(channel1, bundle2, lookupServiceAddress2); waitUntilNewOwner(channel2, bundle2, lookupServiceAddress2); - waitUntilNewOwner(channel1, heartbeatNamespaceBroker1V1Bundle, null); - waitUntilNewOwner(channel1, heartbeatNamespaceBroker1V2Bundle, null); - waitUntilNewOwner(channel2, heartbeatNamespaceBroker1V1Bundle, null); - waitUntilNewOwner(channel2, heartbeatNamespaceBroker1V2Bundle, null); - - waitUntilNewOwner(channel1, heartbeatNamespaceBroker2V1Bundle, null); - waitUntilNewOwner(channel1, heartbeatNamespaceBroker2V2Bundle, null); - waitUntilNewOwner(channel2, heartbeatNamespaceBroker2V1Bundle, null); - waitUntilNewOwner(channel2, heartbeatNamespaceBroker2V2Bundle, null); - verify(leaderCleanupJobs, times(1)).computeIfAbsent(eq(broker), any()); verify(followerCleanupJobs, times(0)).computeIfAbsent(eq(broker), any()); @@ -827,7 +786,7 @@ public void handleBrokerDeletionEventTest() validateMonitorCounters(leaderChannel, 2, 0, - 7, + 3, 0, 2, 0, @@ -858,7 +817,7 @@ public void handleBrokerDeletionEventTest() validateMonitorCounters(leaderChannel, 2, 0, - 7, + 3, 0, 3, 0, @@ -879,7 +838,7 @@ public void handleBrokerDeletionEventTest() validateMonitorCounters(leaderChannel, 2, 0, - 7, + 3, 0, 3, 0, @@ -901,7 +860,7 @@ public void handleBrokerDeletionEventTest() validateMonitorCounters(leaderChannel, 2, 0, - 7, + 3, 0, 4, 0, @@ -923,7 +882,7 @@ public void handleBrokerDeletionEventTest() validateMonitorCounters(leaderChannel, 3, 0, - 9, + 5, 0, 4, 0, @@ -952,7 +911,7 @@ public void handleBrokerDeletionEventTest() validateMonitorCounters(leaderChannel, 3, 0, - 9, + 5, 0, 4, 1, @@ -1447,7 +1406,7 @@ public void splitAndRetryFailureTest() throws Exception { validateMonitorCounters(leader, 0, - 1, + 3, 1, 0, 0, From ca1ccd9326487f5438692ffea35fc591d62f781b Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 18 Oct 2023 18:21:03 +0300 Subject: [PATCH 120/699] [improve][ci] Add new CI unit test group "Broker Group 4" with cluster migration tests (#21391) (cherry picked from commit e2c6c08b3598e01aba1bac90b56412e451140385) --- .github/workflows/pulsar-ci.yaml | 2 ++ build/run_unit_group.sh | 4 ++++ .../apache/pulsar/broker/service/ClusterMigrationTest.java | 2 +- 3 files changed, 7 insertions(+), 1 deletion(-) diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index b8d498c27be93..f4d258a5b4317 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -190,6 +190,8 @@ jobs: group: BROKER_GROUP_2 - name: Brokers - Broker Group 3 group: BROKER_GROUP_3 + - name: Brokers - Broker Group 4 + group: BROKER_GROUP_4 - name: Brokers - Client Api group: BROKER_CLIENT_API - name: Brokers - Client Impl diff --git a/build/run_unit_group.sh b/build/run_unit_group.sh index f6c212dcdd28f..8597848b69bf8 100755 --- a/build/run_unit_group.sh +++ b/build/run_unit_group.sh @@ -87,6 +87,10 @@ function test_group_broker_group_3() { mvn_test -pl pulsar-broker -Dgroups='broker-admin' } +function test_group_broker_group_4() { + mvn_test -pl pulsar-broker -Dgroups='cluster-migration' +} + function test_group_broker_client_api() { mvn_test -pl pulsar-broker -Dgroups='broker-api' } 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 f6a8a03527eb4..c4f2cd047aff6 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 @@ -50,7 +50,7 @@ import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -@Test(groups = "broker") +@Test(groups = "cluster-migration") public class ClusterMigrationTest { private static final Logger log = LoggerFactory.getLogger(ClusterMigrationTest.class); From a1d3336180aeafd9a3d5adfe5b3474f2510c8d9f Mon Sep 17 00:00:00 2001 From: ken <1647023764@qq.com> Date: Sat, 7 Oct 2023 20:24:45 +0800 Subject: [PATCH 121/699] [fix][broker] rackaware policy is ineffective when delete zk rack info after bkclient initialize (#20944) --- .../BookieRackAffinityMapping.java | 4 ++-- .../BookieRackAffinityMappingTest.java | 17 +++++++++++++++++ 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java index e9e350800b44e..d54ef2a5f4cef 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java @@ -121,8 +121,6 @@ public synchronized void setConf(Configuration conf) { store.registerListener(this::handleUpdates); racksWithHost = bookieMappingCache.get(BOOKIE_INFO_ROOT_PATH).get() .orElseGet(BookiesRackConfiguration::new); - updateRacksWithHost(racksWithHost); - watchAvailableBookies(); for (Map bookieMapping : racksWithHost.values()) { for (String address : bookieMapping.keySet()) { bookieAddressListLastTime.add(BookieId.parse(address)); @@ -132,6 +130,8 @@ public synchronized void setConf(Configuration conf) { bookieAddressListLastTime); } } + updateRacksWithHost(racksWithHost); + watchAvailableBookies(); } catch (InterruptedException | ExecutionException | MetadataException e) { throw new RuntimeException(METADATA_STORE_INSTANCE + " failed to init BookieId list"); } diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMappingTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMappingTest.java index d7be7dabd0db1..d7df5afb4bebe 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMappingTest.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMappingTest.java @@ -254,6 +254,7 @@ public void testWithPulsarRegistrationClient() throws Exception { bkClientConf.getTimeoutTimerNumTicks()); RackawareEnsemblePlacementPolicy repp = new RackawareEnsemblePlacementPolicy(); + mapping.registerRackChangeListener(repp); Class clazz1 = Class.forName("org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy"); Field field1 = clazz1.getDeclaredField("knownBookies"); field1.setAccessible(true); @@ -323,6 +324,22 @@ public void testWithPulsarRegistrationClient() throws Exception { assertEquals(knownBookies.get(BOOKIE2.toBookieId()).getNetworkLocation(), "/rack1"); assertEquals(knownBookies.get(BOOKIE3.toBookieId()).getNetworkLocation(), "/default-rack"); + //remove bookie2 rack, the bookie2 rack should be /default-rack + data = "{\"group1\": {\"" + BOOKIE1 + + "\": {\"rack\": \"/rack0\", \"hostname\": \"bookie1.example.com\"}}}"; + store.put(BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH, data.getBytes(), Optional.empty()).join(); + Awaitility.await().atMost(30, TimeUnit.SECONDS).until(() -> ((BookiesRackConfiguration)field.get(mapping)).get("group1").size() == 1); + + racks = mapping + .resolve(Lists.newArrayList(BOOKIE1.getHostName(), BOOKIE2.getHostName(), BOOKIE3.getHostName())) + .stream().filter(Objects::nonNull).toList(); + assertEquals(racks.size(), 1); + assertEquals(racks.get(0), "/rack0"); + assertEquals(knownBookies.size(), 3); + assertEquals(knownBookies.get(BOOKIE1.toBookieId()).getNetworkLocation(), "/rack0"); + assertEquals(knownBookies.get(BOOKIE2.toBookieId()).getNetworkLocation(), "/default-rack"); + assertEquals(knownBookies.get(BOOKIE3.toBookieId()).getNetworkLocation(), "/default-rack"); + timer.stop(); } } From 40e4dc27cdc6a888d78bdf0bc0ed2ea937da7ad4 Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Tue, 26 Sep 2023 10:33:35 +0800 Subject: [PATCH 122/699] [fix][broker] Fix inconsistent topic policy (#21231) --- .../apache/pulsar/broker/auth/MockedPulsarServiceBaseTest.java | 2 ++ 1 file changed, 2 insertions(+) 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 fa99885488134..034ea673bf0b0 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 @@ -56,6 +56,8 @@ import org.apache.pulsar.client.impl.auth.AuthenticationDisabled; import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.client.impl.ProducerImpl; +import org.apache.pulsar.client.impl.auth.AuthenticationDisabled; +import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TenantInfoImpl; From 795962293d6ac7cba63f22a243d330555fb222be Mon Sep 17 00:00:00 2001 From: ken <1647023764@qq.com> Date: Thu, 19 Oct 2023 10:18:12 +0800 Subject: [PATCH 123/699] [fix][broker] Fix heartbeat namespace create transaction internal topic (#21348) --- .../service/persistent/PersistentTopic.java | 3 ++- .../systopic/PartitionedSystemTopicTest.java | 19 +++++++++++++++++++ 2 files changed, 21 insertions(+), 1 deletion(-) 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 1bfe55cf74c29..a090aa1bf2565 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 @@ -305,7 +305,8 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS TopicName topicName = TopicName.get(topic); if (brokerService.getPulsar().getConfiguration().isTransactionCoordinatorEnabled() - && !isEventSystemTopic(topicName)) { + && !isEventSystemTopic(topicName) + && !NamespaceService.isHeartbeatNamespace(topicName.getNamespaceObject())) { this.transactionBuffer = brokerService.getPulsar() .getTransactionBufferProvider().newTransactionBuffer(this); } else { 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 4af0bd9052391..42d941e616809 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 @@ -81,6 +81,7 @@ protected void setup() throws Exception { conf.setDefaultNumPartitions(PARTITIONS); conf.setManagedLedgerMaxEntriesPerLedger(1); conf.setBrokerDeleteInactiveTopicsEnabled(false); + conf.setTransactionCoordinatorEnabled(true); super.baseSetup(); } @@ -207,6 +208,24 @@ public void testHeartbeatTopicNotAllowedToSendEvent() throws Exception { }); } + @Test + public void testHeartbeatNamespaceNotCreateTransactionInternalTopic() throws Exception { + admin.brokers().healthcheck(TopicVersion.V2); + NamespaceName namespaceName = NamespaceService.getHeartbeatNamespaceV2(pulsar.getLookupServiceAddress(), + pulsar.getConfig()); + TopicName topicName = TopicName.get("persistent", + namespaceName, SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT); + Optional optionalTopic = pulsar.getBrokerService() + .getTopic(topicName.getPartition(1).toString(), false).join(); + Assert.assertTrue(optionalTopic.isEmpty()); + + List topics = getPulsar().getNamespaceService().getListOfPersistentTopics(namespaceName).join(); + Assert.assertEquals(topics.size(), 1); + TopicName heartbeatTopicName = TopicName.get("persistent", + namespaceName, BrokersBase.HEALTH_CHECK_TOPIC_SUFFIX); + Assert.assertEquals(topics.get(0), heartbeatTopicName.toString()); + } + @Test public void testSetBacklogCausedCreatingProducerFailure() throws Exception { final String ns = "prop/ns-test"; From 4cc95c1ec66ff6059b2e0d6ba17b4174e0eaa28b Mon Sep 17 00:00:00 2001 From: ken <1647023764@qq.com> Date: Thu, 19 Oct 2023 18:39:41 +0800 Subject: [PATCH 124/699] [fix][broker] Fix heartbeat namespace create event topic and cannot delete heartbeat topic (#21360) Co-authored-by: fanjianye Co-authored-by: Jiwei Guo --- .../SystemTopicBasedTopicPoliciesService.java | 14 +++++++---- .../systopic/PartitionedSystemTopicTest.java | 23 +++++++++++++++++++ 2 files changed, 33 insertions(+), 4 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 ed76d37ae2536..da31234095446 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 @@ -95,20 +95,23 @@ public SystemTopicBasedTopicPoliciesService(PulsarService pulsarService) { @Override public CompletableFuture deleteTopicPoliciesAsync(TopicName topicName) { + if (NamespaceService.isHeartbeatNamespace(topicName.getNamespaceObject())) { + return CompletableFuture.completedFuture(null); + } return sendTopicPolicyEvent(topicName, ActionType.DELETE, null); } @Override public CompletableFuture updateTopicPoliciesAsync(TopicName topicName, TopicPolicies policies) { + if (NamespaceService.isHeartbeatNamespace(topicName.getNamespaceObject())) { + return CompletableFuture.failedFuture(new BrokerServiceException.NotAllowedException( + "Not allowed to update topic policy for the heartbeat topic")); + } return sendTopicPolicyEvent(topicName, ActionType.UPDATE, policies); } private CompletableFuture sendTopicPolicyEvent(TopicName topicName, ActionType actionType, TopicPolicies policies) { - if (NamespaceService.isHeartbeatNamespace(topicName.getNamespaceObject())) { - return CompletableFuture.failedFuture( - new BrokerServiceException.NotAllowedException("Not allowed to send event to health check topic")); - } return pulsarService.getPulsarResources().getNamespaceResources() .getPoliciesAsync(topicName.getNamespaceObject()) .thenCompose(namespacePolicies -> { @@ -220,6 +223,9 @@ public TopicPolicies getTopicPolicies(TopicName topicName) throws TopicPoliciesC @Override public TopicPolicies getTopicPolicies(TopicName topicName, boolean isGlobal) throws TopicPoliciesCacheNotInitException { + if (NamespaceService.isHeartbeatNamespace(topicName.getNamespaceObject())) { + return null; + } if (!policyCacheInitMap.containsKey(topicName.getNamespaceObject())) { NamespaceName namespace = topicName.getNamespaceObject(); prepareInitPoliciesCacheAsync(namespace); 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 42d941e616809..416d7ed02708e 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 @@ -191,6 +191,13 @@ public void testSystemNamespaceNotCreateChangeEventsTopic() throws Exception { Optional optionalTopic = pulsar.getBrokerService() .getTopic(topicName.getPartition(1).toString(), false).join(); Assert.assertTrue(optionalTopic.isEmpty()); + + TopicName heartbeatTopicName = TopicName.get("persistent", + namespaceName, BrokersBase.HEALTH_CHECK_TOPIC_SUFFIX); + admin.topics().getRetention(heartbeatTopicName.toString()); + optionalTopic = pulsar.getBrokerService() + .getTopic(topicName.getPartition(1).toString(), false).join(); + Assert.assertTrue(optionalTopic.isEmpty()); } @Test @@ -208,6 +215,22 @@ public void testHeartbeatTopicNotAllowedToSendEvent() throws Exception { }); } + @Test + public void testHeartbeatTopicBeDeleted() throws Exception { + admin.brokers().healthcheck(TopicVersion.V2); + NamespaceName namespaceName = NamespaceService.getHeartbeatNamespaceV2(pulsar.getLookupServiceAddress(), + pulsar.getConfig()); + TopicName heartbeatTopicName = TopicName.get("persistent", namespaceName, BrokersBase.HEALTH_CHECK_TOPIC_SUFFIX); + + List topics = getPulsar().getNamespaceService().getListOfPersistentTopics(namespaceName).join(); + Assert.assertEquals(topics.size(), 1); + Assert.assertEquals(topics.get(0), heartbeatTopicName.toString()); + + admin.topics().delete(heartbeatTopicName.toString(), true); + topics = getPulsar().getNamespaceService().getListOfPersistentTopics(namespaceName).join(); + Assert.assertEquals(topics.size(), 0); + } + @Test public void testHeartbeatNamespaceNotCreateTransactionInternalTopic() throws Exception { admin.brokers().healthcheck(TopicVersion.V2); From d36a3a8f6a1a7f082f25465bece7b3dc8597133c Mon Sep 17 00:00:00 2001 From: Yan Zhao Date: Wed, 11 Oct 2023 11:07:50 +0800 Subject: [PATCH 125/699] [fix] [bk-client] Fix bk client MinNumRacksPerWriteQuorum and EnforceMinNumRacksPerWriteQuorum not work problem. (#21327) --- .../broker/BookKeeperClientFactoryImpl.java | 5 ++++- .../BookKeeperClientFactoryImplTest.java | 19 +++++++++++++++++++ 2 files changed, 23 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/BookKeeperClientFactoryImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/BookKeeperClientFactoryImpl.java index 0ecca75595603..e5293cee24e4a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/BookKeeperClientFactoryImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/BookKeeperClientFactoryImpl.java @@ -219,7 +219,7 @@ static void setDefaultEnsemblePlacementPolicy( } } - private void setEnsemblePlacementPolicy(ClientConfiguration bkConf, ServiceConfiguration conf, MetadataStore store, + static void setEnsemblePlacementPolicy(ClientConfiguration bkConf, ServiceConfiguration conf, MetadataStore store, Class policyClass) { bkConf.setEnsemblePlacementPolicy(policyClass); bkConf.setProperty(BookieRackAffinityMapping.METADATA_STORE_INSTANCE, store); @@ -227,6 +227,9 @@ private void setEnsemblePlacementPolicy(ClientConfiguration bkConf, ServiceConfi bkConf.setProperty(REPP_DNS_RESOLVER_CLASS, conf.getProperties().getProperty(REPP_DNS_RESOLVER_CLASS, BookieRackAffinityMapping.class.getName())); + bkConf.setMinNumRacksPerWriteQuorum(conf.getBookkeeperClientMinNumRacksPerWriteQuorum()); + bkConf.setEnforceMinNumRacksPerWriteQuorum(conf.isBookkeeperClientEnforceMinNumRacksPerWriteQuorum()); + bkConf.setProperty(NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY, conf.getProperties().getProperty( NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BookKeeperClientFactoryImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BookKeeperClientFactoryImplTest.java index 0dea84e727a88..3c0e4d0c409df 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BookKeeperClientFactoryImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BookKeeperClientFactoryImplTest.java @@ -41,6 +41,7 @@ import org.apache.pulsar.bookie.rackawareness.BookieRackAffinityMapping; import org.apache.pulsar.metadata.api.MetadataStore; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; +import org.apache.pulsar.zookeeper.ZkIsolatedBookieEnsemblePlacementPolicy; import org.testng.annotations.Test; /** @@ -152,6 +153,24 @@ public void testSetDefaultEnsemblePlacementPolicyRackAwareEnabledChangedValues() assertEquals(20, bkConf.getMinNumRacksPerWriteQuorum()); } + @Test + public void testSetEnsemblePlacementPolicys() { + ClientConfiguration bkConf = new ClientConfiguration(); + ServiceConfiguration conf = new ServiceConfiguration(); + conf.setBookkeeperClientMinNumRacksPerWriteQuorum(3); + conf.setBookkeeperClientEnforceMinNumRacksPerWriteQuorum(true); + + MetadataStore store = mock(MetadataStore.class); + + BookKeeperClientFactoryImpl.setEnsemblePlacementPolicy( + bkConf, + conf, + store, + ZkIsolatedBookieEnsemblePlacementPolicy.class); + assertEquals(bkConf.getMinNumRacksPerWriteQuorum(), 3); + assertTrue(bkConf.getEnforceMinNumRacksPerWriteQuorum()); + } + @Test public void testSetDiskWeightBasedPlacementEnabled() { BookKeeperClientFactoryImpl factory = new BookKeeperClientFactoryImpl(); From 3c3551b75830f8ff00179ecfc6954030ad55f8ca Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Wed, 11 Oct 2023 09:22:44 -0700 Subject: [PATCH 126/699] [improve][broker] use ConcurrentHashMap in ServiceUnitStateChannel and avoid recursive update error (#21282) (cherry picked from commit aecdb03e0e64605d60f03d9b76f99c1136677dff) --- .../channel/ServiceUnitStateChannelImpl.java | 90 +++++++++++-------- .../channel/ServiceUnitStateChannelTest.java | 10 +-- 2 files changed, 59 insertions(+), 41 deletions(-) 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 d71513652e9b8..f7e09a2bec546 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 @@ -54,6 +54,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledFuture; @@ -67,6 +68,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.mutable.MutableInt; +import org.apache.commons.lang3.mutable.MutableObject; import org.apache.pulsar.PulsarClusterMetadataSetup; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; @@ -97,7 +99,6 @@ import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.topics.TopicCompactionStrategy; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.NotificationType; import org.apache.pulsar.metadata.api.extended.SessionEvent; @@ -125,9 +126,9 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { private final PulsarService pulsar; private final ServiceConfiguration config; private final Schema schema; - private final ConcurrentOpenHashMap> getOwnerRequests; + private final Map> getOwnerRequests; private final String lookupServiceAddress; - private final ConcurrentOpenHashMap> cleanupJobs; + private final Map> cleanupJobs; private final StateChangeListeners stateChangeListeners; private ExtensibleLoadManagerImpl loadManager; private BrokerRegistry brokerRegistry; @@ -204,9 +205,8 @@ public ServiceUnitStateChannelImpl(PulsarService pulsar) { this.config = pulsar.getConfig(); this.lookupServiceAddress = pulsar.getLookupServiceAddress(); this.schema = Schema.JSON(ServiceUnitStateData.class); - this.getOwnerRequests = ConcurrentOpenHashMap.>newBuilder().build(); - this.cleanupJobs = ConcurrentOpenHashMap.>newBuilder().build(); + this.getOwnerRequests = new ConcurrentHashMap<>(); + this.cleanupJobs = new ConcurrentHashMap<>(); this.stateChangeListeners = new StateChangeListeners(); this.semiTerminalStateWaitingTimeInMillis = config.getLoadBalancerServiceUnitStateTombstoneDelayTimeInSeconds() * 1000; @@ -826,20 +826,28 @@ private boolean isTargetBroker(String broker) { } private CompletableFuture deferGetOwnerRequest(String serviceUnit) { - return getOwnerRequests - .computeIfAbsent(serviceUnit, k -> { - CompletableFuture future = new CompletableFuture<>(); - future.orTimeout(inFlightStateWaitingTimeInMillis, TimeUnit.MILLISECONDS) - .whenComplete((v, e) -> { - if (e != null) { - getOwnerRequests.remove(serviceUnit, future); - log.warn("Failed to getOwner for serviceUnit:{}", - serviceUnit, e); - } + var requested = new MutableObject>(); + try { + return getOwnerRequests + .computeIfAbsent(serviceUnit, k -> { + CompletableFuture future = new CompletableFuture<>(); + requested.setValue(future); + return future; + }); + } finally { + var future = requested.getValue(); + if (future != null) { + future.orTimeout(inFlightStateWaitingTimeInMillis, TimeUnit.MILLISECONDS) + .whenComplete((v, e) -> { + if (e != null) { + getOwnerRequests.remove(serviceUnit, future); + log.warn("Failed to getOwner for serviceUnit:{}", + serviceUnit, e); } - ); - return future; - }); + } + ); + } + } } private CompletableFuture closeServiceUnit(String serviceUnit) { @@ -1114,24 +1122,34 @@ private void handleBrokerDeletionEvent(String broker) { } private void scheduleCleanup(String broker, long delayInSecs) { - cleanupJobs.computeIfAbsent(broker, k -> { - Executor delayed = CompletableFuture - .delayedExecutor(delayInSecs, TimeUnit.SECONDS, pulsar.getLoadManagerExecutor()); - totalInactiveBrokerCleanupScheduledCnt++; - return CompletableFuture - .runAsync(() -> { - try { - doCleanup(broker); - } catch (Throwable e) { - log.error("Failed to run the cleanup job for the broker {}, " - + "totalCleanupErrorCnt:{}.", - broker, totalCleanupErrorCnt.incrementAndGet(), e); - } finally { - cleanupJobs.remove(broker); + var scheduled = new MutableObject>(); + try { + cleanupJobs.computeIfAbsent(broker, k -> { + Executor delayed = CompletableFuture + .delayedExecutor(delayInSecs, TimeUnit.SECONDS, pulsar.getLoadManagerExecutor()); + totalInactiveBrokerCleanupScheduledCnt++; + var future = CompletableFuture + .runAsync(() -> { + try { + doCleanup(broker); + } catch (Throwable e) { + log.error("Failed to run the cleanup job for the broker {}, " + + "totalCleanupErrorCnt:{}.", + broker, totalCleanupErrorCnt.incrementAndGet(), e); + } } - } - , delayed); - }); + , delayed); + scheduled.setValue(future); + return future; + }); + } finally { + var future = scheduled.getValue(); + if (future != null) { + future.whenComplete((v, ex) -> { + cleanupJobs.remove(broker); + }); + } + } log.info("Scheduled ownership cleanup for broker:{} with delay:{} secs. Pending clean jobs:{}.", broker, delayInSecs, cleanupJobs.size()); 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 a226df53e12f3..f9893ea3f63dc 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 @@ -60,6 +60,7 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -88,7 +89,6 @@ import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.client.impl.TableViewImpl; import org.apache.pulsar.common.policies.data.TopicType; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.metadata.api.NotificationType; import org.apache.pulsar.metadata.api.coordination.LeaderElectionState; @@ -1558,9 +1558,9 @@ public void testOverrideOrphanStateData() } - private static ConcurrentOpenHashMap>> getOwnerRequests( + private static ConcurrentHashMap>> getOwnerRequests( ServiceUnitStateChannel channel) throws IllegalAccessException { - return (ConcurrentOpenHashMap>>) + return (ConcurrentHashMap>>) FieldUtils.readDeclaredField(channel, "getOwnerRequests", true); } @@ -1577,9 +1577,9 @@ private static long getLastMetadataSessionEventTimestamp(ServiceUnitStateChannel FieldUtils.readField(channel, "lastMetadataSessionEventTimestamp", true); } - private static ConcurrentOpenHashMap> getCleanupJobs( + private static ConcurrentHashMap> getCleanupJobs( ServiceUnitStateChannel channel) throws IllegalAccessException { - return (ConcurrentOpenHashMap>) + return (ConcurrentHashMap>) FieldUtils.readField(channel, "cleanupJobs", true); } From 076e90db066ec5bd4d4e99ff8b0ab361489e8c55 Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Thu, 19 Oct 2023 07:08:49 -0500 Subject: [PATCH 127/699] [fix][broker] Fix unload operation stuck when use ExtensibleLoadManager (#21332) (cherry picked from commit ecd40e43a6b90b58e209bc9bced84b35d933619e) --- .../extensions/ExtensibleLoadManagerImpl.java | 2 +- .../channel/ServiceUnitStateChannelImpl.java | 17 +- .../extensions/manager/UnloadManager.java | 7 + .../broker/namespace/NamespaceService.java | 4 - .../pulsar/broker/service/BrokerService.java | 15 + .../ExtensibleLoadManagerImplTest.java | 309 ++++++++++-------- .../channel/ServiceUnitStateChannelTest.java | 12 +- 7 files changed, 218 insertions(+), 148 deletions(-) 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 85baf9ec4fbdf..d3119365ddfea 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 @@ -304,7 +304,7 @@ public void start() throws PulsarServerException { } }); }); - this.serviceUnitStateChannel = new ServiceUnitStateChannelImpl(pulsar); + this.serviceUnitStateChannel = ServiceUnitStateChannelImpl.newInstance(pulsar); this.brokerRegistry.start(); this.splitManager = new SplitManager(splitCounter); this.unloadManager = new UnloadManager(unloadCounter); 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 f7e09a2bec546..3cf16709cde1b 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 @@ -200,7 +200,18 @@ enum MetadataState { Unstable } + public static ServiceUnitStateChannelImpl newInstance(PulsarService pulsar) { + return new ServiceUnitStateChannelImpl(pulsar); + } + public ServiceUnitStateChannelImpl(PulsarService pulsar) { + this(pulsar, MAX_IN_FLIGHT_STATE_WAITING_TIME_IN_MILLIS, OWNERSHIP_MONITOR_DELAY_TIME_IN_SECS); + } + + @VisibleForTesting + public ServiceUnitStateChannelImpl(PulsarService pulsar, + long inFlightStateWaitingTimeInMillis, + long ownershipMonitorDelayTimeInSecs) { this.pulsar = pulsar; this.config = pulsar.getConfig(); this.lookupServiceAddress = pulsar.getLookupServiceAddress(); @@ -210,8 +221,8 @@ public ServiceUnitStateChannelImpl(PulsarService pulsar) { this.stateChangeListeners = new StateChangeListeners(); this.semiTerminalStateWaitingTimeInMillis = config.getLoadBalancerServiceUnitStateTombstoneDelayTimeInSeconds() * 1000; - this.inFlightStateWaitingTimeInMillis = MAX_IN_FLIGHT_STATE_WAITING_TIME_IN_MILLIS; - this.ownershipMonitorDelayTimeInSecs = OWNERSHIP_MONITOR_DELAY_TIME_IN_SECS; + this.inFlightStateWaitingTimeInMillis = inFlightStateWaitingTimeInMillis; + this.ownershipMonitorDelayTimeInSecs = ownershipMonitorDelayTimeInSecs; if (semiTerminalStateWaitingTimeInMillis < inFlightStateWaitingTimeInMillis) { throw new IllegalArgumentException( "Invalid Config: loadBalancerServiceUnitStateCleanUpDelayTimeInSeconds < " @@ -837,7 +848,7 @@ private CompletableFuture deferGetOwnerRequest(String serviceUnit) { } finally { var future = requested.getValue(); if (future != null) { - future.orTimeout(inFlightStateWaitingTimeInMillis, TimeUnit.MILLISECONDS) + future.orTimeout(inFlightStateWaitingTimeInMillis + 5 * 1000, TimeUnit.MILLISECONDS) .whenComplete((v, e) -> { if (e != null) { getOwnerRequests.remove(serviceUnit, future); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java index 2dde0c4708e41..ffdbbc2af4219 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java @@ -88,6 +88,13 @@ public CompletableFuture waitAsync(CompletableFuture eventPubFuture, @Override public void handleEvent(String serviceUnit, ServiceUnitStateData data, Throwable t) { + if (t != null && inFlightUnloadRequest.containsKey(serviceUnit)) { + if (log.isDebugEnabled()) { + log.debug("Handling {} for service unit {} with exception.", data, serviceUnit, t); + } + this.complete(serviceUnit, t); + return; + } ServiceUnitState state = ServiceUnitStateData.state(data); switch (state) { case Free, Owned -> this.complete(serviceUnit, t); 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 57c0cc7c0464b..c69e30173aacb 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 @@ -1558,10 +1558,6 @@ public boolean checkOwnershipPresent(NamespaceBundle bundle) throws Exception { public CompletableFuture checkOwnershipPresentAsync(NamespaceBundle bundle) { if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { - if (bundle.getNamespaceObject().equals(SYSTEM_NAMESPACE)) { - return FutureUtil.failedFuture(new UnsupportedOperationException( - "Ownership check for system namespace is not supported")); - } ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); return extensibleLoadManager.getOwnershipAsync(Optional.empty(), bundle) .thenApply(Optional::isPresent); 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 05d6bca962b1d..4fe4b6dcd5bea 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 @@ -2197,6 +2197,21 @@ private CompletableFuture unloadServiceUnit(NamespaceBundle serviceUnit if (serviceUnit.includes(topicName)) { // Topic needs to be unloaded log.info("[{}] Unloading topic", topicName); + if (topicFuture.isCompletedExceptionally()) { + try { + topicFuture.get(); + } catch (InterruptedException | ExecutionException ex) { + if (ex.getCause() instanceof ServiceUnitNotReadyException) { + // Topic was already unloaded + if (log.isDebugEnabled()) { + log.debug("[{}] Topic was already unloaded", topicName); + } + return; + } else { + log.warn("[{}] Got exception when closing topic", topicName, ex); + } + } + } closeFutures.add(topicFuture .thenCompose(t -> t.isPresent() ? t.get().close(closeWithoutWaitingClientDisconnect) : CompletableFuture.completedFuture(null))); 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 011e7174cbec2..20ba9500cb1fd 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 @@ -38,9 +38,11 @@ import static org.apache.pulsar.broker.namespace.NamespaceService.getHeartbeatNamespace; import static org.apache.pulsar.broker.namespace.NamespaceService.getHeartbeatNamespaceV2; import static org.apache.pulsar.broker.namespace.NamespaceService.getSLAMonitorNamespace; +import static org.mockito.ArgumentMatchers.isA; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; @@ -113,6 +115,7 @@ import org.apache.pulsar.policies.data.loadbalancer.ResourceUsage; import org.apache.pulsar.policies.data.loadbalancer.SystemResourceUsage; import org.awaitility.Awaitility; +import org.mockito.MockedStatic; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; @@ -142,46 +145,56 @@ public class ExtensibleLoadManagerImplTest extends MockedPulsarServiceBaseTest { @BeforeClass @Override public void setup() throws Exception { - conf.setForceDeleteNamespaceAllowed(true); - conf.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); - conf.setAllowAutoTopicCreation(true); - conf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); - conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); - conf.setLoadBalancerSheddingEnabled(false); - conf.setLoadBalancerDebugModeEnabled(true); - conf.setTopicLevelPoliciesEnabled(false); - super.internalSetup(conf); - pulsar1 = pulsar; - ServiceConfiguration defaultConf = getDefaultConf(); - defaultConf.setAllowAutoTopicCreation(true); - defaultConf.setForceDeleteNamespaceAllowed(true); - defaultConf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); - defaultConf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); - defaultConf.setLoadBalancerSheddingEnabled(false); - defaultConf.setTopicLevelPoliciesEnabled(false); - additionalPulsarTestContext = createAdditionalPulsarTestContext(defaultConf); - pulsar2 = additionalPulsarTestContext.getPulsarService(); - - setPrimaryLoadManager(); - - setSecondaryLoadManager(); - - admin.clusters().createCluster(this.conf.getClusterName(), - ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); - admin.tenants().createTenant("public", - new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), - Sets.newHashSet(this.conf.getClusterName()))); - admin.namespaces().createNamespace("public/default"); - admin.namespaces().setNamespaceReplicationClusters("public/default", - Sets.newHashSet(this.conf.getClusterName())); - - admin.namespaces().createNamespace(defaultTestNamespace); - admin.namespaces().setNamespaceReplicationClusters(defaultTestNamespace, - Sets.newHashSet(this.conf.getClusterName())); + try (MockedStatic channelMockedStatic = + mockStatic(ServiceUnitStateChannelImpl.class)) { + channelMockedStatic.when(() -> ServiceUnitStateChannelImpl.newInstance(isA(PulsarService.class))) + .thenAnswer(invocation -> { + PulsarService pulsarService = invocation.getArgument(0); + // Set the inflight state waiting time and ownership monitor delay time to 5 seconds to avoid + // stuck when doing unload. + return new ServiceUnitStateChannelImpl(pulsarService, 5 * 1000, 1); + }); + conf.setForceDeleteNamespaceAllowed(true); + conf.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); + conf.setAllowAutoTopicCreation(true); + conf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); + conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); + conf.setLoadBalancerSheddingEnabled(false); + conf.setLoadBalancerDebugModeEnabled(true); + conf.setTopicLevelPoliciesEnabled(true); + super.internalSetup(conf); + pulsar1 = pulsar; + ServiceConfiguration defaultConf = getDefaultConf(); + defaultConf.setAllowAutoTopicCreation(true); + defaultConf.setForceDeleteNamespaceAllowed(true); + defaultConf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); + defaultConf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); + defaultConf.setLoadBalancerSheddingEnabled(false); + defaultConf.setTopicLevelPoliciesEnabled(true); + additionalPulsarTestContext = createAdditionalPulsarTestContext(defaultConf); + pulsar2 = additionalPulsarTestContext.getPulsarService(); + + setPrimaryLoadManager(); + + setSecondaryLoadManager(); + + admin.clusters().createCluster(this.conf.getClusterName(), + ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); + admin.tenants().createTenant("public", + new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), + Sets.newHashSet(this.conf.getClusterName()))); + admin.namespaces().createNamespace("public/default"); + admin.namespaces().setNamespaceReplicationClusters("public/default", + Sets.newHashSet(this.conf.getClusterName())); + + admin.namespaces().createNamespace(defaultTestNamespace); + admin.namespaces().setNamespaceReplicationClusters(defaultTestNamespace, + Sets.newHashSet(this.conf.getClusterName())); + } } @Override - @AfterClass + @AfterClass(alwaysRun = true) protected void cleanup() throws Exception { pulsar1 = null; pulsar2.close(); @@ -557,119 +570,134 @@ public CompletableFuture> filterAsync(Map webServiceUrl1 = - pulsar1.getNamespaceService().getWebServiceUrl(bundle, options); - assertTrue(webServiceUrl1.isPresent()); - assertEquals(webServiceUrl1.get().toString(), pulsar3.getWebServiceAddress()); - - Optional webServiceUrl2 = - pulsar2.getNamespaceService().getWebServiceUrl(bundle, options); - assertTrue(webServiceUrl2.isPresent()); - assertEquals(webServiceUrl2.get().toString(), webServiceUrl1.get().toString()); - - Optional webServiceUrl3 = - pulsar3.getNamespaceService().getWebServiceUrl(bundle, options); - assertTrue(webServiceUrl3.isPresent()); - assertEquals(webServiceUrl3.get().toString(), webServiceUrl1.get().toString()); - - List pulsarServices = List.of(pulsar1, pulsar2, pulsar3); - for (PulsarService pulsarService : pulsarServices) { - // Test lookup heartbeat namespace's topic - for (PulsarService pulsar : pulsarServices) { - assertLookupHeartbeatOwner(pulsarService, pulsar.getLookupServiceAddress(), pulsar.getBrokerServiceUrl()); - } - // Test lookup SLA namespace's topic - for (PulsarService pulsar : pulsarServices) { - assertLookupSLANamespaceOwner(pulsarService, pulsar.getLookupServiceAddress(), pulsar.getBrokerServiceUrl()); - } - } - - // Test deploy new broker with new load manager - ServiceConfiguration conf = getDefaultConf(); - conf.setAllowAutoTopicCreation(true); - conf.setForceDeleteNamespaceAllowed(true); - conf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); - conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); - try (var additionPulsarTestContext = createAdditionalPulsarTestContext(conf)) { - var pulsar4 = additionPulsarTestContext.getPulsarService(); - - Set availableCandidates = Sets.newHashSet(pulsar1.getBrokerServiceUrl(), - pulsar2.getBrokerServiceUrl(), - pulsar4.getBrokerServiceUrl()); - String lookupResult4 = pulsar4.getAdminClient().lookups().lookupTopic(topic); - assertTrue(availableCandidates.contains(lookupResult4)); - - String lookupResult5 = pulsar1.getAdminClient().lookups().lookupTopic(topic); - String lookupResult6 = pulsar2.getAdminClient().lookups().lookupTopic(topic); - String lookupResult7 = pulsar3.getAdminClient().lookups().lookupTopic(topic); - assertEquals(lookupResult4, lookupResult5); - assertEquals(lookupResult4, lookupResult6); - assertEquals(lookupResult4, lookupResult7); - - Set availableWebUrlCandidates = Sets.newHashSet(pulsar1.getWebServiceAddress(), - pulsar2.getWebServiceAddress(), - pulsar4.getWebServiceAddress()); - - webServiceUrl1 = + try (MockedStatic channelMockedStatic = + mockStatic(ServiceUnitStateChannelImpl.class)) { + channelMockedStatic.when(() -> ServiceUnitStateChannelImpl.newInstance(isA(PulsarService.class))) + .thenAnswer(invocation -> { + PulsarService pulsarService = invocation.getArgument(0); + // Set the inflight state waiting time and ownership monitor delay time to 5 seconds to avoid + // stuck when doing unload. + return new ServiceUnitStateChannelImpl(pulsarService, 5 * 1000, 1); + }); + // Test rollback to modular load manager. + ServiceConfiguration defaultConf = getDefaultConf(); + defaultConf.setAllowAutoTopicCreation(true); + defaultConf.setForceDeleteNamespaceAllowed(true); + defaultConf.setLoadManagerClassName(ModularLoadManagerImpl.class.getName()); + defaultConf.setLoadBalancerSheddingEnabled(false); + try (var additionalPulsarTestContext = createAdditionalPulsarTestContext(defaultConf)) { + // start pulsar3 with old load manager + var pulsar3 = additionalPulsarTestContext.getPulsarService(); + String topic = "persistent://" + defaultTestNamespace + "/test"; + + String lookupResult1 = pulsar3.getAdminClient().lookups().lookupTopic(topic); + assertEquals(lookupResult1, pulsar3.getBrokerServiceUrl()); + + String lookupResult2 = pulsar1.getAdminClient().lookups().lookupTopic(topic); + String lookupResult3 = pulsar2.getAdminClient().lookups().lookupTopic(topic); + assertEquals(lookupResult1, lookupResult2); + assertEquals(lookupResult1, lookupResult3); + + NamespaceBundle bundle = getBundleAsync(pulsar1, TopicName.get(topic)).get(); + LookupOptions options = LookupOptions.builder() + .authoritative(false) + .requestHttps(false) + .readOnly(false) + .loadTopicsInBundle(false).build(); + Optional webServiceUrl1 = pulsar1.getNamespaceService().getWebServiceUrl(bundle, options); assertTrue(webServiceUrl1.isPresent()); - assertTrue(availableWebUrlCandidates.contains(webServiceUrl1.get().toString())); + assertEquals(webServiceUrl1.get().toString(), pulsar3.getWebServiceAddress()); - webServiceUrl2 = + Optional webServiceUrl2 = pulsar2.getNamespaceService().getWebServiceUrl(bundle, options); assertTrue(webServiceUrl2.isPresent()); assertEquals(webServiceUrl2.get().toString(), webServiceUrl1.get().toString()); - // The pulsar3 will redirect to pulsar4 - webServiceUrl3 = + Optional webServiceUrl3 = pulsar3.getNamespaceService().getWebServiceUrl(bundle, options); assertTrue(webServiceUrl3.isPresent()); - // It will redirect to pulsar4 - assertTrue(availableWebUrlCandidates.contains(webServiceUrl3.get().toString())); - - var webServiceUrl4 = - pulsar4.getNamespaceService().getWebServiceUrl(bundle, options); - assertTrue(webServiceUrl4.isPresent()); - assertEquals(webServiceUrl4.get().toString(), webServiceUrl1.get().toString()); + assertEquals(webServiceUrl3.get().toString(), webServiceUrl1.get().toString()); - pulsarServices = List.of(pulsar1, pulsar2, pulsar3, pulsar4); + List pulsarServices = List.of(pulsar1, pulsar2, pulsar3); for (PulsarService pulsarService : pulsarServices) { // Test lookup heartbeat namespace's topic for (PulsarService pulsar : pulsarServices) { - assertLookupHeartbeatOwner(pulsarService, pulsar.getLookupServiceAddress(), pulsar.getBrokerServiceUrl()); + assertLookupHeartbeatOwner(pulsarService, + pulsar.getLookupServiceAddress(), pulsar.getBrokerServiceUrl()); } // Test lookup SLA namespace's topic for (PulsarService pulsar : pulsarServices) { - assertLookupSLANamespaceOwner(pulsarService, pulsar.getLookupServiceAddress(), pulsar.getBrokerServiceUrl()); + assertLookupSLANamespaceOwner(pulsarService, + pulsar.getLookupServiceAddress(), pulsar.getBrokerServiceUrl()); + } + } + + // Test deploy new broker with new load manager + ServiceConfiguration conf = getDefaultConf(); + conf.setAllowAutoTopicCreation(true); + conf.setForceDeleteNamespaceAllowed(true); + conf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); + conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); + try (var additionPulsarTestContext = createAdditionalPulsarTestContext(conf)) { + var pulsar4 = additionPulsarTestContext.getPulsarService(); + + Set availableCandidates = Sets.newHashSet(pulsar1.getBrokerServiceUrl(), + pulsar2.getBrokerServiceUrl(), + pulsar4.getBrokerServiceUrl()); + String lookupResult4 = pulsar4.getAdminClient().lookups().lookupTopic(topic); + assertTrue(availableCandidates.contains(lookupResult4)); + + String lookupResult5 = pulsar1.getAdminClient().lookups().lookupTopic(topic); + String lookupResult6 = pulsar2.getAdminClient().lookups().lookupTopic(topic); + String lookupResult7 = pulsar3.getAdminClient().lookups().lookupTopic(topic); + assertEquals(lookupResult4, lookupResult5); + assertEquals(lookupResult4, lookupResult6); + assertEquals(lookupResult4, lookupResult7); + + Set availableWebUrlCandidates = Sets.newHashSet(pulsar1.getWebServiceAddress(), + pulsar2.getWebServiceAddress(), + pulsar4.getWebServiceAddress()); + + webServiceUrl1 = + pulsar1.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl1.isPresent()); + assertTrue(availableWebUrlCandidates.contains(webServiceUrl1.get().toString())); + + webServiceUrl2 = + pulsar2.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl2.isPresent()); + assertEquals(webServiceUrl2.get().toString(), webServiceUrl1.get().toString()); + + // The pulsar3 will redirect to pulsar4 + webServiceUrl3 = + pulsar3.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl3.isPresent()); + // It will redirect to pulsar4 + assertTrue(availableWebUrlCandidates.contains(webServiceUrl3.get().toString())); + + var webServiceUrl4 = + pulsar4.getNamespaceService().getWebServiceUrl(bundle, options); + assertTrue(webServiceUrl4.isPresent()); + assertEquals(webServiceUrl4.get().toString(), webServiceUrl1.get().toString()); + + pulsarServices = List.of(pulsar1, pulsar2, pulsar3, pulsar4); + for (PulsarService pulsarService : pulsarServices) { + // Test lookup heartbeat namespace's topic + for (PulsarService pulsar : pulsarServices) { + assertLookupHeartbeatOwner(pulsarService, + pulsar.getLookupServiceAddress(), pulsar.getBrokerServiceUrl()); + } + // Test lookup SLA namespace's topic + for (PulsarService pulsar : pulsarServices) { + assertLookupSLANamespaceOwner(pulsarService, + pulsar.getLookupServiceAddress(), pulsar.getBrokerServiceUrl()); + } } } } } + } private void assertLookupHeartbeatOwner(PulsarService pulsar, @@ -1108,6 +1136,12 @@ public void testGetOwnedServiceUnitsAndGetOwnedNamespaceStatus() throws Exceptio NamespaceName heartbeatNamespacePulsar2V2 = NamespaceService.getHeartbeatNamespaceV2(pulsar2.getLookupServiceAddress(), pulsar2.getConfiguration()); + NamespaceName slaMonitorNamespacePulsar1 = + getSLAMonitorNamespace(pulsar1.getLookupServiceAddress(), pulsar1.getConfiguration()); + + NamespaceName slaMonitorNamespacePulsar2 = + getSLAMonitorNamespace(pulsar2.getLookupServiceAddress(), pulsar2.getConfiguration()); + NamespaceBundle bundle1 = pulsar1.getNamespaceService().getNamespaceBundleFactory() .getFullBundle(heartbeatNamespacePulsar1V1); NamespaceBundle bundle2 = pulsar1.getNamespaceService().getNamespaceBundleFactory() @@ -1118,27 +1152,34 @@ public void testGetOwnedServiceUnitsAndGetOwnedNamespaceStatus() throws Exceptio NamespaceBundle bundle4 = pulsar2.getNamespaceService().getNamespaceBundleFactory() .getFullBundle(heartbeatNamespacePulsar2V2); + NamespaceBundle slaBundle1 = pulsar1.getNamespaceService().getNamespaceBundleFactory() + .getFullBundle(slaMonitorNamespacePulsar1); + NamespaceBundle slaBundle2 = pulsar2.getNamespaceService().getNamespaceBundleFactory() + .getFullBundle(slaMonitorNamespacePulsar2); + + Set ownedServiceUnitsByPulsar1 = primaryLoadManager.getOwnedServiceUnits(); log.info("Owned service units: {}", ownedServiceUnitsByPulsar1); // heartbeat namespace bundle will own by pulsar1 - assertEquals(ownedServiceUnitsByPulsar1.size(), 3); assertTrue(ownedServiceUnitsByPulsar1.contains(bundle1)); assertTrue(ownedServiceUnitsByPulsar1.contains(bundle2)); + assertTrue(ownedServiceUnitsByPulsar1.contains(slaBundle1)); Set ownedServiceUnitsByPulsar2 = secondaryLoadManager.getOwnedServiceUnits(); log.info("Owned service units: {}", ownedServiceUnitsByPulsar2); - assertEquals(ownedServiceUnitsByPulsar2.size(), 3); assertTrue(ownedServiceUnitsByPulsar2.contains(bundle3)); assertTrue(ownedServiceUnitsByPulsar2.contains(bundle4)); + assertTrue(ownedServiceUnitsByPulsar2.contains(slaBundle2)); Map ownedNamespacesByPulsar1 = admin.brokers().getOwnedNamespaces(conf.getClusterName(), pulsar1.getLookupServiceAddress()); Map ownedNamespacesByPulsar2 = admin.brokers().getOwnedNamespaces(conf.getClusterName(), pulsar2.getLookupServiceAddress()); - assertEquals(ownedNamespacesByPulsar1.size(), 3); assertTrue(ownedNamespacesByPulsar1.containsKey(bundle1.toString())); assertTrue(ownedNamespacesByPulsar1.containsKey(bundle2.toString())); - assertEquals(ownedNamespacesByPulsar2.size(), 3); + assertTrue(ownedNamespacesByPulsar1.containsKey(slaBundle1.toString())); + assertTrue(ownedNamespacesByPulsar2.containsKey(bundle3.toString())); assertTrue(ownedNamespacesByPulsar2.containsKey(bundle4.toString())); + assertTrue(ownedNamespacesByPulsar2.containsKey(slaBundle2.toString())); String topic = "persistent://" + defaultTestNamespace + "/test-get-owned-service-units"; admin.topics().createPartitionedTopic(topic, 1); 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 f9893ea3f63dc..1da7059bf49ce 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 @@ -507,10 +507,10 @@ public void transferTestWhenDestBrokerFails() assertEquals(1, getOwnerRequests1.size()); assertEquals(1, getOwnerRequests2.size()); - // In 5 secs, the getOwnerAsync requests(lookup requests) should time out. - Awaitility.await().atMost(5, TimeUnit.SECONDS) + // In 10 secs, the getOwnerAsync requests(lookup requests) should time out. + Awaitility.await().atMost(10, TimeUnit.SECONDS) .untilAsserted(() -> assertTrue(owner1.isCompletedExceptionally())); - Awaitility.await().atMost(5, TimeUnit.SECONDS) + Awaitility.await().atMost(10, TimeUnit.SECONDS) .untilAsserted(() -> assertTrue(owner2.isCompletedExceptionally())); assertEquals(0, getOwnerRequests1.size()); @@ -1139,10 +1139,10 @@ public void assignTestWhenDestBrokerProducerFails() assertFalse(owner1.isDone()); assertFalse(owner2.isDone()); - // In 5 secs, the getOwnerAsync requests(lookup requests) should time out. - Awaitility.await().atMost(5, TimeUnit.SECONDS) + // In 10 secs, the getOwnerAsync requests(lookup requests) should time out. + Awaitility.await().atMost(10, TimeUnit.SECONDS) .untilAsserted(() -> assertTrue(owner1.isCompletedExceptionally())); - Awaitility.await().atMost(5, TimeUnit.SECONDS) + Awaitility.await().atMost(10, TimeUnit.SECONDS) .untilAsserted(() -> assertTrue(owner2.isCompletedExceptionally())); // recovered, check the monitor update state : Assigned -> Owned From d9b28ad645b8ff26ecff1e210a2083082d0898f4 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 3 Oct 2023 14:06:05 +0300 Subject: [PATCH 128/699] [fix][sec] Upgrade snappy-java to 1.1.10.5 (#21280) (cherry picked from commit 643428bb295ee4781e86aa2c72e5ad5d61b98870) --- distribution/server/src/assemble/LICENSE.bin.txt | 2 +- pom.xml | 2 +- pulsar-sql/presto-distribution/LICENSE | 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 d32c44a6651c5..979c23bdf2481 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -484,7 +484,7 @@ The Apache Software License, Version 2.0 - org.apache.zookeeper-zookeeper-jute-3.8.1.jar - org.apache.zookeeper-zookeeper-prometheus-metrics-3.8.1.jar * Snappy Java - - org.xerial.snappy-snappy-java-1.1.10.1.jar + - org.xerial.snappy-snappy-java-1.1.10.5.jar * Google HTTP Client - com.google.http-client-google-http-client-gson-1.41.0.jar - com.google.http-client-google-http-client-1.41.0.jar diff --git a/pom.xml b/pom.xml index fc003333e2e82..badcced8b86a0 100644 --- a/pom.xml +++ b/pom.xml @@ -137,7 +137,7 @@ flexible messaging model and an intuitive client API. 3.8.1 1.5.0 1.10.0 - 1.1.10.1 + 1.1.10.5 4.1.12.1 5.1.0 4.1.94.Final diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 61332595c65b8..b9fc58d336cee 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -458,7 +458,7 @@ The Apache Software License, Version 2.0 * JSON Simple - json-simple-1.1.1.jar * Snappy - - snappy-java-1.1.10.1.jar + - snappy-java-1.1.10.5.jar * Jackson - jackson-module-parameter-names-2.14.2.jar * Java Assist From b049adcb31b76864174a92b87d458caef3f9fdf0 Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Tue, 24 Oct 2023 19:08:21 +0800 Subject: [PATCH 129/699] [fix][proxy] Move status endpoint out of auth coverage (#21428) (cherry picked from commit fe2d61d5a44344042ec1994d0943cfc7977fbdcd) --- .../proxy/server/ProxyServiceStarter.java | 6 +++-- .../apache/pulsar/proxy/server/WebServer.java | 27 ++++++++++++++++++- .../server/ProxyWithJwtAuthorizationTest.java | 24 +++++++++++++++++ 3 files changed, 54 insertions(+), 3 deletions(-) diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java index ee8f648182dac..7427331641318 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java @@ -253,9 +253,11 @@ public static void addWebServerHandlers(WebServer server, ProxyConfiguration config, ProxyService service, BrokerDiscoveryProvider discoveryProvider) throws Exception { + // We can make 'status.html' publicly accessible without authentication since + // it does not contain any sensitive data. + server.addRestResource("/", VipStatus.ATTRIBUTE_STATUS_FILE_PATH, config.getStatusFilePath(), + VipStatus.class, false); if (config.isEnableProxyStatsEndpoints()) { - server.addRestResource("/", VipStatus.ATTRIBUTE_STATUS_FILE_PATH, config.getStatusFilePath(), - VipStatus.class); server.addRestResource("/proxy-stats", ProxyStats.ATTRIBUTE_PULSAR_PROXY_NAME, service, ProxyStats.class); if (service != null) { 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 edbcfe0847c4e..b95bbcab08b11 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 @@ -239,7 +239,31 @@ private static void popularServletParams(ServletHolder servletHolder, ProxyConfi } } + /** + * Add a REST resource to the servlet context with authentication coverage. + * + * @see WebServer#addRestResource(String, String, Object, Class, boolean) + * + * @param basePath The base path for the resource. + * @param attribute An attribute associated with the resource. + * @param attributeValue The value of the attribute. + * @param resourceClass The class representing the resource. + */ public void addRestResource(String basePath, String attribute, Object attributeValue, Class resourceClass) { + addRestResource(basePath, attribute, attributeValue, resourceClass, true); + } + + /** + * Add a REST resource to the servlet context. + * + * @param basePath The base path for the resource. + * @param attribute An attribute associated with the resource. + * @param attributeValue The value of the attribute. + * @param resourceClass The class representing the resource. + * @param requireAuthentication A boolean indicating whether authentication is required for this resource. + */ + public void addRestResource(String basePath, String attribute, Object attributeValue, + Class resourceClass, boolean requireAuthentication) { ResourceConfig config = new ResourceConfig(); config.register(resourceClass); config.register(JsonMapperProvider.class); @@ -247,7 +271,8 @@ public void addRestResource(String basePath, String attribute, Object attributeV servletHolder.setAsyncSupported(true); // This method has not historically checked for existing paths, so we don't check here either. The // method call is added to reduce code duplication. - addServlet(basePath, servletHolder, Collections.singletonList(Pair.of(attribute, attributeValue)), true, false); + addServlet(basePath, servletHolder, Collections.singletonList(Pair.of(attribute, attributeValue)), + requireAuthentication, false); } public int getExternalServicePort() { diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java index e912006faa022..88ecfe8a3187b 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java @@ -116,6 +116,7 @@ protected void setup() throws Exception { proxyConfig.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName()); proxyConfig.setBrokerClientAuthenticationParameters(PROXY_TOKEN); proxyConfig.setAuthenticationProviders(providers); + proxyConfig.setStatusFilePath("./src/test/resources/vip_status.html"); AuthenticationService authService = new AuthenticationService(PulsarConfigurationLoader.convertFrom(proxyConfig)); @@ -405,6 +406,29 @@ public void testProxyAuthorizationWithPrefixSubscriptionAuthMode() throws Except log.info("-- Exiting {} test --", methodName); } + @Test + void testGetStatus() throws Exception { + log.info("-- Starting {} test --", methodName); + final PulsarResources resource = new PulsarResources(new ZKMetadataStore(mockZooKeeper), + new ZKMetadataStore(mockZooKeeperGlobal)); + final AuthenticationService authService = new AuthenticationService( + PulsarConfigurationLoader.convertFrom(proxyConfig)); + final WebServer webServer = new WebServer(proxyConfig, authService); + ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, proxyService, + new BrokerDiscoveryProvider(proxyConfig, resource)); + webServer.start(); + @Cleanup + final Client client = javax.ws.rs.client.ClientBuilder + .newClient(new ClientConfig().register(LoggingFeature.class)); + try { + final Response r = client.target(webServer.getServiceUri()).path("/status.html").request().get(); + Assert.assertEquals(r.getStatus(), Response.Status.OK.getStatusCode()); + } finally { + webServer.stop(); + } + log.info("-- Exiting {} test --", methodName); + } + @Test void testGetMetrics() throws Exception { log.info("-- Starting {} test --", methodName); From a59e29339a4f6fe2913d85ed31aa11714e0c026e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 19 Oct 2023 16:04:49 +0300 Subject: [PATCH 130/699] [fix][sec] Upgrade Jetty to 9.4.53 to address CVE-2023-44487 (#21395) (cherry picked from commit 22fd8c26c97238348b251980407ec3c338834f29) --- .../server/src/assemble/LICENSE.bin.txt | 38 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 16 ++++---- pom.xml | 2 +- pulsar-sql/presto-distribution/LICENSE | 32 ++++++++-------- 4 files changed, 44 insertions(+), 44 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 979c23bdf2481..78f507008fddd 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -383,25 +383,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.51.v20230217.jar - - org.eclipse.jetty-jetty-continuation-9.4.51.v20230217.jar - - org.eclipse.jetty-jetty-http-9.4.51.v20230217.jar - - org.eclipse.jetty-jetty-io-9.4.51.v20230217.jar - - org.eclipse.jetty-jetty-proxy-9.4.51.v20230217.jar - - org.eclipse.jetty-jetty-security-9.4.51.v20230217.jar - - org.eclipse.jetty-jetty-server-9.4.51.v20230217.jar - - org.eclipse.jetty-jetty-servlet-9.4.51.v20230217.jar - - org.eclipse.jetty-jetty-servlets-9.4.51.v20230217.jar - - org.eclipse.jetty-jetty-util-9.4.51.v20230217.jar - - org.eclipse.jetty-jetty-util-ajax-9.4.51.v20230217.jar - - org.eclipse.jetty.websocket-javax-websocket-client-impl-9.4.51.v20230217.jar - - org.eclipse.jetty.websocket-websocket-api-9.4.51.v20230217.jar - - org.eclipse.jetty.websocket-websocket-client-9.4.51.v20230217.jar - - org.eclipse.jetty.websocket-websocket-common-9.4.51.v20230217.jar - - org.eclipse.jetty.websocket-websocket-server-9.4.51.v20230217.jar - - org.eclipse.jetty.websocket-websocket-servlet-9.4.51.v20230217.jar - - org.eclipse.jetty-jetty-alpn-conscrypt-server-9.4.51.v20230217.jar - - org.eclipse.jetty-jetty-alpn-server-9.4.51.v20230217.jar + - org.eclipse.jetty-jetty-client-9.4.53.v20231009.jar + - org.eclipse.jetty-jetty-continuation-9.4.53.v20231009.jar + - org.eclipse.jetty-jetty-http-9.4.53.v20231009.jar + - org.eclipse.jetty-jetty-io-9.4.53.v20231009.jar + - org.eclipse.jetty-jetty-proxy-9.4.53.v20231009.jar + - org.eclipse.jetty-jetty-security-9.4.53.v20231009.jar + - org.eclipse.jetty-jetty-server-9.4.53.v20231009.jar + - org.eclipse.jetty-jetty-servlet-9.4.53.v20231009.jar + - org.eclipse.jetty-jetty-servlets-9.4.53.v20231009.jar + - org.eclipse.jetty-jetty-util-9.4.53.v20231009.jar + - org.eclipse.jetty-jetty-util-ajax-9.4.53.v20231009.jar + - org.eclipse.jetty.websocket-javax-websocket-client-impl-9.4.53.v20231009.jar + - org.eclipse.jetty.websocket-websocket-api-9.4.53.v20231009.jar + - org.eclipse.jetty.websocket-websocket-client-9.4.53.v20231009.jar + - org.eclipse.jetty.websocket-websocket-common-9.4.53.v20231009.jar + - org.eclipse.jetty.websocket-websocket-server-9.4.53.v20231009.jar + - org.eclipse.jetty.websocket-websocket-servlet-9.4.53.v20231009.jar + - org.eclipse.jetty-jetty-alpn-conscrypt-server-9.4.53.v20231009.jar + - org.eclipse.jetty-jetty-alpn-server-9.4.53.v20231009.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.5.1.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index f92d95928298a..f179e46c3bd5f 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -395,14 +395,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.51.v20230217.jar - - jetty-http-9.4.51.v20230217.jar - - jetty-io-9.4.51.v20230217.jar - - jetty-util-9.4.51.v20230217.jar - - javax-websocket-client-impl-9.4.51.v20230217.jar - - websocket-api-9.4.51.v20230217.jar - - websocket-client-9.4.51.v20230217.jar - - websocket-common-9.4.51.v20230217.jar + - jetty-client-9.4.53.v20231009.jar + - jetty-http-9.4.53.v20231009.jar + - jetty-io-9.4.53.v20231009.jar + - jetty-util-9.4.53.v20231009.jar + - javax-websocket-client-impl-9.4.53.v20231009.jar + - websocket-api-9.4.53.v20231009.jar + - websocket-client-9.4.53.v20231009.jar + - websocket-common-9.4.53.v20231009.jar * SnakeYaml -- snakeyaml-2.0.jar * Google Error Prone Annotations - error_prone_annotations-2.5.1.jar * Javassist -- javassist-3.25.0-GA.jar diff --git a/pom.xml b/pom.xml index badcced8b86a0..a65900dbb0957 100644 --- a/pom.xml +++ b/pom.xml @@ -142,7 +142,7 @@ flexible messaging model and an intuitive client API. 5.1.0 4.1.94.Final 0.0.21.Final - 9.4.51.v20230217 + 9.4.53.v20231009 2.5.2 2.34 1.10.50 diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index b9fc58d336cee..23f20acddc18c 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -283,22 +283,22 @@ The Apache Software License, Version 2.0 - joda-time-2.10.10.jar - failsafe-2.4.4.jar * Jetty - - http2-client-9.4.51.v20230217.jar - - http2-common-9.4.51.v20230217.jar - - http2-hpack-9.4.51.v20230217.jar - - http2-http-client-transport-9.4.51.v20230217.jar - - jetty-alpn-client-9.4.51.v20230217.jar - - http2-server-9.4.51.v20230217.jar - - jetty-alpn-java-client-9.4.51.v20230217.jar - - jetty-client-9.4.51.v20230217.jar - - jetty-http-9.4.51.v20230217.jar - - jetty-io-9.4.51.v20230217.jar - - jetty-jmx-9.4.51.v20230217.jar - - jetty-security-9.4.51.v20230217.jar - - jetty-server-9.4.51.v20230217.jar - - jetty-servlet-9.4.51.v20230217.jar - - jetty-util-9.4.51.v20230217.jar - - jetty-util-ajax-9.4.51.v20230217.jar + - http2-client-9.4.53.v20231009.jar + - http2-common-9.4.53.v20231009.jar + - http2-hpack-9.4.53.v20231009.jar + - http2-http-client-transport-9.4.53.v20231009.jar + - jetty-alpn-client-9.4.53.v20231009.jar + - http2-server-9.4.53.v20231009.jar + - jetty-alpn-java-client-9.4.53.v20231009.jar + - jetty-client-9.4.53.v20231009.jar + - jetty-http-9.4.53.v20231009.jar + - jetty-io-9.4.53.v20231009.jar + - jetty-jmx-9.4.53.v20231009.jar + - jetty-security-9.4.53.v20231009.jar + - jetty-server-9.4.53.v20231009.jar + - jetty-servlet-9.4.53.v20231009.jar + - jetty-util-9.4.53.v20231009.jar + - jetty-util-ajax-9.4.53.v20231009.jar * Byte Buddy - byte-buddy-1.11.13.jar * Apache BVal From b03ffaccb51f3d4d0b3fafbe06667ff76bb6ad5f Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 19 Oct 2023 16:05:09 +0300 Subject: [PATCH 131/699] [fix][sec] Upgrade Netty to 4.1.100 to address CVE-2023-44487 (#21397) (cherry picked from commit aae6c716b6f7b32c96484b9004b62359e27f158e) # Conflicts: # buildtools/pom.xml # distribution/server/src/assemble/LICENSE.bin.txt # distribution/shell/src/assemble/LICENSE.bin.txt # pom.xml # pulsar-sql/presto-distribution/LICENSE --- buildtools/pom.xml | 2 +- .../server/src/assemble/LICENSE.bin.txt | 42 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 38 ++++++++--------- pom.xml | 2 +- pulsar-sql/presto-distribution/LICENSE | 40 +++++++++--------- 5 files changed, 62 insertions(+), 62 deletions(-) diff --git a/buildtools/pom.xml b/buildtools/pom.xml index c2f993387bd9a..71fc4b9061c8e 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -47,7 +47,7 @@ 4.1 8.37 3.1.2 - 4.1.94.Final + 4.1.100.Final 4.2.3 32.1.1-jre 1.10.12 diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 78f507008fddd..6307780c07dec 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -289,27 +289,27 @@ 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.94.Final.jar - - io.netty-netty-codec-4.1.94.Final.jar - - io.netty-netty-codec-dns-4.1.94.Final.jar - - io.netty-netty-codec-http-4.1.94.Final.jar - - io.netty-netty-codec-http2-4.1.94.Final.jar - - io.netty-netty-codec-socks-4.1.94.Final.jar - - io.netty-netty-codec-haproxy-4.1.94.Final.jar - - io.netty-netty-common-4.1.94.Final.jar - - io.netty-netty-handler-4.1.94.Final.jar - - io.netty-netty-handler-proxy-4.1.94.Final.jar - - io.netty-netty-resolver-4.1.94.Final.jar - - io.netty-netty-resolver-dns-4.1.94.Final.jar - - io.netty-netty-resolver-dns-classes-macos-4.1.94.Final.jar - - io.netty-netty-resolver-dns-native-macos-4.1.94.Final-osx-aarch_64.jar - - io.netty-netty-resolver-dns-native-macos-4.1.94.Final-osx-x86_64.jar - - io.netty-netty-transport-4.1.94.Final.jar - - io.netty-netty-transport-classes-epoll-4.1.94.Final.jar - - io.netty-netty-transport-native-epoll-4.1.94.Final-linux-x86_64.jar - - io.netty-netty-transport-native-epoll-4.1.94.Final.jar - - io.netty-netty-transport-native-unix-common-4.1.94.Final.jar - - io.netty-netty-transport-native-unix-common-4.1.94.Final-linux-x86_64.jar + - io.netty-netty-buffer-4.1.100.Final.jar + - io.netty-netty-codec-4.1.100.Final.jar + - io.netty-netty-codec-dns-4.1.100.Final.jar + - io.netty-netty-codec-http-4.1.100.Final.jar + - io.netty-netty-codec-http2-4.1.100.Final.jar + - io.netty-netty-codec-socks-4.1.100.Final.jar + - io.netty-netty-codec-haproxy-4.1.100.Final.jar + - io.netty-netty-common-4.1.100.Final.jar + - io.netty-netty-handler-4.1.100.Final.jar + - io.netty-netty-handler-proxy-4.1.100.Final.jar + - io.netty-netty-resolver-4.1.100.Final.jar + - io.netty-netty-resolver-dns-4.1.100.Final.jar + - io.netty-netty-resolver-dns-classes-macos-4.1.100.Final.jar + - io.netty-netty-resolver-dns-native-macos-4.1.100.Final-osx-aarch_64.jar + - io.netty-netty-resolver-dns-native-macos-4.1.100.Final-osx-x86_64.jar + - io.netty-netty-transport-4.1.100.Final.jar + - io.netty-netty-transport-classes-epoll-4.1.100.Final.jar + - io.netty-netty-transport-native-epoll-4.1.100.Final-linux-x86_64.jar + - io.netty-netty-transport-native-epoll-4.1.100.Final.jar + - io.netty-netty-transport-native-unix-common-4.1.100.Final.jar + - io.netty-netty-transport-native-unix-common-4.1.100.Final-linux-x86_64.jar - io.netty-netty-tcnative-boringssl-static-2.0.61.Final.jar - io.netty-netty-tcnative-boringssl-static-2.0.61.Final-linux-aarch_64.jar - io.netty-netty-tcnative-boringssl-static-2.0.61.Final-linux-x86_64.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index f179e46c3bd5f..b2df59482b979 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -344,22 +344,22 @@ The Apache Software License, Version 2.0 - commons-text-1.10.0.jar - commons-compress-1.21.jar * Netty - - netty-buffer-4.1.94.Final.jar - - netty-codec-4.1.94.Final.jar - - netty-codec-dns-4.1.94.Final.jar - - netty-codec-http-4.1.94.Final.jar - - netty-codec-socks-4.1.94.Final.jar - - netty-codec-haproxy-4.1.94.Final.jar - - netty-common-4.1.94.Final.jar - - netty-handler-4.1.94.Final.jar - - netty-handler-proxy-4.1.94.Final.jar - - netty-resolver-4.1.94.Final.jar - - netty-resolver-dns-4.1.94.Final.jar - - netty-transport-4.1.94.Final.jar - - netty-transport-classes-epoll-4.1.94.Final.jar - - netty-transport-native-epoll-4.1.94.Final-linux-x86_64.jar - - netty-transport-native-unix-common-4.1.94.Final.jar - - netty-transport-native-unix-common-4.1.94.Final-linux-x86_64.jar + - netty-buffer-4.1.100.Final.jar + - netty-codec-4.1.100.Final.jar + - netty-codec-dns-4.1.100.Final.jar + - netty-codec-http-4.1.100.Final.jar + - netty-codec-socks-4.1.100.Final.jar + - netty-codec-haproxy-4.1.100.Final.jar + - netty-common-4.1.100.Final.jar + - netty-handler-4.1.100.Final.jar + - netty-handler-proxy-4.1.100.Final.jar + - netty-resolver-4.1.100.Final.jar + - netty-resolver-dns-4.1.100.Final.jar + - netty-transport-4.1.100.Final.jar + - netty-transport-classes-epoll-4.1.100.Final.jar + - netty-transport-native-epoll-4.1.100.Final-linux-x86_64.jar + - netty-transport-native-unix-common-4.1.100.Final.jar + - netty-transport-native-unix-common-4.1.100.Final-linux-x86_64.jar - netty-tcnative-boringssl-static-2.0.61.Final.jar - netty-tcnative-boringssl-static-2.0.61.Final-linux-aarch_64.jar - netty-tcnative-boringssl-static-2.0.61.Final-linux-x86_64.jar @@ -370,9 +370,9 @@ The Apache Software License, Version 2.0 - netty-incubator-transport-classes-io_uring-0.0.21.Final.jar - netty-incubator-transport-native-io_uring-0.0.21.Final-linux-aarch_64.jar - netty-incubator-transport-native-io_uring-0.0.21.Final-linux-x86_64.jar - - netty-resolver-dns-classes-macos-4.1.94.Final.jar - - netty-resolver-dns-native-macos-4.1.94.Final-osx-aarch_64.jar - - netty-resolver-dns-native-macos-4.1.94.Final-osx-x86_64.jar + - netty-resolver-dns-classes-macos-4.1.100.Final.jar + - netty-resolver-dns-native-macos-4.1.100.Final-osx-aarch_64.jar + - netty-resolver-dns-native-macos-4.1.100.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 a65900dbb0957..e5a849176cd2f 100644 --- a/pom.xml +++ b/pom.xml @@ -140,7 +140,7 @@ flexible messaging model and an intuitive client API. 1.1.10.5 4.1.12.1 5.1.0 - 4.1.94.Final + 4.1.100.Final 0.0.21.Final 9.4.53.v20231009 2.5.2 diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 23f20acddc18c..8c9f7932a3b6d 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -231,21 +231,21 @@ The Apache Software License, Version 2.0 - commons-compress-1.21.jar - commons-lang3-3.11.jar * Netty - - netty-buffer-4.1.94.Final.jar - - netty-codec-4.1.94.Final.jar - - netty-codec-dns-4.1.94.Final.jar - - netty-codec-http-4.1.94.Final.jar - - netty-codec-haproxy-4.1.94.Final.jar - - netty-codec-socks-4.1.94.Final.jar - - netty-handler-proxy-4.1.94.Final.jar - - netty-common-4.1.94.Final.jar - - netty-handler-4.1.94.Final.jar + - netty-buffer-4.1.100.Final.jar + - netty-codec-4.1.100.Final.jar + - netty-codec-dns-4.1.100.Final.jar + - netty-codec-http-4.1.100.Final.jar + - netty-codec-haproxy-4.1.100.Final.jar + - netty-codec-socks-4.1.100.Final.jar + - netty-handler-proxy-4.1.100.Final.jar + - netty-common-4.1.100.Final.jar + - netty-handler-4.1.100.Final.jar - netty-reactive-streams-2.0.6.jar - - netty-resolver-4.1.94.Final.jar - - netty-resolver-dns-4.1.94.Final.jar - - netty-resolver-dns-classes-macos-4.1.94.Final.jar - - netty-resolver-dns-native-macos-4.1.94.Final-osx-aarch_64.jar - - netty-resolver-dns-native-macos-4.1.94.Final-osx-x86_64.jar + - netty-resolver-4.1.100.Final.jar + - netty-resolver-dns-4.1.100.Final.jar + - netty-resolver-dns-classes-macos-4.1.100.Final.jar + - netty-resolver-dns-native-macos-4.1.100.Final-osx-aarch_64.jar + - netty-resolver-dns-native-macos-4.1.100.Final-osx-x86_64.jar - netty-tcnative-boringssl-static-2.0.61.Final.jar - netty-tcnative-boringssl-static-2.0.61.Final-linux-aarch_64.jar - netty-tcnative-boringssl-static-2.0.61.Final-linux-x86_64.jar @@ -253,12 +253,12 @@ The Apache Software License, Version 2.0 - netty-tcnative-boringssl-static-2.0.61.Final-osx-x86_64.jar - netty-tcnative-boringssl-static-2.0.61.Final-windows-x86_64.jar - netty-tcnative-classes-2.0.61.Final.jar - - netty-transport-4.1.94.Final.jar - - netty-transport-classes-epoll-4.1.94.Final.jar - - netty-transport-native-epoll-4.1.94.Final-linux-x86_64.jar - - netty-transport-native-unix-common-4.1.94.Final.jar - - netty-transport-native-unix-common-4.1.94.Final-linux-x86_64.jar - - netty-codec-http2-4.1.94.Final.jar + - netty-transport-4.1.100.Final.jar + - netty-transport-classes-epoll-4.1.100.Final.jar + - netty-transport-native-epoll-4.1.100.Final-linux-x86_64.jar + - netty-transport-native-unix-common-4.1.100.Final.jar + - netty-transport-native-unix-common-4.1.100.Final-linux-x86_64.jar + - netty-codec-http2-4.1.100.Final.jar - netty-incubator-transport-classes-io_uring-0.0.21.Final.jar - netty-incubator-transport-native-io_uring-0.0.21.Final-linux-x86_64.jar - netty-incubator-transport-native-io_uring-0.0.21.Final-linux-aarch_64.jar From e1ec699625eb4061a33afc5d026e740412869e32 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 26 Oct 2023 08:48:12 -0700 Subject: [PATCH 132/699] Bump version to 3.1.2-SNAPSHOT --- 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 | 2 +- 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 +- managed-ledger/pom.xml | 2 +- pom.xml | 2 +- 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-client-1x-base/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-1x/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-2x-shaded/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/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/hdfs2/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-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-sql/pom.xml | 2 +- pulsar-sql/presto-distribution/pom.xml | 2 +- pulsar-sql/presto-pulsar-plugin/pom.xml | 2 +- pulsar-sql/presto-pulsar/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 +- 133 files changed, 133 insertions(+), 133 deletions(-) diff --git a/bouncy-castle/bc/pom.xml b/bouncy-castle/bc/pom.xml index f1d314a98a815..7e57a206bf3d9 100644 --- a/bouncy-castle/bc/pom.xml +++ b/bouncy-castle/bc/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/bouncy-castle/bcfips-include-test/pom.xml b/bouncy-castle/bcfips-include-test/pom.xml index 3ec9511a7b042..27f8e1dfef9e9 100644 --- a/bouncy-castle/bcfips-include-test/pom.xml +++ b/bouncy-castle/bcfips-include-test/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/bouncy-castle/bcfips/pom.xml b/bouncy-castle/bcfips/pom.xml index 3f217731ea3e1..c526abb1d650d 100644 --- a/bouncy-castle/bcfips/pom.xml +++ b/bouncy-castle/bcfips/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/bouncy-castle/pom.xml b/bouncy-castle/pom.xml index de82aa6b30e6c..bd603050a41b6 100644 --- a/bouncy-castle/pom.xml +++ b/bouncy-castle/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 71fc4b9061c8e..b5a78470c0952 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -31,7 +31,7 @@ com.datastax.oss buildtools - 3.1.1 + 3.1.2-SNAPSHOT jar Pulsar Build Tools diff --git a/distribution/io/pom.xml b/distribution/io/pom.xml index 74800f514e6fb..12d8a5be1bf9d 100644 --- a/distribution/io/pom.xml +++ b/distribution/io/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/distribution/offloaders/pom.xml b/distribution/offloaders/pom.xml index 797d8324e4c6f..21e8fce1acf5d 100644 --- a/distribution/offloaders/pom.xml +++ b/distribution/offloaders/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/distribution/pom.xml b/distribution/pom.xml index d839d51e80d98..5dadef0b48986 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index 4895467b2dd2d..b302792b4160e 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/distribution/shell/pom.xml b/distribution/shell/pom.xml index caff9367f32bd..27d1ef5fa00ec 100644 --- a/distribution/shell/pom.xml +++ b/distribution/shell/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/docker/pom.xml b/docker/pom.xml index a9fb7603863e9..70291bfa9cdb6 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT docker-images Apache Pulsar :: Docker Images diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index cfe6b2558efcd..64f7d8953ca21 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.1 + 3.1.2-SNAPSHOT 4.0.0 pulsar-all-docker-image diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 3e7dd305ef055..aacf4a09cd695 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.1 + 3.1.2-SNAPSHOT 4.0.0 pulsar-docker-image diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml index 8c36e6caed14c..980f204b1c275 100644 --- a/jclouds-shaded/pom.xml +++ b/jclouds-shaded/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index aab2fb407d4b5..83ba015ea4b7a 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pom.xml b/pom.xml index e5a849176cd2f..eabee9c9d5d0e 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT Pulsar Pulsar is a distributed pub-sub messaging platform with a very diff --git a/pulsar-broker-auth-athenz/pom.xml b/pulsar-broker-auth-athenz/pom.xml index ff4e78e1d4a9f..7a12f4dc879d9 100644 --- a/pulsar-broker-auth-athenz/pom.xml +++ b/pulsar-broker-auth-athenz/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT pulsar-broker-auth-athenz diff --git a/pulsar-broker-auth-oidc/pom.xml b/pulsar-broker-auth-oidc/pom.xml index 4b20bab7bbb6e..ec8d7639c71ac 100644 --- a/pulsar-broker-auth-oidc/pom.xml +++ b/pulsar-broker-auth-oidc/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT pulsar-broker-auth-oidc diff --git a/pulsar-broker-auth-sasl/pom.xml b/pulsar-broker-auth-sasl/pom.xml index fd8c193ce7bd6..e7af037a92d6c 100644 --- a/pulsar-broker-auth-sasl/pom.xml +++ b/pulsar-broker-auth-sasl/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT pulsar-broker-auth-sasl diff --git a/pulsar-broker-common/pom.xml b/pulsar-broker-common/pom.xml index c3796a96927e4..3ab30329fd364 100644 --- a/pulsar-broker-common/pom.xml +++ b/pulsar-broker-common/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT pulsar-broker-common diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index 2be655f64138f..138a6f9e20962 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-client-1x-base/pom.xml b/pulsar-client-1x-base/pom.xml index 2cd99a31c0bda..d20100003c6c4 100644 --- a/pulsar-client-1x-base/pom.xml +++ b/pulsar-client-1x-base/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-client-1x-base/pulsar-client-1x/pom.xml b/pulsar-client-1x-base/pulsar-client-1x/pom.xml index ab6d160a8a9fc..0a98905245dbf 100644 --- a/pulsar-client-1x-base/pulsar-client-1x/pom.xml +++ b/pulsar-client-1x-base/pulsar-client-1x/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar-client-1x-base - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml b/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml index 5b2aac86c2c24..574b8859f913a 100644 --- a/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml +++ b/pulsar-client-1x-base/pulsar-client-2x-shaded/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar-client-1x-base - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-client-admin-api/pom.xml b/pulsar-client-admin-api/pom.xml index d779c1b1be77d..60fe4d0b46c08 100644 --- a/pulsar-client-admin-api/pom.xml +++ b/pulsar-client-admin-api/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-client-admin-shaded/pom.xml b/pulsar-client-admin-shaded/pom.xml index 896233f5928e6..02fb18c6c67c5 100644 --- a/pulsar-client-admin-shaded/pom.xml +++ b/pulsar-client-admin-shaded/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-client-admin/pom.xml b/pulsar-client-admin/pom.xml index 7c80c19cb326f..11a5961e4f3ff 100644 --- a/pulsar-client-admin/pom.xml +++ b/pulsar-client-admin/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index 42b0eb28a1d53..09df6dca321d9 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-client-api/pom.xml b/pulsar-client-api/pom.xml index ba9ff3f2e5d72..b5051d82dfa45 100644 --- a/pulsar-client-api/pom.xml +++ b/pulsar-client-api/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-client-auth-athenz/pom.xml b/pulsar-client-auth-athenz/pom.xml index 0003d43f6fd8e..7e3e90f55fa1b 100644 --- a/pulsar-client-auth-athenz/pom.xml +++ b/pulsar-client-auth-athenz/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-client-auth-sasl/pom.xml b/pulsar-client-auth-sasl/pom.xml index c1b2b55e34eb0..dd813c00a3778 100644 --- a/pulsar-client-auth-sasl/pom.xml +++ b/pulsar-client-auth-sasl/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-client-messagecrypto-bc/pom.xml b/pulsar-client-messagecrypto-bc/pom.xml index 0aaf0a4ed8d37..e286a43a45b0c 100644 --- a/pulsar-client-messagecrypto-bc/pom.xml +++ b/pulsar-client-messagecrypto-bc/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-client-shaded/pom.xml b/pulsar-client-shaded/pom.xml index a72cba5695c86..35c0a3eba28ce 100644 --- a/pulsar-client-shaded/pom.xml +++ b/pulsar-client-shaded/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-client-tools-api/pom.xml b/pulsar-client-tools-api/pom.xml index b9f8ded828dd3..54eb97310d39a 100644 --- a/pulsar-client-tools-api/pom.xml +++ b/pulsar-client-tools-api/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-client-tools-customcommand-example/pom.xml b/pulsar-client-tools-customcommand-example/pom.xml index abac5000a889b..65863eb12b830 100644 --- a/pulsar-client-tools-customcommand-example/pom.xml +++ b/pulsar-client-tools-customcommand-example/pom.xml @@ -22,7 +22,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. 4.0.0 diff --git a/pulsar-client-tools-test/pom.xml b/pulsar-client-tools-test/pom.xml index 0312da9b63d84..d5ba3195fd7a1 100644 --- a/pulsar-client-tools-test/pom.xml +++ b/pulsar-client-tools-test/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-client-tools/pom.xml b/pulsar-client-tools/pom.xml index f0113e8100b64..7e3f23a13884c 100644 --- a/pulsar-client-tools/pom.xml +++ b/pulsar-client-tools/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-client/pom.xml b/pulsar-client/pom.xml index a707fc57aed43..22cc344a88605 100644 --- a/pulsar-client/pom.xml +++ b/pulsar-client/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index affc86822a4b6..f3e4ad29a5a17 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-config-validation/pom.xml b/pulsar-config-validation/pom.xml index 992ec42a74b6f..d12e615ad5e7a 100644 --- a/pulsar-config-validation/pom.xml +++ b/pulsar-config-validation/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-docs-tools/pom.xml b/pulsar-docs-tools/pom.xml index 735ac5e98508d..a8c295ce20a30 100644 --- a/pulsar-docs-tools/pom.xml +++ b/pulsar-docs-tools/pom.xml @@ -27,7 +27,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT pulsar-docs-tools diff --git a/pulsar-functions/api-java/pom.xml b/pulsar-functions/api-java/pom.xml index 23abbf55385bc..d3828b44699c0 100644 --- a/pulsar-functions/api-java/pom.xml +++ b/pulsar-functions/api-java/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-functions - 3.1.1 + 3.1.2-SNAPSHOT pulsar-functions-api diff --git a/pulsar-functions/instance/pom.xml b/pulsar-functions/instance/pom.xml index f077371efe0f7..7b292c1843b0e 100644 --- a/pulsar-functions/instance/pom.xml +++ b/pulsar-functions/instance/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar-functions - 3.1.1 + 3.1.2-SNAPSHOT pulsar-functions-instance diff --git a/pulsar-functions/java-examples-builtin/pom.xml b/pulsar-functions/java-examples-builtin/pom.xml index 3006a54bd6e0d..76c2e0a29d39e 100644 --- a/pulsar-functions/java-examples-builtin/pom.xml +++ b/pulsar-functions/java-examples-builtin/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-functions - 3.1.1 + 3.1.2-SNAPSHOT pulsar-functions-api-examples-builtin diff --git a/pulsar-functions/java-examples/pom.xml b/pulsar-functions/java-examples/pom.xml index 3227b6e98198c..d370d5356c046 100644 --- a/pulsar-functions/java-examples/pom.xml +++ b/pulsar-functions/java-examples/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-functions - 3.1.1 + 3.1.2-SNAPSHOT pulsar-functions-api-examples diff --git a/pulsar-functions/localrun-shaded/pom.xml b/pulsar-functions/localrun-shaded/pom.xml index a1d200b0fb178..472f74c73e1d0 100644 --- a/pulsar-functions/localrun-shaded/pom.xml +++ b/pulsar-functions/localrun-shaded/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar-functions - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-functions/localrun/pom.xml b/pulsar-functions/localrun/pom.xml index 012d56400d69b..4cd5748d8b640 100644 --- a/pulsar-functions/localrun/pom.xml +++ b/pulsar-functions/localrun/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar-functions - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-functions/pom.xml b/pulsar-functions/pom.xml index 72c61013eb11d..9967fe903e535 100644 --- a/pulsar-functions/pom.xml +++ b/pulsar-functions/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT pulsar-functions diff --git a/pulsar-functions/proto/pom.xml b/pulsar-functions/proto/pom.xml index 99a12d248f9e9..dbf289155a49e 100644 --- a/pulsar-functions/proto/pom.xml +++ b/pulsar-functions/proto/pom.xml @@ -27,7 +27,7 @@ com.datastax.oss pulsar-functions - 3.1.1 + 3.1.2-SNAPSHOT pulsar-functions-proto diff --git a/pulsar-functions/runtime-all/pom.xml b/pulsar-functions/runtime-all/pom.xml index ec542f169aac8..b0784f6a6db1e 100644 --- a/pulsar-functions/runtime-all/pom.xml +++ b/pulsar-functions/runtime-all/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar-functions - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-functions/runtime/pom.xml b/pulsar-functions/runtime/pom.xml index b06fd4bb1cda1..708d1916f8025 100644 --- a/pulsar-functions/runtime/pom.xml +++ b/pulsar-functions/runtime/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar-functions - 3.1.1 + 3.1.2-SNAPSHOT pulsar-functions-runtime diff --git a/pulsar-functions/secrets/pom.xml b/pulsar-functions/secrets/pom.xml index ba27a8a9f0e19..a162186beba2d 100644 --- a/pulsar-functions/secrets/pom.xml +++ b/pulsar-functions/secrets/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-functions - 3.1.1 + 3.1.2-SNAPSHOT pulsar-functions-secrets diff --git a/pulsar-functions/utils/pom.xml b/pulsar-functions/utils/pom.xml index 1475b81f3b4b8..57a2659763ec1 100644 --- a/pulsar-functions/utils/pom.xml +++ b/pulsar-functions/utils/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar-functions - 3.1.1 + 3.1.2-SNAPSHOT pulsar-functions-utils diff --git a/pulsar-functions/worker/pom.xml b/pulsar-functions/worker/pom.xml index fba52eedb79a3..f2a35d23401c9 100644 --- a/pulsar-functions/worker/pom.xml +++ b/pulsar-functions/worker/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar-functions - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-io/aerospike/pom.xml b/pulsar-io/aerospike/pom.xml index dde84a9df63db..ce5d0ba75cb27 100644 --- a/pulsar-io/aerospike/pom.xml +++ b/pulsar-io/aerospike/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-aerospike diff --git a/pulsar-io/alluxio/pom.xml b/pulsar-io/alluxio/pom.xml index 28ca96642329b..37243b812d7b2 100644 --- a/pulsar-io/alluxio/pom.xml +++ b/pulsar-io/alluxio/pom.xml @@ -25,7 +25,7 @@ pulsar-io com.datastax.oss - 3.1.1 + 3.1.2 diff --git a/pulsar-io/aws/pom.xml b/pulsar-io/aws/pom.xml index 5613cb9e0bb48..14da68276190c 100644 --- a/pulsar-io/aws/pom.xml +++ b/pulsar-io/aws/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-aws diff --git a/pulsar-io/batch-data-generator/pom.xml b/pulsar-io/batch-data-generator/pom.xml index d012b69bfd351..b32941829e42c 100644 --- a/pulsar-io/batch-data-generator/pom.xml +++ b/pulsar-io/batch-data-generator/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-batch-data-generator diff --git a/pulsar-io/batch-discovery-triggerers/pom.xml b/pulsar-io/batch-discovery-triggerers/pom.xml index 7c13c09921289..a2214f8668cfd 100644 --- a/pulsar-io/batch-discovery-triggerers/pom.xml +++ b/pulsar-io/batch-discovery-triggerers/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-batch-discovery-triggerers diff --git a/pulsar-io/canal/pom.xml b/pulsar-io/canal/pom.xml index 065985ef8465a..d9e55490994de 100644 --- a/pulsar-io/canal/pom.xml +++ b/pulsar-io/canal/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT 4.0.0 diff --git a/pulsar-io/cassandra/pom.xml b/pulsar-io/cassandra/pom.xml index e1d1056e82b1d..1dff3919aa8c4 100644 --- a/pulsar-io/cassandra/pom.xml +++ b/pulsar-io/cassandra/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-cassandra diff --git a/pulsar-io/common/pom.xml b/pulsar-io/common/pom.xml index 8acfa4397e8bd..44475f779873a 100644 --- a/pulsar-io/common/pom.xml +++ b/pulsar-io/common/pom.xml @@ -27,7 +27,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-common diff --git a/pulsar-io/core/pom.xml b/pulsar-io/core/pom.xml index 920282159bb8a..c0eeefed7f68f 100644 --- a/pulsar-io/core/pom.xml +++ b/pulsar-io/core/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-core diff --git a/pulsar-io/data-generator/pom.xml b/pulsar-io/data-generator/pom.xml index 54d45ebeee34b..50f802b1cc2bb 100644 --- a/pulsar-io/data-generator/pom.xml +++ b/pulsar-io/data-generator/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-data-generator diff --git a/pulsar-io/debezium/core/pom.xml b/pulsar-io/debezium/core/pom.xml index e265de906f3e4..c4c14b343993e 100644 --- a/pulsar-io/debezium/core/pom.xml +++ b/pulsar-io/debezium/core/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io-debezium - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-debezium-core diff --git a/pulsar-io/debezium/mongodb/pom.xml b/pulsar-io/debezium/mongodb/pom.xml index f62b9208aa8ac..d9896fdc47008 100644 --- a/pulsar-io/debezium/mongodb/pom.xml +++ b/pulsar-io/debezium/mongodb/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io-debezium - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-debezium-mongodb diff --git a/pulsar-io/debezium/mssql/pom.xml b/pulsar-io/debezium/mssql/pom.xml index 8580d17cd8b64..611145e99e629 100644 --- a/pulsar-io/debezium/mssql/pom.xml +++ b/pulsar-io/debezium/mssql/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io-debezium - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-debezium-mssql diff --git a/pulsar-io/debezium/mysql/pom.xml b/pulsar-io/debezium/mysql/pom.xml index 338bcd028d86e..f872a223c47dc 100644 --- a/pulsar-io/debezium/mysql/pom.xml +++ b/pulsar-io/debezium/mysql/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io-debezium - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-debezium-mysql diff --git a/pulsar-io/debezium/oracle/pom.xml b/pulsar-io/debezium/oracle/pom.xml index ab0cdde410a47..6b864f33fb88e 100644 --- a/pulsar-io/debezium/oracle/pom.xml +++ b/pulsar-io/debezium/oracle/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io-debezium - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-debezium-oracle diff --git a/pulsar-io/debezium/pom.xml b/pulsar-io/debezium/pom.xml index 1effdd6c0c07d..1fb1f29d45ebf 100644 --- a/pulsar-io/debezium/pom.xml +++ b/pulsar-io/debezium/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-debezium diff --git a/pulsar-io/debezium/postgres/pom.xml b/pulsar-io/debezium/postgres/pom.xml index feb70de92fbae..61d7577f29e6c 100644 --- a/pulsar-io/debezium/postgres/pom.xml +++ b/pulsar-io/debezium/postgres/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io-debezium - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-debezium-postgres diff --git a/pulsar-io/docs/pom.xml b/pulsar-io/docs/pom.xml index 0f5f20ec34665..ea1dba4cff7a2 100644 --- a/pulsar-io/docs/pom.xml +++ b/pulsar-io/docs/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-docs diff --git a/pulsar-io/dynamodb/pom.xml b/pulsar-io/dynamodb/pom.xml index 5ad7867ec4053..a30369d6a9880 100644 --- a/pulsar-io/dynamodb/pom.xml +++ b/pulsar-io/dynamodb/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-dynamodb diff --git a/pulsar-io/elastic-search/pom.xml b/pulsar-io/elastic-search/pom.xml index ca2d40169d5ed..7f115b2daaf45 100644 --- a/pulsar-io/elastic-search/pom.xml +++ b/pulsar-io/elastic-search/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-elastic-search Pulsar IO :: ElasticSearch diff --git a/pulsar-io/file/pom.xml b/pulsar-io/file/pom.xml index 6723a6ee351b3..2262f8c7339f8 100644 --- a/pulsar-io/file/pom.xml +++ b/pulsar-io/file/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-file diff --git a/pulsar-io/flume/pom.xml b/pulsar-io/flume/pom.xml index 833f8f9009ed7..ca5fffde93ea3 100644 --- a/pulsar-io/flume/pom.xml +++ b/pulsar-io/flume/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-flume diff --git a/pulsar-io/hbase/pom.xml b/pulsar-io/hbase/pom.xml index c674544992cbc..8a713f8dedefd 100644 --- a/pulsar-io/hbase/pom.xml +++ b/pulsar-io/hbase/pom.xml @@ -25,7 +25,7 @@ pulsar-io com.datastax.oss - 3.1.1 + 3.1.2 pulsar-io-hbase Pulsar IO :: Hbase diff --git a/pulsar-io/hdfs2/pom.xml b/pulsar-io/hdfs2/pom.xml index 9466406978af9..584a53441d08a 100644 --- a/pulsar-io/hdfs2/pom.xml +++ b/pulsar-io/hdfs2/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-hdfs2 Pulsar IO :: Hdfs2 diff --git a/pulsar-io/hdfs3/pom.xml b/pulsar-io/hdfs3/pom.xml index 18aac80c8356e..5eb6a29a1774b 100644 --- a/pulsar-io/hdfs3/pom.xml +++ b/pulsar-io/hdfs3/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-hdfs3 Pulsar IO :: Hdfs3 diff --git a/pulsar-io/http/pom.xml b/pulsar-io/http/pom.xml index 3744e2f6fc4a0..467e361270ff2 100644 --- a/pulsar-io/http/pom.xml +++ b/pulsar-io/http/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-http diff --git a/pulsar-io/influxdb/pom.xml b/pulsar-io/influxdb/pom.xml index 6b609208ea7e6..21306c4f90d92 100644 --- a/pulsar-io/influxdb/pom.xml +++ b/pulsar-io/influxdb/pom.xml @@ -25,7 +25,7 @@ pulsar-io com.datastax.oss - 3.1.1 + 3.1.2 pulsar-io-influxdb diff --git a/pulsar-io/jdbc/clickhouse/pom.xml b/pulsar-io/jdbc/clickhouse/pom.xml index c2f97c4720b6b..08659a4a8836e 100644 --- a/pulsar-io/jdbc/clickhouse/pom.xml +++ b/pulsar-io/jdbc/clickhouse/pom.xml @@ -24,7 +24,7 @@ pulsar-io-jdbc com.datastax.oss - 3.1.1 + 3.1.2 4.0.0 diff --git a/pulsar-io/jdbc/core/pom.xml b/pulsar-io/jdbc/core/pom.xml index af5bb33647b8c..e3014fa04cd2d 100644 --- a/pulsar-io/jdbc/core/pom.xml +++ b/pulsar-io/jdbc/core/pom.xml @@ -24,7 +24,7 @@ pulsar-io-jdbc com.datastax.oss - 3.1.1 + 3.1.2 4.0.0 diff --git a/pulsar-io/jdbc/mariadb/pom.xml b/pulsar-io/jdbc/mariadb/pom.xml index 7baf21d53bbb9..acb5479bb5acc 100644 --- a/pulsar-io/jdbc/mariadb/pom.xml +++ b/pulsar-io/jdbc/mariadb/pom.xml @@ -24,7 +24,7 @@ pulsar-io-jdbc com.datastax.oss - 3.1.1 + 3.1.2 4.0.0 diff --git a/pulsar-io/jdbc/openmldb/pom.xml b/pulsar-io/jdbc/openmldb/pom.xml index 32377db9999c8..14a5f638c1943 100644 --- a/pulsar-io/jdbc/openmldb/pom.xml +++ b/pulsar-io/jdbc/openmldb/pom.xml @@ -24,7 +24,7 @@ pulsar-io-jdbc com.datastax.oss - 3.1.1 + 3.1.2 4.0.0 diff --git a/pulsar-io/jdbc/pom.xml b/pulsar-io/jdbc/pom.xml index bee2df61a2cf3..45476efcb1cb5 100644 --- a/pulsar-io/jdbc/pom.xml +++ b/pulsar-io/jdbc/pom.xml @@ -33,7 +33,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-jdbc diff --git a/pulsar-io/jdbc/postgres/pom.xml b/pulsar-io/jdbc/postgres/pom.xml index 246c1b01c8225..a37a74f115134 100644 --- a/pulsar-io/jdbc/postgres/pom.xml +++ b/pulsar-io/jdbc/postgres/pom.xml @@ -24,7 +24,7 @@ pulsar-io-jdbc com.datastax.oss - 3.1.1 + 3.1.2 4.0.0 diff --git a/pulsar-io/jdbc/sqlite/pom.xml b/pulsar-io/jdbc/sqlite/pom.xml index 21a3edbb9d4ab..c05f09cb82660 100644 --- a/pulsar-io/jdbc/sqlite/pom.xml +++ b/pulsar-io/jdbc/sqlite/pom.xml @@ -24,7 +24,7 @@ pulsar-io-jdbc com.datastax.oss - 3.1.1 + 3.1.2 4.0.0 pulsar-io-jdbc-sqlite diff --git a/pulsar-io/kafka-connect-adaptor-nar/pom.xml b/pulsar-io/kafka-connect-adaptor-nar/pom.xml index eb003a9d9f8c1..3d137603944ed 100644 --- a/pulsar-io/kafka-connect-adaptor-nar/pom.xml +++ b/pulsar-io/kafka-connect-adaptor-nar/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-kafka-connect-adaptor-nar diff --git a/pulsar-io/kafka-connect-adaptor/pom.xml b/pulsar-io/kafka-connect-adaptor/pom.xml index 59f3ceb2ccf1c..2a2560edc1f81 100644 --- a/pulsar-io/kafka-connect-adaptor/pom.xml +++ b/pulsar-io/kafka-connect-adaptor/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-kafka-connect-adaptor diff --git a/pulsar-io/kafka/pom.xml b/pulsar-io/kafka/pom.xml index 6ab5b247c137d..daae4d470471e 100644 --- a/pulsar-io/kafka/pom.xml +++ b/pulsar-io/kafka/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-kafka diff --git a/pulsar-io/kinesis/pom.xml b/pulsar-io/kinesis/pom.xml index b859532bc29e3..ba5a30a766a31 100644 --- a/pulsar-io/kinesis/pom.xml +++ b/pulsar-io/kinesis/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-kinesis diff --git a/pulsar-io/mongo/pom.xml b/pulsar-io/mongo/pom.xml index e288d4da0a2ca..f1d5b4c0b424c 100644 --- a/pulsar-io/mongo/pom.xml +++ b/pulsar-io/mongo/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-mongo diff --git a/pulsar-io/netty/pom.xml b/pulsar-io/netty/pom.xml index 8b027ab0c6054..0f6a37799bd11 100644 --- a/pulsar-io/netty/pom.xml +++ b/pulsar-io/netty/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-netty diff --git a/pulsar-io/nsq/pom.xml b/pulsar-io/nsq/pom.xml index f09ad6bd96fd1..9240f23769321 100644 --- a/pulsar-io/nsq/pom.xml +++ b/pulsar-io/nsq/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-nsq diff --git a/pulsar-io/pom.xml b/pulsar-io/pom.xml index 8d36fd5267e5a..21aa813e198ce 100644 --- a/pulsar-io/pom.xml +++ b/pulsar-io/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io diff --git a/pulsar-io/rabbitmq/pom.xml b/pulsar-io/rabbitmq/pom.xml index 1dc7b4e056532..ea8f80e9bc074 100644 --- a/pulsar-io/rabbitmq/pom.xml +++ b/pulsar-io/rabbitmq/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-rabbitmq diff --git a/pulsar-io/redis/pom.xml b/pulsar-io/redis/pom.xml index ce95e9f8ae466..5926a70f6a2ec 100644 --- a/pulsar-io/redis/pom.xml +++ b/pulsar-io/redis/pom.xml @@ -25,7 +25,7 @@ pulsar-io com.datastax.oss - 3.1.1 + 3.1.2 pulsar-io-redis diff --git a/pulsar-io/solr/pom.xml b/pulsar-io/solr/pom.xml index 5fcb97b7c56d1..ca3735121898f 100644 --- a/pulsar-io/solr/pom.xml +++ b/pulsar-io/solr/pom.xml @@ -25,7 +25,7 @@ pulsar-io com.datastax.oss - 3.1.1 + 3.1.2 diff --git a/pulsar-io/twitter/pom.xml b/pulsar-io/twitter/pom.xml index 9b7aac7bb3f37..9ffee29af151b 100644 --- a/pulsar-io/twitter/pom.xml +++ b/pulsar-io/twitter/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar-io - 3.1.1 + 3.1.2-SNAPSHOT pulsar-io-twitter diff --git a/pulsar-metadata/pom.xml b/pulsar-metadata/pom.xml index 8b1496c66fea7..bb8299135af46 100644 --- a/pulsar-metadata/pom.xml +++ b/pulsar-metadata/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-package-management/bookkeeper-storage/pom.xml b/pulsar-package-management/bookkeeper-storage/pom.xml index 72920e30ba513..621b13b2a5c5a 100644 --- a/pulsar-package-management/bookkeeper-storage/pom.xml +++ b/pulsar-package-management/bookkeeper-storage/pom.xml @@ -25,7 +25,7 @@ pulsar-package-management com.datastax.oss - 3.1.1 + 3.1.2 4.0.0 diff --git a/pulsar-package-management/core/pom.xml b/pulsar-package-management/core/pom.xml index e4af2999fad79..dc9204c3f682e 100644 --- a/pulsar-package-management/core/pom.xml +++ b/pulsar-package-management/core/pom.xml @@ -25,7 +25,7 @@ pulsar-package-management com.datastax.oss - 3.1.1 + 3.1.2 4.0.0 diff --git a/pulsar-package-management/filesystem-storage/pom.xml b/pulsar-package-management/filesystem-storage/pom.xml index 14aa1cff1c24a..32adc2178c25b 100644 --- a/pulsar-package-management/filesystem-storage/pom.xml +++ b/pulsar-package-management/filesystem-storage/pom.xml @@ -25,7 +25,7 @@ pulsar-package-management com.datastax.oss - 3.1.1 + 3.1.2 4.0.0 diff --git a/pulsar-package-management/pom.xml b/pulsar-package-management/pom.xml index 766341c8b0752..22acc9c69bae1 100644 --- a/pulsar-package-management/pom.xml +++ b/pulsar-package-management/pom.xml @@ -25,7 +25,7 @@ pulsar com.datastax.oss - 3.1.1 + 3.1.2 .. 4.0.0 diff --git a/pulsar-proxy/pom.xml b/pulsar-proxy/pom.xml index cbc62c5faed28..f86bd4ec22f3e 100644 --- a/pulsar-proxy/pom.xml +++ b/pulsar-proxy/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT pulsar-proxy diff --git a/pulsar-sql/pom.xml b/pulsar-sql/pom.xml index 3e2d5574a8af4..e972a6887c751 100644 --- a/pulsar-sql/pom.xml +++ b/pulsar-sql/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT pulsar-sql diff --git a/pulsar-sql/presto-distribution/pom.xml b/pulsar-sql/presto-distribution/pom.xml index 694eaf366058c..5ad818bcbce70 100644 --- a/pulsar-sql/presto-distribution/pom.xml +++ b/pulsar-sql/presto-distribution/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar-sql - 3.1.1 + 3.1.2-SNAPSHOT pulsar-presto-distribution diff --git a/pulsar-sql/presto-pulsar-plugin/pom.xml b/pulsar-sql/presto-pulsar-plugin/pom.xml index 5ca7f3546b4f8..43ff06fe084d4 100644 --- a/pulsar-sql/presto-pulsar-plugin/pom.xml +++ b/pulsar-sql/presto-pulsar-plugin/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar-sql - 3.1.1 + 3.1.2-SNAPSHOT pulsar-presto-connector diff --git a/pulsar-sql/presto-pulsar/pom.xml b/pulsar-sql/presto-pulsar/pom.xml index c196e133a7ff7..0bc4c6407276e 100644 --- a/pulsar-sql/presto-pulsar/pom.xml +++ b/pulsar-sql/presto-pulsar/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar-sql - 3.1.1 + 3.1.2-SNAPSHOT pulsar-presto-connector-original diff --git a/pulsar-testclient/pom.xml b/pulsar-testclient/pom.xml index 969e5e832e1fc..a763393d6962a 100644 --- a/pulsar-testclient/pom.xml +++ b/pulsar-testclient/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/pulsar-transaction/common/pom.xml b/pulsar-transaction/common/pom.xml index 095cecf105fc9..6fa45de69a016 100644 --- a/pulsar-transaction/common/pom.xml +++ b/pulsar-transaction/common/pom.xml @@ -27,7 +27,7 @@ com.datastax.oss pulsar-transaction-parent - 3.1.1 + 3.1.2-SNAPSHOT pulsar-transaction-common diff --git a/pulsar-transaction/coordinator/pom.xml b/pulsar-transaction/coordinator/pom.xml index c877a653f98eb..c1a7e4378f31d 100644 --- a/pulsar-transaction/coordinator/pom.xml +++ b/pulsar-transaction/coordinator/pom.xml @@ -27,7 +27,7 @@ com.datastax.oss pulsar-transaction-parent - 3.1.1 + 3.1.2-SNAPSHOT pulsar-transaction-coordinator diff --git a/pulsar-transaction/pom.xml b/pulsar-transaction/pom.xml index a43b77ecc1fb1..f1b389e710143 100644 --- a/pulsar-transaction/pom.xml +++ b/pulsar-transaction/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT pulsar-transaction-parent diff --git a/pulsar-websocket/pom.xml b/pulsar-websocket/pom.xml index 4f41498ef4a5f..295462fea8820 100644 --- a/pulsar-websocket/pom.xml +++ b/pulsar-websocket/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/structured-event-log/pom.xml b/structured-event-log/pom.xml index 382504c32ace5..71efacb869ad8 100644 --- a/structured-event-log/pom.xml +++ b/structured-event-log/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/testmocks/pom.xml b/testmocks/pom.xml index a76b847d9014f..0cd526fafb095 100644 --- a/testmocks/pom.xml +++ b/testmocks/pom.xml @@ -25,7 +25,7 @@ pulsar com.datastax.oss - 3.1.1 + 3.1.2 testmocks diff --git a/tests/bc_2_0_0/pom.xml b/tests/bc_2_0_0/pom.xml index a2e46072e8622..ece40d51feba4 100644 --- a/tests/bc_2_0_0/pom.xml +++ b/tests/bc_2_0_0/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar.tests tests-parent - 3.1.1 + 3.1.2-SNAPSHOT bc_2_0_0 diff --git a/tests/bc_2_0_1/pom.xml b/tests/bc_2_0_1/pom.xml index 9aa8bde8d6a02..3c0ab5617cbc6 100644 --- a/tests/bc_2_0_1/pom.xml +++ b/tests/bc_2_0_1/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar.tests tests-parent - 3.1.1 + 3.1.2-SNAPSHOT bc_2_0_1 diff --git a/tests/bc_2_6_0/pom.xml b/tests/bc_2_6_0/pom.xml index 22f610a4a6e45..da3f7b5ea8ae0 100644 --- a/tests/bc_2_6_0/pom.xml +++ b/tests/bc_2_6_0/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar.tests tests-parent - 3.1.1 + 3.1.2-SNAPSHOT 4.0.0 diff --git a/tests/docker-images/java-test-functions/pom.xml b/tests/docker-images/java-test-functions/pom.xml index afa2394ed0186..98e31fbd1d7dc 100644 --- a/tests/docker-images/java-test-functions/pom.xml +++ b/tests/docker-images/java-test-functions/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar.tests docker-images - 3.1.1 + 3.1.2-SNAPSHOT 4.0.0 java-test-functions diff --git a/tests/docker-images/java-test-image/pom.xml b/tests/docker-images/java-test-image/pom.xml index c6ca39feb1c90..ea312390f407a 100644 --- a/tests/docker-images/java-test-image/pom.xml +++ b/tests/docker-images/java-test-image/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar.tests docker-images - 3.1.1 + 3.1.2-SNAPSHOT 4.0.0 java-test-image diff --git a/tests/docker-images/java-test-plugins/pom.xml b/tests/docker-images/java-test-plugins/pom.xml index d4c48360d112c..398bac40b0a92 100644 --- a/tests/docker-images/java-test-plugins/pom.xml +++ b/tests/docker-images/java-test-plugins/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar.tests docker-images - 3.1.1 + 3.1.2-SNAPSHOT 4.0.0 java-test-plugins diff --git a/tests/docker-images/latest-version-image/pom.xml b/tests/docker-images/latest-version-image/pom.xml index 5c159bf6746f0..bba2038701e84 100644 --- a/tests/docker-images/latest-version-image/pom.xml +++ b/tests/docker-images/latest-version-image/pom.xml @@ -23,7 +23,7 @@ org.apache.pulsar.tests docker-images - 3.1.1 + 3.1.2-SNAPSHOT 4.0.0 latest-version-image diff --git a/tests/docker-images/pom.xml b/tests/docker-images/pom.xml index 61677d1801933..1fcca50508eeb 100644 --- a/tests/docker-images/pom.xml +++ b/tests/docker-images/pom.xml @@ -27,7 +27,7 @@ org.apache.pulsar.tests tests-parent - 3.1.1 + 3.1.2-SNAPSHOT docker-images Apache Pulsar :: Tests :: Docker Images diff --git a/tests/integration/pom.xml b/tests/integration/pom.xml index f60c8dd8a424a..8387d9b25e6d5 100644 --- a/tests/integration/pom.xml +++ b/tests/integration/pom.xml @@ -25,7 +25,7 @@ org.apache.pulsar.tests tests-parent - 3.1.1 + 3.1.2-SNAPSHOT integration diff --git a/tests/pom.xml b/tests/pom.xml index ec19f0156b3c2..7ced4efbc63c9 100644 --- a/tests/pom.xml +++ b/tests/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT org.apache.pulsar.tests tests-parent diff --git a/tests/pulsar-client-admin-shade-test/pom.xml b/tests/pulsar-client-admin-shade-test/pom.xml index 3841f0e2a75ef..4069d10363eec 100644 --- a/tests/pulsar-client-admin-shade-test/pom.xml +++ b/tests/pulsar-client-admin-shade-test/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar.tests tests-parent - 3.1.1 + 3.1.2-SNAPSHOT pulsar-client-admin-shade-test diff --git a/tests/pulsar-client-all-shade-test/pom.xml b/tests/pulsar-client-all-shade-test/pom.xml index fcce474f4c3a5..0ac73d0a9576f 100644 --- a/tests/pulsar-client-all-shade-test/pom.xml +++ b/tests/pulsar-client-all-shade-test/pom.xml @@ -26,7 +26,7 @@ org.apache.pulsar.tests tests-parent - 3.1.1 + 3.1.2-SNAPSHOT pulsar-client-all-shade-test diff --git a/tests/pulsar-client-shade-test/pom.xml b/tests/pulsar-client-shade-test/pom.xml index 60704cbc2593a..6faa3a20c07bb 100644 --- a/tests/pulsar-client-shade-test/pom.xml +++ b/tests/pulsar-client-shade-test/pom.xml @@ -27,7 +27,7 @@ org.apache.pulsar.tests tests-parent - 3.1.1 + 3.1.2-SNAPSHOT pulsar-client-shade-test diff --git a/tiered-storage/file-system/pom.xml b/tiered-storage/file-system/pom.xml index a739e406441b2..a4662ff364243 100644 --- a/tiered-storage/file-system/pom.xml +++ b/tiered-storage/file-system/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss tiered-storage-parent - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/tiered-storage/jcloud/pom.xml b/tiered-storage/jcloud/pom.xml index 49ee5ea8f8c46..b075e555cf9da 100644 --- a/tiered-storage/jcloud/pom.xml +++ b/tiered-storage/jcloud/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss tiered-storage-parent - 3.1.1 + 3.1.2-SNAPSHOT .. diff --git a/tiered-storage/pom.xml b/tiered-storage/pom.xml index cbc028da7fd86..2a247eaef1799 100644 --- a/tiered-storage/pom.xml +++ b/tiered-storage/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.1 + 3.1.2-SNAPSHOT .. From 0c361ce6e00c88617dc445cbed72da838b573de4 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 19 Oct 2023 16:48:37 +0300 Subject: [PATCH 133/699] [fix][sec] Upgrade Zookeeper to 3.8.3 to address CVE-2023-44981 (#21398) (cherry picked from commit e5120ec68907525177f5add5c95b022f3106da1a) --- distribution/server/src/assemble/LICENSE.bin.txt | 6 +++--- pom.xml | 2 +- pulsar-sql/presto-distribution/LICENSE | 4 ++-- 3 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 6307780c07dec..258ba8e9bafe8 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -480,9 +480,9 @@ The Apache Software License, Version 2.0 - io.vertx-vertx-web-common-4.3.8.jar - io.vertx-vertx-grpc-4.3.5.jar * Apache ZooKeeper - - org.apache.zookeeper-zookeeper-3.8.1.jar - - org.apache.zookeeper-zookeeper-jute-3.8.1.jar - - org.apache.zookeeper-zookeeper-prometheus-metrics-3.8.1.jar + - org.apache.zookeeper-zookeeper-3.8.3.jar + - org.apache.zookeeper-zookeeper-jute-3.8.3.jar + - org.apache.zookeeper-zookeeper-prometheus-metrics-3.8.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 eabee9c9d5d0e..1c95513ef4e20 100644 --- a/pom.xml +++ b/pom.xml @@ -134,7 +134,7 @@ flexible messaging model and an intuitive client API. 1.21 4.16.3 - 3.8.1 + 3.8.3 1.5.0 1.10.0 1.1.10.5 diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 8c9f7932a3b6d..0f61055260dcd 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -472,8 +472,8 @@ The Apache Software License, Version 2.0 - memory-0.8.3.jar - sketches-core-0.8.3.jar * Apache Zookeeper - - zookeeper-3.8.1.jar - - zookeeper-jute-3.8.1.jar + - zookeeper-3.8.3.jar + - zookeeper-jute-3.8.3.jar * Apache Yetus Audience Annotations - audience-annotations-0.12.0.jar * Perfmark From af877cc5dacc0ee3f85617447b3e28a4396393ee Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Wed, 25 Oct 2023 23:02:19 +0800 Subject: [PATCH 134/699] [fix][broker] Fix MultiRoles token provider NPE when using anonymous clients (#21429) --- .../AuthenticationDataSubscription.java | 4 ++++ .../MultiRolesTokenAuthorizationProvider.java | 24 ++++++++++++------- ...tiRolesTokenAuthorizationProviderTest.java | 22 +++++++++++++++++ 3 files changed, 42 insertions(+), 8 deletions(-) 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 69ef526012daa..9a7324a6d077a 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 @@ -71,4 +71,8 @@ public boolean hasSubscription() { public String getSubscription() { return subscription; } + + public AuthenticationDataSource getAuthData() { + return authData; + } } diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java index 6376b60217fef..7d17d180cf1f0 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java @@ -35,6 +35,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; +import org.apache.pulsar.broker.authentication.AuthenticationDataSubscription; import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; @@ -144,7 +145,8 @@ public CompletableFuture validateTenantAdminAccess(String tenantName, S } private Set getRoles(String role, AuthenticationDataSource authData) { - if (authData == null) { + if (authData == null || (authData instanceof AuthenticationDataSubscription + && ((AuthenticationDataSubscription) authData).getAuthData() == null)) { return Collections.singleton(role); } @@ -198,13 +200,19 @@ private Set getRoles(String role, AuthenticationDataSource authData) { public CompletableFuture authorize(String role, AuthenticationDataSource authenticationData, Function> authorizeFunc) { - Set roles = getRoles(role, authenticationData); - if (roles.isEmpty()) { - return CompletableFuture.completedFuture(false); - } - List> futures = new ArrayList<>(roles.size()); - roles.forEach(r -> futures.add(authorizeFunc.apply(r))); - return FutureUtil.waitForAny(futures, ret -> (boolean) ret).thenApply(v -> v.isPresent()); + return isSuperUser(role, authenticationData, conf) + .thenCompose(superUser -> { + if (superUser) { + return CompletableFuture.completedFuture(true); + } + Set roles = getRoles(role, authenticationData); + if (roles.isEmpty()) { + return CompletableFuture.completedFuture(false); + } + List> futures = new ArrayList<>(roles.size()); + roles.forEach(r -> futures.add(authorizeFunc.apply(r))); + return FutureUtil.waitForAny(futures, ret -> (boolean) ret).thenApply(v -> v.isPresent()); + }); } /** diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java index 4b67f52075ca9..c4fc35f64019e 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java @@ -28,6 +28,7 @@ import lombok.Cleanup; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; +import org.apache.pulsar.broker.authentication.AuthenticationDataSubscription; import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils; import org.apache.pulsar.broker.resources.PulsarResources; import org.testng.annotations.Test; @@ -46,6 +47,8 @@ public void testMultiRolesAuthz() throws Exception { String token = Jwts.builder().claim("sub", new String[]{userA, userB}).signWith(secretKey).compact(); MultiRolesTokenAuthorizationProvider provider = new MultiRolesTokenAuthorizationProvider(); + ServiceConfiguration conf = new ServiceConfiguration(); + provider.initialize(conf, mock(PulsarResources.class)); AuthenticationDataSource ads = new AuthenticationDataSource() { @Override @@ -85,6 +88,8 @@ public void testMultiRolesAuthzWithEmptyRoles() throws Exception { String token = Jwts.builder().claim("sub", new String[]{}).signWith(secretKey).compact(); MultiRolesTokenAuthorizationProvider provider = new MultiRolesTokenAuthorizationProvider(); + ServiceConfiguration conf = new ServiceConfiguration(); + provider.initialize(conf, mock(PulsarResources.class)); AuthenticationDataSource ads = new AuthenticationDataSource() { @Override @@ -112,6 +117,8 @@ public void testMultiRolesAuthzWithSingleRole() throws Exception { String token = Jwts.builder().claim("sub", testRole).signWith(secretKey).compact(); MultiRolesTokenAuthorizationProvider provider = new MultiRolesTokenAuthorizationProvider(); + ServiceConfiguration conf = new ServiceConfiguration(); + provider.initialize(conf, mock(PulsarResources.class)); AuthenticationDataSource ads = new AuthenticationDataSource() { @Override @@ -141,6 +148,9 @@ public String getHttpHeader(String name) { public void testMultiRolesAuthzWithAnonymousUser() throws Exception { @Cleanup MultiRolesTokenAuthorizationProvider provider = new MultiRolesTokenAuthorizationProvider(); + ServiceConfiguration conf = new ServiceConfiguration(); + + provider.initialize(conf, mock(PulsarResources.class)); Function> authorizeFunc = (String role) -> { if (role.equals("test-role")) { @@ -150,6 +160,7 @@ public void testMultiRolesAuthzWithAnonymousUser() throws Exception { }; assertTrue(provider.authorize("test-role", null, authorizeFunc).get()); assertFalse(provider.authorize("test-role-x", null, authorizeFunc).get()); + assertTrue(provider.authorize("test-role", new AuthenticationDataSubscription(null, "test-sub"), authorizeFunc).get()); } @Test @@ -157,6 +168,8 @@ public void testMultiRolesNotFailNonJWT() throws Exception { String token = "a-non-jwt-token"; MultiRolesTokenAuthorizationProvider provider = new MultiRolesTokenAuthorizationProvider(); + ServiceConfiguration conf = new ServiceConfiguration(); + provider.initialize(conf, mock(PulsarResources.class)); AuthenticationDataSource ads = new AuthenticationDataSource() { @Override @@ -246,5 +259,14 @@ public String getHttpHeader(String name) { }; assertTrue(provider.isSuperUser(testAdminRole, ads, conf).get()); + Function> authorizeFunc = (String role) -> { + if (role.equals("admin1")) { + return CompletableFuture.completedFuture(true); + } + return CompletableFuture.completedFuture(false); + }; + assertTrue(provider.authorize(testAdminRole, ads, (String role) -> CompletableFuture.completedFuture(false)).get()); + assertTrue(provider.authorize("admin1", null, authorizeFunc).get()); + assertFalse(provider.authorize("admin2", null, authorizeFunc).get()); } } From 51d273d309fed5708755536deb90d5d771d0d011 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Wed, 25 Oct 2023 21:27:37 +0800 Subject: [PATCH 135/699] [fix][broker] Ignore individual acknowledgment for CompactorSubscription when an entry has been filtered. (#21434) --- .../service/AbstractBaseDispatcher.java | 16 ++--- .../service/plugin/FilterEntryTest.java | 61 ++++++++++++++++++- 2 files changed, 68 insertions(+), 9 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java index eb8b015139586..b36389ab2dada 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java @@ -176,14 +176,16 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i if (Markers.isTxnMarker(msgMetadata)) { // because consumer can receive message is smaller than maxReadPosition, // so this marker is useless for this subscription - individualAcknowledgeMessageIfNeeded(entry.getPosition(), Collections.emptyMap()); + individualAcknowledgeMessageIfNeeded(Collections.singletonList(entry.getPosition()), + Collections.emptyMap()); entries.set(i, null); entry.release(); continue; } else if (((PersistentTopic) subscription.getTopic()) .isTxnAborted(new TxnID(msgMetadata.getTxnidMostBits(), msgMetadata.getTxnidLeastBits()), (PositionImpl) entry.getPosition())) { - individualAcknowledgeMessageIfNeeded(entry.getPosition(), Collections.emptyMap()); + individualAcknowledgeMessageIfNeeded(Collections.singletonList(entry.getPosition()), + Collections.emptyMap()); entries.set(i, null); entry.release(); continue; @@ -200,7 +202,8 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i entries.set(i, null); entry.release(); - individualAcknowledgeMessageIfNeeded(pos, Collections.emptyMap()); + individualAcknowledgeMessageIfNeeded(Collections.singletonList(pos), + Collections.emptyMap()); continue; } else if (trackDelayedDelivery(entry.getLedgerId(), entry.getEntryId(), msgMetadata)) { // The message is marked for delayed delivery. Ignore for now. @@ -271,8 +274,7 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i } } if (CollectionUtils.isNotEmpty(entriesToFiltered)) { - subscription.acknowledgeMessage(entriesToFiltered, AckType.Individual, - Collections.emptyMap()); + individualAcknowledgeMessageIfNeeded(entriesToFiltered, Collections.emptyMap()); int filtered = entriesToFiltered.size(); Topic topic = subscription.getTopic(); @@ -301,9 +303,9 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i return totalEntries; } - private void individualAcknowledgeMessageIfNeeded(Position position, Map properties) { + private void individualAcknowledgeMessageIfNeeded(List positions, Map properties) { if (!(subscription instanceof PulsarCompactorSubscription)) { - subscription.acknowledgeMessage(Collections.singletonList(position), AckType.Individual, properties); + subscription.acknowledgeMessage(positions, AckType.Individual, properties); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java index b868858646c50..1c4f88bc0273c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java @@ -22,6 +22,7 @@ import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgsRecordingInvocations; import static org.apache.pulsar.client.api.SubscriptionInitialPosition.Earliest; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; @@ -30,8 +31,9 @@ import static org.testng.Assert.assertTrue; import static org.testng.AssertJUnit.assertEquals; import static org.testng.AssertJUnit.assertNotNull; - +import io.netty.buffer.ByteBuf; import java.lang.reflect.Field; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -39,7 +41,6 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; - import lombok.Cleanup; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; @@ -58,11 +59,15 @@ 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.RawMessage; +import org.apache.pulsar.client.api.RawReader; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.nar.NarClassLoader; +import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.stats.AnalyzeSubscriptionBacklogResult; +import org.apache.pulsar.compaction.Compactor; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterMethod; @@ -150,6 +155,58 @@ public void testOverride() throws Exception { consumer.close(); } + @Test + public void testEntryFilterWithCompactor() throws Exception { + conf.setAllowOverrideEntryFilters(true); + String topic = "persistent://prop/ns-abc/topic" + UUID.randomUUID(); + + List messages = new ArrayList<>(); + Producer producer = pulsarClient.newProducer(Schema.STRING) + .enableBatching(false).topic(topic).create(); + producer.newMessage().key("K1").value("V1").send(); + producer.newMessage().key("K2").value("V2").send(); + producer.newMessage().key("K3").value("V3").send(); + producer.newMessage().key("K4").value("V4").send(); + messages.add("V2"); + messages.add("V4"); + + PersistentTopic topicRef = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topic).get(); + + // set topic level entry filters + EntryFilter mockFilter = mock(EntryFilter.class); + doAnswer(invocationOnMock -> { + FilterContext filterContext = invocationOnMock.getArgument(1); + String partitionKey = filterContext.getMsgMetadata().getPartitionKey(); + if (partitionKey.equals("K1") || partitionKey.equals("K3")) { + return EntryFilter.FilterResult.REJECT; + } else { + return EntryFilter.FilterResult.ACCEPT; + } + }).when(mockFilter).filterEntry(any(Entry.class), any(FilterContext.class)); + setMockFilterToTopic(topicRef, List.of(mockFilter)); + + List results = new ArrayList<>(); + RawReader rawReader = RawReader.create(pulsarClient, topic, Compactor.COMPACTION_SUBSCRIPTION).get(); + while (true) { + boolean hasMsg = rawReader.hasMessageAvailableAsync().get(); + if (hasMsg) { + try (RawMessage m = rawReader.readNextAsync().get()) { + ByteBuf headersAndPayload = m.getHeadersAndPayload(); + Commands.skipMessageMetadata(headersAndPayload); + byte[] bytes = new byte[headersAndPayload.readableBytes()]; + headersAndPayload.readBytes(bytes); + + results.add(new String(bytes)); + } + } else { + break; + } + } + rawReader.closeAsync().get(); + + Assert.assertEquals(messages, results); + } + @SneakyThrows private void setMockFilterToTopic(PersistentTopic topicRef, List mockFilter) { FieldUtils.writeField(topicRef, "entryFilters", Pair.of(null, mockFilter), true); From 81d8ce50223c64252fee039ef36b408b1a135022 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 1 Nov 2023 00:23:52 +0200 Subject: [PATCH 136/699] [fix][build] Fix apt download issue in building the docker image (#21489) (cherry picked from commit 7c6a4b8b5e97c19f2dad37b402a57d26172b6cd2) --- docker/pulsar/Dockerfile | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index 593401f57be8c..c8ff0ef4f189a 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -61,9 +61,9 @@ RUN sed -i -e "s|http://archive\.ubuntu\.com/ubuntu/|${UBUNTU_MIRROR:-mirror://m && echo 'Acquire::http::Timeout "30";\nAcquire::ftp::Timeout "30";\nAcquire::Retries "3";' > /etc/apt/apt.conf.d/99timeout_and_retries \ && apt-get update \ && apt-get -y dist-upgrade \ - && apt-get -y install --no-install-recommends netcat dnsutils less procps iputils-ping \ - python3 python3-kazoo python3-pip \ - curl ca-certificates wget apt-transport-https + && apt-get -y install netcat dnsutils less procps iputils-ping \ + curl ca-certificates wget apt-transport-https \ + && apt-get -y install --no-install-recommends python3 python3-kazoo python3-pip # Install Eclipse Temurin Package RUN mkdir -p /etc/apt/keyrings \ From aa51fff6021bbdf517caa1844b7ee84bc780b34d Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Wed, 8 Nov 2023 11:51:03 +0800 Subject: [PATCH 137/699] [fix][txn] Ack all message ids when ack chunk messages with transaction. (#21268) ### Motivation Now, only the last chunk will be acknowledged when acknowledging chunk messages with transactions. If the messageId is a `ChunkMessageIdImpl`, the ledger ID and entry ID will belong to the `lastChunkMsgId`. https://github.com/apache/pulsar/blob/2b5c199053a5b2d7f849e6604d619bae9197a8c9/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java#L2791-L2814 https://github.com/apache/pulsar/blob/2b5c199053a5b2d7f849e6604d619bae9197a8c9/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ChunkMessageIdImpl.java#L30-L33 ### Modifications Flow the common message acknowledge logic, ack all the chunks when acknowledging messages with transactions. (cherry picked from commit f5814176efc08dc05553cf1059b3d55f6bcb2b6b) --- .../transaction/TransactionConsumeTest.java | 46 ++++++++++ .../pulsar/client/impl/ConsumerImpl.java | 90 +++++++++++++++++-- 2 files changed, 130 insertions(+), 6 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionConsumeTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionConsumeTest.java index 0e17719aca7e7..9e262d1cb5617 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionConsumeTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionConsumeTest.java @@ -19,6 +19,8 @@ package org.apache.pulsar.broker.transaction; import static java.nio.charset.StandardCharsets.UTF_8; +import static org.testng.Assert.assertNotEquals; +import static org.testng.Assert.assertTrue; import com.google.common.collect.Sets; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; @@ -39,12 +41,15 @@ import org.apache.pulsar.broker.service.persistent.PersistentTopic; 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.PulsarClientException; import org.apache.pulsar.client.api.Schema; 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.client.impl.ChunkMessageIdImpl; import org.apache.pulsar.common.api.proto.MessageIdData; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.api.proto.TxnAction; @@ -368,4 +373,45 @@ public void completed(Exception e, long ledgerId, long entryId) { return positionList; } + @Test + public void testAckChunkMessage() throws Exception { + String producerName = "test-producer"; + String subName = "testAckChunkMessage"; + @Cleanup + PulsarClient pulsarClient1 = PulsarClient.builder().serviceUrl(pulsarServiceList.get(0).getBrokerServiceUrl()) + .enableTransaction(true).build(); + @Cleanup + Producer producer = pulsarClient1 + .newProducer(Schema.STRING) + .producerName(producerName) + .topic(CONSUME_TOPIC) + .enableChunking(true) + .enableBatching(false) + .create(); + Consumer consumer = pulsarClient1 + .newConsumer(Schema.STRING) + .subscriptionType(SubscriptionType.Shared) + .topic(CONSUME_TOPIC) + .subscriptionName(subName) + .subscribe(); + + int messageSize = 6000; // payload size in KB + String message = "a".repeat(messageSize * 1000); + MessageId messageId = producer.newMessage().value(message).send(); + assertTrue(messageId instanceof ChunkMessageIdImpl); + assertNotEquals(((ChunkMessageIdImpl) messageId).getLastChunkMessageId(), + ((ChunkMessageIdImpl) messageId).getFirstChunkMessageId()); + + Transaction transaction = pulsarClient1.newTransaction() + .withTransactionTimeout(5, TimeUnit.HOURS) + .build() + .get(); + + Message msg = consumer.receive(); + consumer.acknowledgeAsync(msg.getMessageId(), transaction); + transaction.commit().get(); + + Assert.assertEquals(admin.topics().getStats(CONSUME_TOPIC).getSubscriptions().get(subName) + .getUnackedMessages(), 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 a275554d917f0..e2333065f8881 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 @@ -21,6 +21,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; import static org.apache.pulsar.common.protocol.Commands.hasChecksum; +import static org.apache.pulsar.common.protocol.Commands.serializeWithSize; import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ComparisonChain; @@ -32,6 +33,7 @@ import io.netty.util.Recycler.Handle; import io.netty.util.ReferenceCountUtil; import io.netty.util.Timeout; +import io.netty.util.concurrent.FastThreadLocal; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -67,6 +69,7 @@ import lombok.AccessLevel; import lombok.Getter; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.Triple; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerCryptoFailureAction; import org.apache.pulsar.client.api.DeadLetterPolicy; @@ -94,7 +97,9 @@ import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.api.EncryptionContext; import org.apache.pulsar.common.api.EncryptionContext.EncryptionKey; +import org.apache.pulsar.common.api.proto.BaseCommand; import org.apache.pulsar.common.api.proto.BrokerEntryMetadata; +import org.apache.pulsar.common.api.proto.CommandAck; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.api.proto.CommandAck.ValidationError; import org.apache.pulsar.common.api.proto.CommandMessage; @@ -117,6 +122,7 @@ import org.apache.pulsar.common.util.FutureUtil; 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; @@ -2781,7 +2787,7 @@ private CompletableFuture doTransactionAcknowledgeForResponse(MessageId me final MessageIdAdv messageIdAdv = (MessageIdAdv) messageId; final long ledgerId = messageIdAdv.getLedgerId(); final long entryId = messageIdAdv.getEntryId(); - final ByteBuf cmd; + final List cmdList; if (MessageIdAdvUtils.isBatch(messageIdAdv)) { BitSetRecyclable bitSetRecyclable = BitSetRecyclable.create(); bitSetRecyclable.set(0, messageIdAdv.getBatchSize()); @@ -2791,12 +2797,37 @@ private CompletableFuture doTransactionAcknowledgeForResponse(MessageId me } else { bitSetRecyclable.clear(messageIdAdv.getBatchIndex()); } - cmd = Commands.newAck(consumerId, ledgerId, entryId, bitSetRecyclable, ackType, validationError, properties, - txnID.getLeastSigBits(), txnID.getMostSigBits(), requestId, messageIdAdv.getBatchSize()); + cmdList = Collections.singletonList(Commands.newAck(consumerId, ledgerId, entryId, bitSetRecyclable, + ackType, validationError, properties, txnID.getLeastSigBits(), txnID.getMostSigBits(), requestId, + messageIdAdv.getBatchSize())); bitSetRecyclable.recycle(); } else { - cmd = Commands.newAck(consumerId, ledgerId, entryId, null, ackType, validationError, properties, - txnID.getLeastSigBits(), txnID.getMostSigBits(), requestId); + MessageIdImpl[] chunkMsgIds = this.unAckedChunkedMessageIdSequenceMap.remove(messageIdAdv); + // cumulative ack chunk by the last messageId + if (chunkMsgIds == null || ackType == AckType.Cumulative) { + cmdList = Collections.singletonList(Commands.newAck(consumerId, ledgerId, entryId, null, ackType, + validationError, properties, txnID.getLeastSigBits(), txnID.getMostSigBits(), requestId)); + } else { + if (Commands.peerSupportsMultiMessageAcknowledgment( + getClientCnx().getRemoteEndpointProtocolVersion())) { + List> entriesToAck = + new ArrayList<>(chunkMsgIds.length); + for (MessageIdImpl cMsgId : chunkMsgIds) { + if (cMsgId != null && chunkMsgIds.length > 1) { + entriesToAck.add(Triple.of(cMsgId.getLedgerId(), cMsgId.getEntryId(), null)); + } + } + cmdList = Collections.singletonList( + newMultiTransactionMessageAck(consumerId, txnID, entriesToAck, requestId)); + } else { + cmdList = new ArrayList<>(); + for (MessageIdImpl cMsgId : chunkMsgIds) { + cmdList.add(Commands.newAck(consumerId, cMsgId.ledgerId, cMsgId.entryId, null, ackType, + validationError, properties, + txnID.getLeastSigBits(), txnID.getMostSigBits(), requestId)); + } + } + } } if (ackType == AckType.Cumulative) { @@ -2810,8 +2841,55 @@ private CompletableFuture doTransactionAcknowledgeForResponse(MessageId me .ConnectException("Failed to ack message [" + messageId + "] " + "for transaction [" + txnID + "] due to consumer connect fail, consumer state: " + getState())); } else { - return cnx.newAckForReceipt(cmd, requestId); + List> completableFutures = new LinkedList<>(); + cmdList.forEach(cmd -> completableFutures.add(cnx.newAckForReceipt(cmd, requestId))); + return FutureUtil.waitForAll(completableFutures); + } + } + + private ByteBuf newMultiTransactionMessageAck(long consumerId, TxnID txnID, + List> entries, + long requestID) { + BaseCommand cmd = newMultiMessageAckCommon(entries); + cmd.getAck() + .setConsumerId(consumerId) + .setAckType(AckType.Individual) + .setTxnidLeastBits(txnID.getLeastSigBits()) + .setTxnidMostBits(txnID.getMostSigBits()) + .setRequestId(requestID); + return serializeWithSize(cmd); + } + + private static final FastThreadLocal LOCAL_BASE_COMMAND = new FastThreadLocal() { + @Override + protected BaseCommand initialValue() throws Exception { + return new BaseCommand(); + } + }; + + private static BaseCommand newMultiMessageAckCommon(List> entries) { + BaseCommand cmd = LOCAL_BASE_COMMAND.get() + .clear() + .setType(BaseCommand.Type.ACK); + CommandAck ack = cmd.setAck(); + int entriesCount = entries.size(); + for (int i = 0; i < entriesCount; i++) { + long ledgerId = entries.get(i).getLeft(); + long entryId = entries.get(i).getMiddle(); + ConcurrentBitSetRecyclable bitSet = entries.get(i).getRight(); + MessageIdData msgId = ack.addMessageId() + .setLedgerId(ledgerId) + .setEntryId(entryId); + if (bitSet != null) { + long[] ackSet = bitSet.toLongArray(); + for (int j = 0; j < ackSet.length; j++) { + msgId.addAckSet(ackSet[j]); + } + bitSet.recycle(); + } } + + return cmd; } private CompletableFuture doTransactionAcknowledgeForResponse(List messageIds, AckType ackType, From 6a19476e681d0b48ecf13ba29b099a06643f9210 Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Mon, 6 Nov 2023 15:55:07 +0800 Subject: [PATCH 138/699] [fix][client] Avert extensive time consumption during table view construction (#21270) Reopen https://github.com/apache/pulsar/pull/21170 ### Motivation If a topic persistently experiences a substantial quantity of data inputs, the act of reading all the messages present in this topic to build a TableView can take an excessive amount of time. ### Modification In the process of constructing the TableView, initially, the last message ID of the current topic is procured. Consequently, once this last message ID has been reached, the creation ensues to its completion. --- .../pulsar/client/impl/TableViewTest.java | 60 +++++++++++++++++++ .../pulsar/client/impl/TableViewImpl.java | 28 +++++++-- 2 files changed, 84 insertions(+), 4 deletions(-) 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 6c6da5870aed9..523360884c1bf 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 @@ -20,16 +20,21 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; 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 com.google.common.collect.Sets; +import java.lang.reflect.Method; import java.time.Duration; import java.util.HashSet; +import java.util.List; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -39,6 +44,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; @@ -46,6 +52,7 @@ import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.TableView; +import org.apache.pulsar.client.api.TopicMessageId; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; @@ -438,4 +445,57 @@ public void testTableViewTailMessageReadRetry() throws Exception { }); verify(consumer, times(msgCnt)).receiveAsync(); } + + @Test + public void testBuildTableViewWithMessagesAlwaysAvailable() throws Exception { + String topic = "persistent://public/default/testBuildTableViewWithMessagesAlwaysAvailable"; + admin.topics().createPartitionedTopic(topic, 10); + @Cleanup + Reader reader = pulsarClient.newReader() + .topic(topic) + .startMessageId(MessageId.earliest) + .create(); + @Cleanup + Producer producer = pulsarClient.newProducer() + .topic(topic) + .create(); + // Prepare real data to do test. + for (int i = 0; i < 1000; i++) { + producer.newMessage().send(); + } + List lastMessageIds = reader.getLastMessageIds(); + + // Use mock reader to build tableview. In the old implementation, the readAllExistingMessages method + // will not be completed because the `mockReader.hasMessageAvailable()` always return ture. + Reader mockReader = spy(reader); + when(mockReader.hasMessageAvailable()).thenReturn(true); + when(mockReader.getLastMessageIdsAsync()).thenReturn(CompletableFuture.completedFuture(lastMessageIds)); + AtomicInteger index = new AtomicInteger(lastMessageIds.size()); + when(mockReader.readNextAsync()).thenAnswer(invocation -> { + Message message = spy(Message.class); + int localIndex = index.decrementAndGet(); + if (localIndex >= 0) { + when(message.getTopicName()).thenReturn(lastMessageIds.get(localIndex).getOwnerTopic()); + when(message.getMessageId()).thenReturn(lastMessageIds.get(localIndex)); + when(message.hasKey()).thenReturn(false); + doNothing().when(message).release(); + } + return CompletableFuture.completedFuture(message); + }); + @Cleanup + TableViewImpl tableView = (TableViewImpl) pulsarClient.newTableView() + .topic(topic) + .createAsync() + .get(); + TableViewImpl mockTableView = spy(tableView); + Method readAllExistingMessagesMethod = TableViewImpl.class + .getDeclaredMethod("readAllExistingMessages", Reader.class); + readAllExistingMessagesMethod.setAccessible(true); + CompletableFuture> future = + (CompletableFuture>) readAllExistingMessagesMethod.invoke(mockTableView, mockReader); + + // The future will complete after receive all the messages from lastMessageIds. + future.get(3, TimeUnit.SECONDS); + assertTrue(index.get() <= 0); + } } 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 560636f94622b..151c96d96aa40 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 @@ -41,6 +41,7 @@ import org.apache.pulsar.client.api.ReaderBuilder; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.TableView; +import org.apache.pulsar.client.api.TopicMessageId; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.topics.TopicCompactionStrategy; @@ -235,20 +236,40 @@ private CompletableFuture> readAllExistingMessages(Reader reader) { AtomicLong messagesRead = new AtomicLong(); CompletableFuture> future = new CompletableFuture<>(); - readAllExistingMessages(reader, future, startTime, messagesRead); + reader.getLastMessageIdsAsync().thenAccept(lastMessageIds -> { + Map maxMessageIds = new ConcurrentHashMap<>(); + lastMessageIds.forEach(topicMessageId -> { + maxMessageIds.put(topicMessageId.getOwnerTopic(), topicMessageId); + }); + readAllExistingMessages(reader, future, startTime, messagesRead, maxMessageIds); + }).exceptionally(ex -> { + future.completeExceptionally(ex); + return null; + }); + future.thenAccept(__ -> readTailMessages(reader)); return future; } private void readAllExistingMessages(Reader reader, CompletableFuture> future, long startTime, - AtomicLong messagesRead) { + AtomicLong messagesRead, Map maxMessageIds) { reader.hasMessageAvailableAsync() .thenAccept(hasMessage -> { if (hasMessage) { reader.readNextAsync() .thenAccept(msg -> { messagesRead.incrementAndGet(); + // We need remove the partition from the maxMessageIds map + // once the partition has been read completely. + TopicMessageId maxMessageId = maxMessageIds.get(msg.getTopicName()); + if (maxMessageId != null && msg.getMessageId().compareTo(maxMessageId) >= 0) { + maxMessageIds.remove(msg.getTopicName()); + } handleMessage(msg); - readAllExistingMessages(reader, future, startTime, messagesRead); + if (maxMessageIds.isEmpty()) { + future.complete(reader); + } else { + readAllExistingMessages(reader, future, startTime, messagesRead, maxMessageIds); + } }).exceptionally(ex -> { if (ex.getCause() instanceof PulsarClientException.AlreadyClosedException) { log.error("Reader {} was closed while reading existing messages.", @@ -269,7 +290,6 @@ private void readAllExistingMessages(Reader reader, CompletableFuture Date: Mon, 23 Oct 2023 15:05:18 +0300 Subject: [PATCH 139/699] [fix][test] Fix LocalBookkeeperEnsemble resource leak in tests (#21407) --- .../zookeeper/LocalBookkeeperEnsemble.java | 10 ++++++---- .../LocalBookkeeperEnsembleTest.java | 20 ------------------- 2 files changed, 6 insertions(+), 24 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java b/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java index d73d1d7ed6bed..63d146a3a1521 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java @@ -52,7 +52,6 @@ import org.apache.bookkeeper.clients.exceptions.NamespaceExistsException; import org.apache.bookkeeper.clients.exceptions.NamespaceNotFoundException; import org.apache.bookkeeper.common.allocator.PoolingPolicy; -import org.apache.bookkeeper.common.component.ComponentStarter; import org.apache.bookkeeper.common.component.LifecycleComponent; import org.apache.bookkeeper.common.component.LifecycleComponentStack; import org.apache.bookkeeper.common.concurrent.FutureUtils; @@ -132,7 +131,7 @@ public LocalBookkeeperEnsemble(int numberOfBookies, boolean clearOldData, String advertisedAddress) { this(numberOfBookies, zkPort, streamStoragePort, zkDataDirName, bkDataDirName, clearOldData, advertisedAddress, - new BasePortManager(bkBasePort)); + bkBasePort != 0 ? new BasePortManager(bkBasePort) : () -> 0); } public LocalBookkeeperEnsemble(int numberOfBookies, @@ -311,6 +310,7 @@ private void runBookies(ServerConfiguration baseConf) throws Exception { bsConfs[i] = new ServerConfiguration(baseConf); // override settings bsConfs[i].setBookiePort(bookiePort); + bsConfs[i].setBookieId("bk" + i + "test"); String zkServers = "127.0.0.1:" + zkPort; String metadataServiceUriStr = "zk://" + zkServers + "/ledgers"; @@ -455,8 +455,10 @@ public void startBK(int i) throws Exception { try { bookieComponents[i] = org.apache.bookkeeper.server.Main .buildBookieServer(new BookieConfiguration(bsConfs[i])); - ComponentStarter.startComponent(bookieComponents[i]); + bookieComponents[i].start(); } catch (BookieException.InvalidCookieException ice) { + LOG.warn("Invalid cookie found for bookie {}", i, ice); + // InvalidCookieException can happen if the machine IP has changed // Since we are running here a local bookie that is always accessed // from localhost, we can ignore the error @@ -473,7 +475,7 @@ public void startBK(int i) throws Exception { bookieComponents[i] = org.apache.bookkeeper.server.Main .buildBookieServer(new BookieConfiguration(bsConfs[i])); - ComponentStarter.startComponent(bookieComponents[i]); + bookieComponents[i].start(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsembleTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsembleTest.java index 92899feda7371..a4bc69a7266cc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsembleTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsembleTest.java @@ -21,10 +21,6 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; - -import java.util.Collections; -import java.util.List; - import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -39,22 +35,6 @@ void setup() throws Exception { void teardown() throws Exception { } - @Test - public void testAdvertisedAddress() throws Exception { - final int numBk = 1; - - LocalBookkeeperEnsemble ensemble = new LocalBookkeeperEnsemble( - numBk, 0, 0, null, null, true, "127.0.0.2"); - ensemble.startStandalone(); - - List bookies = ensemble.getZkClient().getChildren("/ledgers/available", false); - Collections.sort(bookies); - assertEquals(bookies.size(), 2); - assertTrue(bookies.get(0).startsWith("127.0.0.2:")); - - ensemble.stop(); - } - @Test public void testStartStop() throws Exception { From 6a639d03dc619d4b0a6255657ddc43df21d35c57 Mon Sep 17 00:00:00 2001 From: ken <1647023764@qq.com> Date: Fri, 3 Nov 2023 17:58:18 +0800 Subject: [PATCH 140/699] [fix][txn] OpRequestSend reuse problem cause tbClient commitTxnOnTopic timeout unexpectedly (#21505) Co-authored-by: fanjianye --- .../transaction/buffer/impl/TransactionBufferHandlerImpl.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferHandlerImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferHandlerImpl.java index 625d27329d329..9aac9ab64d0fd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferHandlerImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferHandlerImpl.java @@ -137,8 +137,9 @@ public void endTxn(OpRequestSend op) { if (clientCnx.ctx().channel().isActive()) { clientCnx.registerTransactionBufferHandler(TransactionBufferHandlerImpl.this); outstandingRequests.put(op.requestId, op); + final long requestId = op.requestId; timer.newTimeout(timeout -> { - OpRequestSend peek = outstandingRequests.remove(op.requestId); + OpRequestSend peek = outstandingRequests.remove(requestId); if (peek != null && !peek.cb.isDone() && !peek.cb.isCompletedExceptionally()) { peek.cb.completeExceptionally(new TransactionBufferClientException .RequestTimeoutException()); From 14345f01706dc33936961f290a767de6c72aaaec Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Thu, 2 Nov 2023 16:33:30 +0800 Subject: [PATCH 141/699] [fix][broker] Avoid pass null role in MultiRolesTokenAuthorizationProvider (#21486) Co-authored-by: Jiwe Guo --- .../MultiRolesTokenAuthorizationProvider.java | 9 ++++- ...tiRolesTokenAuthorizationProviderTest.java | 35 ++++++++++++++++++- 2 files changed, 42 insertions(+), 2 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java index 7d17d180cf1f0..fdab233a51098 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProvider.java @@ -183,7 +183,14 @@ private Set getRoles(String role, AuthenticationDataSource authData) { Jwt jwt = parser.parseClaimsJwt(unsignedToken); try { - return new HashSet<>(Collections.singletonList(jwt.getBody().get(roleClaim, String.class))); + final String jwtRole = jwt.getBody().get(roleClaim, String.class); + if (jwtRole == null) { + if (log.isDebugEnabled()) { + log.debug("Do not have corresponding claim in jwt token. claim={}", roleClaim); + } + return Collections.emptySet(); + } + return new HashSet<>(Collections.singletonList(jwtRole)); } catch (RequiredTypeException requiredTypeException) { try { List list = jwt.getBody().get(roleClaim, List.class); diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java index c4fc35f64019e..ed9626dffe23f 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/authorization/MultiRolesTokenAuthorizationProviderTest.java @@ -32,7 +32,6 @@ import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils; import org.apache.pulsar.broker.resources.PulsarResources; import org.testng.annotations.Test; - import javax.crypto.SecretKey; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -144,6 +143,40 @@ public String getHttpHeader(String name) { }).get()); } + @Test + public void testMultiRolesAuthzWithoutClaim() throws Exception { + final SecretKey secretKey = AuthTokenUtils.createSecretKey(SignatureAlgorithm.HS256); + final String testRole = "test-role"; + // broker will use "sub" as the claim by default. + final String token = Jwts.builder() + .claim("whatever", testRole).signWith(secretKey).compact(); + ServiceConfiguration conf = new ServiceConfiguration(); + final MultiRolesTokenAuthorizationProvider provider = new MultiRolesTokenAuthorizationProvider(); + provider.initialize(conf, mock(PulsarResources.class)); + final AuthenticationDataSource ads = new AuthenticationDataSource() { + @Override + public boolean hasDataFromHttp() { + return true; + } + + @Override + public String getHttpHeader(String name) { + if (name.equals("Authorization")) { + return "Bearer " + token; + } else { + throw new IllegalArgumentException("Wrong HTTP header"); + } + } + }; + + assertFalse(provider.authorize("test", ads, role -> { + if (role == null) { + throw new IllegalStateException("We should avoid pass null to sub providers"); + } + return CompletableFuture.completedFuture(role.equals(testRole)); + }).get()); + } + @Test public void testMultiRolesAuthzWithAnonymousUser() throws Exception { @Cleanup From 12a6d60a10d23f5dda11405f5b0793ddaa0aa438 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Fri, 3 Nov 2023 19:03:10 +0800 Subject: [PATCH 142/699] [fix][broker] Fix issue with consumer read uncommitted messages from compacted topic (#21465) --- ...sistentDispatcherSingleActiveConsumer.java | 6 +- .../pulsar/compaction/CompactedTopic.java | 5 +- .../pulsar/compaction/CompactedTopicImpl.java | 3 +- .../compaction/CompactedTopicUtils.java | 10 ++-- .../broker/transaction/TransactionTest.java | 57 ++++++++++++++++++- .../compaction/CompactedTopicUtilsTest.java | 4 +- 6 files changed, 72 insertions(+), 13 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java index d96429693fda8..5e9183df0b1df 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java @@ -55,6 +55,7 @@ import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.util.Codec; import org.apache.pulsar.compaction.CompactedTopicUtils; +import org.apache.pulsar.compaction.TopicCompactionService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -350,8 +351,9 @@ protected void readMoreEntries(Consumer consumer) { havePendingRead = true; if (consumer.readCompacted()) { boolean readFromEarliest = isFirstRead && MessageId.earliest.equals(consumer.getStartMessageId()); - CompactedTopicUtils.asyncReadCompactedEntries(topic.getTopicCompactionService(), cursor, - messagesToRead, bytesToRead, readFromEarliest, this, true, consumer); + TopicCompactionService topicCompactionService = topic.getTopicCompactionService(); + CompactedTopicUtils.asyncReadCompactedEntries(topicCompactionService, cursor, messagesToRead, + bytesToRead, topic.getMaxReadPosition(), readFromEarliest, this, true, consumer); } else { ReadEntriesCtx readEntriesCtx = ReadEntriesCtx.create(consumer, consumer.getConsumerEpoch()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java index 8c17e0f3ca34d..146ba4327d252 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java @@ -24,6 +24,7 @@ import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.service.Consumer; public interface CompactedTopic { @@ -34,12 +35,14 @@ public interface CompactedTopic { * Read entries from compacted topic. * * @deprecated Use {@link CompactedTopicUtils#asyncReadCompactedEntries(TopicCompactionService, ManagedCursor, - * int, long, boolean, ReadEntriesCallback, boolean, Consumer)} instead. + * int, long, org.apache.bookkeeper.mledger.impl.PositionImpl, boolean, ReadEntriesCallback, boolean, Consumer)} + * instead. */ @Deprecated void asyncReadEntriesOrWait(ManagedCursor cursor, int maxEntries, long bytesToRead, + PositionImpl maxReadPosition, boolean isFirstRead, ReadEntriesCallback callback, Consumer consumer); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java index b028b708c49e4..8794e2736d4d4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java @@ -96,6 +96,7 @@ public CompletableFuture deleteCompactedLedger(long compactedLedgerId) { public void asyncReadEntriesOrWait(ManagedCursor cursor, int maxEntries, long bytesToRead, + PositionImpl maxReadPosition, boolean isFirstRead, ReadEntriesCallback callback, Consumer consumer) { PositionImpl cursorPosition; @@ -112,7 +113,7 @@ public void asyncReadEntriesOrWait(ManagedCursor cursor, if (currentCompactionHorizon == null || currentCompactionHorizon.compareTo(cursorPosition) < 0) { - cursor.asyncReadEntriesOrWait(maxEntries, bytesToRead, callback, readEntriesCtx, PositionImpl.LATEST); + cursor.asyncReadEntriesOrWait(maxEntries, bytesToRead, callback, readEntriesCtx, maxReadPosition); } else { ManagedCursorImpl managedCursor = (ManagedCursorImpl) cursor; int numberOfEntriesToRead = managedCursor.applyMaxSizeCap(maxEntries, bytesToRead); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java index 66bcf4c3002bd..d3464d402e9c6 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java @@ -42,8 +42,8 @@ public class CompactedTopicUtils { @Beta public static void asyncReadCompactedEntries(TopicCompactionService topicCompactionService, ManagedCursor cursor, int maxEntries, - long bytesToRead, boolean readFromEarliest, - AsyncCallbacks.ReadEntriesCallback callback, + long bytesToRead, PositionImpl maxReadPosition, + boolean readFromEarliest, AsyncCallbacks.ReadEntriesCallback callback, boolean wait, @Nullable Consumer consumer) { Objects.requireNonNull(topicCompactionService); Objects.requireNonNull(cursor); @@ -68,11 +68,9 @@ public static void asyncReadCompactedEntries(TopicCompactionService topicCompact || readPosition.compareTo( lastCompactedPosition.getLedgerId(), lastCompactedPosition.getEntryId()) > 0) { if (wait) { - cursor.asyncReadEntriesOrWait(maxEntries, bytesToRead, callback, readEntriesCtx, - PositionImpl.LATEST); + cursor.asyncReadEntriesOrWait(maxEntries, bytesToRead, callback, readEntriesCtx, maxReadPosition); } else { - cursor.asyncReadEntries(maxEntries, bytesToRead, callback, readEntriesCtx, - PositionImpl.LATEST); + cursor.asyncReadEntries(maxEntries, bytesToRead, callback, readEntriesCtx, maxReadPosition); } return CompletableFuture.completedFuture(null); } 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 ae0cc912935aa..8630b1970b12f 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 @@ -48,8 +48,8 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.Map; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -1787,6 +1787,61 @@ private void getTopic(String topicName) { }); } + @Test + public void testReadCommittedWithReadCompacted() throws Exception{ + final String namespace = "tnx/ns-prechecks"; + final String topic = "persistent://" + namespace + "/test_transaction_topic"; + admin.namespaces().createNamespace(namespace); + admin.topics().createNonPartitionedTopic(topic); + + admin.topicPolicies().setCompactionThreshold(topic, 100 * 1024 * 1024); + + @Cleanup + Consumer consumer = this.pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("sub") + .subscriptionType(SubscriptionType.Exclusive) + .readCompacted(true) + .subscribe(); + + @Cleanup + Producer producer = this.pulsarClient.newProducer(Schema.STRING) + .topic(topic) + .create(); + + producer.newMessage().key("K1").value("V1").send(); + + Transaction txn = pulsarClient.newTransaction() + .withTransactionTimeout(1, TimeUnit.MINUTES).build().get(); + producer.newMessage(txn).key("K2").value("V2").send(); + producer.newMessage(txn).key("K3").value("V3").send(); + + List messages = new ArrayList<>(); + while (true) { + Message message = consumer.receive(5, TimeUnit.SECONDS); + if (message == null) { + break; + } + messages.add(message.getValue()); + } + + Assert.assertEquals(messages, List.of("V1")); + + txn.commit(); + + messages.clear(); + + while (true) { + Message message = consumer.receive(5, TimeUnit.SECONDS); + if (message == null) { + break; + } + messages.add(message.getValue()); + } + + Assert.assertEquals(messages, List.of("V2", "V3")); + } + @DataProvider(name = "BlockTransactionsIfReplicationEnabledValues") public static Object[][] packageNamesProvider() { return new Object[][]{ diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicUtilsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicUtilsTest.java index 94f2a17a2a3f4..2545c0362e82a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicUtilsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicUtilsTest.java @@ -69,8 +69,8 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { } }; - CompactedTopicUtils.asyncReadCompactedEntries(service, cursor, 1, 100, false, - readEntriesCallback, false, null); + CompactedTopicUtils.asyncReadCompactedEntries(service, cursor, 1, 100, + PositionImpl.LATEST, false, readEntriesCallback, false, null); List entries = completableFuture.get(); Assert.assertTrue(entries.isEmpty()); From 7a7145177c975ea72fa3be84a4edb3619bda907e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 1 Nov 2023 20:38:48 +0200 Subject: [PATCH 143/699] [fix][broker] Fix PulsarService/BrokerService shutdown when brokerShutdownTimeoutMs=0 (#21496) --- .../java/org/apache/pulsar/broker/PulsarService.java | 10 +++++++--- .../apache/pulsar/broker/service/BrokerService.java | 9 ++++++++- 2 files changed, 15 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 6747bbb916d93..a366cf25aa023 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 @@ -640,14 +640,18 @@ private synchronized void resetMetricsServlet() { } private CompletableFuture addTimeoutHandling(CompletableFuture future) { + long brokerShutdownTimeoutMs = getConfiguration().getBrokerShutdownTimeoutMs(); + if (brokerShutdownTimeoutMs <= 0) { + return future; + } ScheduledExecutorService shutdownExecutor = Executors.newSingleThreadScheduledExecutor( new ExecutorProvider.ExtendedThreadFactory(getClass().getSimpleName() + "-shutdown")); FutureUtil.addTimeoutHandling(future, - Duration.ofMillis(Math.max(1L, getConfiguration().getBrokerShutdownTimeoutMs())), + Duration.ofMillis(brokerShutdownTimeoutMs), shutdownExecutor, () -> FutureUtil.createTimeoutException("Timeout in close", getClass(), "close")); future.handle((v, t) -> { - if (t != null && getConfiguration().getBrokerShutdownTimeoutMs() > 0) { - LOG.info("Shutdown timed out after {} ms", getConfiguration().getBrokerShutdownTimeoutMs()); + if (t instanceof TimeoutException) { + LOG.info("Shutdown timed out after {} ms", brokerShutdownTimeoutMs); LOG.info(ThreadDumpUtil.buildThreadDiagnosticString()); } // shutdown the shutdown executor 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 4fe4b6dcd5bea..a01154552dea2 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 @@ -825,6 +825,7 @@ public CompletableFuture closeAsync() { for (EventLoopGroup group : protocolHandlersWorkerGroups) { shutdownEventLoops.add(shutdownEventLoopGracefully(group)); } + CompletableFuture shutdownFuture = CompletableFuture.allOf(shutdownEventLoops.toArray(new CompletableFuture[0])) .handle((v, t) -> { @@ -835,7 +836,7 @@ public CompletableFuture closeAsync() { } return null; }) - .thenCompose(__ -> { + .thenComposeAsync(__ -> { log.info("Continuing to second phase in shutdown."); List> asyncCloseFutures = new ArrayList<>(); @@ -899,6 +900,12 @@ public CompletableFuture closeAsync() { return null; }); return combined; + }, runnable -> { + // run the 2nd phase of the shutdown in a separate thread + Thread thread = new Thread(runnable); + thread.setName("BrokerService-shutdown-phase2"); + thread.setDaemon(false); + thread.start(); }); FutureUtil.whenCancelledOrTimedOut(shutdownFuture, () -> cancellableDownstreamFutureReference .thenAccept(future -> future.cancel(false))); From 20c1ff8b01422c880def916afa1938b25d202d85 Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Wed, 8 Nov 2023 12:22:36 +0800 Subject: [PATCH 144/699] [fix][broker] Fix namespace bundle stuck in unloading status (#21445) PR https://github.com/apache/pulsar/pull/21231 made user topic creation rely on system topic `__change_event` if the user is enabling `topicLevelPoliciesEnabled`. It will introduce a race condition with namespace bundle unloading. All creating topics want to create `__change_event` reader but constantly fail by namespace-bundle inactive and retry mechanism. Unfortunately, the namespace-bundle unloading operation is waiting for all the topics to be completed and then release inactive status. Therefore, they will be stuck in a deadlock until one gets a timeout. - Get the topic policy before loading. --- .../pulsar/broker/service/BrokerService.java | 327 ++++++++++-------- .../SystemTopicBasedTopicPoliciesService.java | 19 +- .../broker/admin/TopicAutoCreationTest.java | 8 +- .../namespace/NamespaceUnloadingTest.java | 29 ++ .../broker/service/BrokerServiceTest.java | 4 +- ...temTopicBasedTopicPoliciesServiceTest.java | 29 -- .../client/cli/PulsarClientToolTest.java | 3 + 7 files changed, 223 insertions(+), 196 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 a01154552dea2..748536b43cd3d 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 @@ -49,6 +49,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.concurrent.CancellationException; @@ -70,6 +71,7 @@ import java.util.function.Consumer; import java.util.function.Predicate; import javax.annotation.Nonnull; +import javax.annotation.Nullable; import lombok.AccessLevel; import lombok.AllArgsConstructor; import lombok.Getter; @@ -1051,19 +1053,32 @@ public CompletableFuture> getTopic(final TopicName topicName, bo } final boolean isPersistentTopic = topicName.getDomain().equals(TopicDomain.persistent); if (isPersistentTopic) { - return topics.computeIfAbsent(topicName.toString(), (tpName) -> { - if (topicName.isPartitioned()) { - return fetchPartitionedTopicMetadataAsync(TopicName.get(topicName.getPartitionedTopicName())) - .thenCompose((metadata) -> { - // Allow crate non-partitioned persistent topic that name includes `partition` - if (metadata.partitions == 0 - || topicName.getPartitionIndex() < metadata.partitions) { - return loadOrCreatePersistentTopic(tpName, createIfMissing, properties); - } - return CompletableFuture.completedFuture(Optional.empty()); - }); - } - return loadOrCreatePersistentTopic(tpName, createIfMissing, properties); + final CompletableFuture> topicPoliciesFuture = + getTopicPoliciesBypassSystemTopic(topicName); + return topicPoliciesFuture.exceptionally(ex -> { + final Throwable rc = FutureUtil.unwrapCompletionException(ex); + final String errorInfo = String.format("Topic creation encountered an exception by initialize" + + " topic policies service. topic_name=%s error_message=%s", topicName, rc.getMessage()); + log.error(errorInfo, rc); + throw FutureUtil.wrapToCompletionException(new ServiceUnitNotReadyException(errorInfo)); + }).thenCompose(optionalTopicPolicies -> { + final TopicPolicies topicPolicies = optionalTopicPolicies.orElse(null); + return topics.computeIfAbsent(topicName.toString(), (tpName) -> { + if (topicName.isPartitioned()) { + final TopicName topicNameEntity = TopicName.get(topicName.getPartitionedTopicName()); + return fetchPartitionedTopicMetadataAsync(topicNameEntity) + .thenCompose((metadata) -> { + // Allow crate non-partitioned persistent topic that name includes `partition` + if (metadata.partitions == 0 + || topicName.getPartitionIndex() < metadata.partitions) { + return loadOrCreatePersistentTopic(tpName, createIfMissing, + properties, topicPolicies); + } + return CompletableFuture.completedFuture(Optional.empty()); + }); + } + return loadOrCreatePersistentTopic(tpName, createIfMissing, properties, topicPolicies); + }); }); } else { return topics.computeIfAbsent(topicName.toString(), (name) -> { @@ -1117,6 +1132,18 @@ public CompletableFuture> getTopic(final TopicName topicName, bo } } + private CompletableFuture> getTopicPoliciesBypassSystemTopic(@Nonnull TopicName topicName) { + Objects.requireNonNull(topicName); + final ServiceConfiguration serviceConfiguration = pulsar.getConfiguration(); + if (serviceConfiguration.isSystemTopicEnabled() && serviceConfiguration.isTopicLevelPoliciesEnabled() + && !NamespaceService.isSystemServiceNamespace(topicName.getNamespace()) + && !SystemTopicNames.isTopicPoliciesSystemTopic(topicName.toString())) { + return pulsar.getTopicPoliciesService().getTopicPoliciesAsync(topicName); + } else { + return CompletableFuture.completedFuture(Optional.empty()); + } + } + public CompletableFuture deleteTopic(String topic, boolean forceDelete) { topicEventsDispatcher.notify(topic, TopicEvent.DELETE, EventStage.BEFORE); CompletableFuture result = deleteTopicInternal(topic, forceDelete); @@ -1506,7 +1533,7 @@ public PulsarAdmin getClusterPulsarAdmin(String cluster, Optional c * @throws RuntimeException */ protected CompletableFuture> loadOrCreatePersistentTopic(final String topic, - boolean createIfMissing, Map properties) throws RuntimeException { + boolean createIfMissing, Map properties, @Nullable TopicPolicies topicPolicies) { final CompletableFuture> topicFuture = FutureUtil.createFutureWithTimeout( Duration.ofSeconds(pulsar.getConfiguration().getTopicLoadTimeoutSeconds()), executor(), () -> FAILED_TO_LOAD_TOPIC_TIMEOUT_EXCEPTION); @@ -1524,7 +1551,8 @@ protected CompletableFuture> loadOrCreatePersistentTopic(final S final Semaphore topicLoadSemaphore = topicLoadRequestSemaphore.get(); if (topicLoadSemaphore.tryAcquire()) { - checkOwnershipAndCreatePersistentTopic(topic, createIfMissing, topicFuture, properties); + checkOwnershipAndCreatePersistentTopic(topic, createIfMissing, topicFuture, + properties, topicPolicies); topicFuture.handle((persistentTopic, ex) -> { // release permit and process pending topic topicLoadSemaphore.release(); @@ -1533,7 +1561,7 @@ protected CompletableFuture> loadOrCreatePersistentTopic(final S }); } else { pendingTopicLoadingQueue.add(new TopicLoadingContext(topic, - createIfMissing, topicFuture, properties)); + createIfMissing, topicFuture, properties, topicPolicies)); if (log.isDebugEnabled()) { log.debug("topic-loading for {} added into pending queue", topic); } @@ -1574,7 +1602,7 @@ protected CompletableFuture> fetchTopicPropertiesAsync(Topic private void checkOwnershipAndCreatePersistentTopic(final String topic, boolean createIfMissing, CompletableFuture> topicFuture, - Map properties) { + Map properties, @Nullable TopicPolicies topicPolicies) { TopicName topicName = TopicName.get(topic); pulsar.getNamespaceService().isServiceUnitActiveAsync(topicName) .thenAccept(isActive -> { @@ -1588,7 +1616,8 @@ private void checkOwnershipAndCreatePersistentTopic(final String topic, boolean } propertiesFuture.thenAccept(finalProperties -> //TODO add topicName in properties? - createPersistentTopic(topic, createIfMissing, topicFuture, finalProperties) + createPersistentTopic(topic, createIfMissing, topicFuture, + finalProperties, topicPolicies) ).exceptionally(throwable -> { log.warn("[{}] Read topic property failed", topic, throwable); pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); @@ -1613,12 +1642,12 @@ private void checkOwnershipAndCreatePersistentTopic(final String topic, boolean public void createPersistentTopic0(final String topic, boolean createIfMissing, CompletableFuture> topicFuture, Map properties) { - createPersistentTopic(topic, createIfMissing, topicFuture, properties); + createPersistentTopic(topic, createIfMissing, topicFuture, properties, null); } private void createPersistentTopic(final String topic, boolean createIfMissing, CompletableFuture> topicFuture, - Map properties) { + Map properties, @Nullable TopicPolicies topicPolicies) { TopicName topicName = TopicName.get(topic); final long topicCreateTimeMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()); @@ -1768,121 +1797,119 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { } public CompletableFuture getManagedLedgerConfig(@Nonnull TopicName topicName) { + final CompletableFuture> topicPoliciesFuture = + getTopicPoliciesBypassSystemTopic(topicName); + return topicPoliciesFuture.thenCompose(optionalTopicPolicies -> + getManagedLedgerConfig(topicName, optionalTopicPolicies.orElse(null))); + } + + private CompletableFuture getManagedLedgerConfig(@Nonnull TopicName topicName, + @Nullable TopicPolicies topicPolicies) { requireNonNull(topicName); NamespaceName namespace = topicName.getNamespaceObject(); ServiceConfiguration serviceConfig = pulsar.getConfiguration(); NamespaceResources nsr = pulsar.getPulsarResources().getNamespaceResources(); LocalPoliciesResources lpr = pulsar.getPulsarResources().getLocalPolicies(); - final CompletableFuture> topicPoliciesFuture; - if (pulsar.getConfig().isTopicLevelPoliciesEnabled() - && !NamespaceService.isSystemServiceNamespace(namespace.toString()) - && !SystemTopicNames.isTopicPoliciesSystemTopic(topicName.toString())) { - topicPoliciesFuture = pulsar.getTopicPoliciesService().getTopicPoliciesAsync(topicName); - } else { - topicPoliciesFuture = CompletableFuture.completedFuture(Optional.empty()); - } - return topicPoliciesFuture.thenCompose(topicPoliciesOptional -> { - final CompletableFuture> nsPolicies = nsr.getPoliciesAsync(namespace); - final CompletableFuture> lcPolicies = lpr.getLocalPoliciesAsync(namespace); - return nsPolicies.thenCombine(lcPolicies, (policies, localPolicies) -> { - PersistencePolicies persistencePolicies = null; - RetentionPolicies retentionPolicies = null; - OffloadPoliciesImpl topicLevelOffloadPolicies = null; - if (topicPoliciesOptional.isPresent()) { - final TopicPolicies topicPolicies = topicPoliciesOptional.get(); - persistencePolicies = topicPolicies.getPersistence(); - retentionPolicies = topicPolicies.getRetentionPolicies(); - topicLevelOffloadPolicies = topicPolicies.getOffloadPolicies(); - } - - if (persistencePolicies == null) { - persistencePolicies = policies.map(p -> p.persistence).orElseGet( - () -> new PersistencePolicies(serviceConfig.getManagedLedgerDefaultEnsembleSize(), - serviceConfig.getManagedLedgerDefaultWriteQuorum(), - serviceConfig.getManagedLedgerDefaultAckQuorum(), - serviceConfig.getManagedLedgerDefaultMarkDeleteRateLimit())); - } + final CompletableFuture> nsPolicies = nsr.getPoliciesAsync(namespace); + final CompletableFuture> lcPolicies = lpr.getLocalPoliciesAsync(namespace); + return nsPolicies.thenCombine(lcPolicies, (policies, localPolicies) -> { + PersistencePolicies persistencePolicies = null; + RetentionPolicies retentionPolicies = null; + OffloadPoliciesImpl topicLevelOffloadPolicies = null; + if (topicPolicies != null) { + persistencePolicies = topicPolicies.getPersistence(); + retentionPolicies = topicPolicies.getRetentionPolicies(); + topicLevelOffloadPolicies = topicPolicies.getOffloadPolicies(); + } - if (retentionPolicies == null) { - retentionPolicies = policies.map(p -> p.retention_policies).orElseGet( - () -> new RetentionPolicies(serviceConfig.getDefaultRetentionTimeInMinutes(), - serviceConfig.getDefaultRetentionSizeInMB()) - ); - } + if (persistencePolicies == null) { + persistencePolicies = policies.map(p -> p.persistence).orElseGet( + () -> new PersistencePolicies(serviceConfig.getManagedLedgerDefaultEnsembleSize(), + serviceConfig.getManagedLedgerDefaultWriteQuorum(), + serviceConfig.getManagedLedgerDefaultAckQuorum(), + serviceConfig.getManagedLedgerDefaultMarkDeleteRateLimit())); + } - ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); - managedLedgerConfig.setEnsembleSize(persistencePolicies.getBookkeeperEnsemble()); - managedLedgerConfig.setWriteQuorumSize(persistencePolicies.getBookkeeperWriteQuorum()); - managedLedgerConfig.setAckQuorumSize(persistencePolicies.getBookkeeperAckQuorum()); + if (retentionPolicies == null) { + retentionPolicies = policies.map(p -> p.retention_policies).orElseGet( + () -> new RetentionPolicies(serviceConfig.getDefaultRetentionTimeInMinutes(), + serviceConfig.getDefaultRetentionSizeInMB()) + ); + } - if (serviceConfig.isStrictBookieAffinityEnabled()) { + ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); + managedLedgerConfig.setEnsembleSize(persistencePolicies.getBookkeeperEnsemble()); + managedLedgerConfig.setWriteQuorumSize(persistencePolicies.getBookkeeperWriteQuorum()); + managedLedgerConfig.setAckQuorumSize(persistencePolicies.getBookkeeperAckQuorum()); + + if (serviceConfig.isStrictBookieAffinityEnabled()) { + managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyClassName( + IsolatedBookieEnsemblePlacementPolicy.class); + if (localPolicies.isPresent() && localPolicies.get().bookieAffinityGroup != null) { + Map properties = new HashMap<>(); + properties.put(IsolatedBookieEnsemblePlacementPolicy.ISOLATION_BOOKIE_GROUPS, + localPolicies.get().bookieAffinityGroup.getBookkeeperAffinityGroupPrimary()); + properties.put(IsolatedBookieEnsemblePlacementPolicy.SECONDARY_ISOLATION_BOOKIE_GROUPS, + localPolicies.get().bookieAffinityGroup.getBookkeeperAffinityGroupSecondary()); + managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyProperties(properties); + } else if (isSystemTopic(topicName)) { + Map properties = new HashMap<>(); + properties.put(IsolatedBookieEnsemblePlacementPolicy.ISOLATION_BOOKIE_GROUPS, "*"); + properties.put(IsolatedBookieEnsemblePlacementPolicy + .SECONDARY_ISOLATION_BOOKIE_GROUPS, "*"); + managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyProperties(properties); + } else { + Map properties = new HashMap<>(); + properties.put(IsolatedBookieEnsemblePlacementPolicy.ISOLATION_BOOKIE_GROUPS, ""); + properties.put(IsolatedBookieEnsemblePlacementPolicy.SECONDARY_ISOLATION_BOOKIE_GROUPS, ""); + managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyProperties(properties); + } + } else { + if (localPolicies.isPresent() && localPolicies.get().bookieAffinityGroup != null) { managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyClassName( IsolatedBookieEnsemblePlacementPolicy.class); - if (localPolicies.isPresent() && localPolicies.get().bookieAffinityGroup != null) { - Map properties = new HashMap<>(); - properties.put(IsolatedBookieEnsemblePlacementPolicy.ISOLATION_BOOKIE_GROUPS, - localPolicies.get().bookieAffinityGroup.getBookkeeperAffinityGroupPrimary()); - properties.put(IsolatedBookieEnsemblePlacementPolicy.SECONDARY_ISOLATION_BOOKIE_GROUPS, - localPolicies.get().bookieAffinityGroup.getBookkeeperAffinityGroupSecondary()); - managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyProperties(properties); - } else if (isSystemTopic(topicName)) { - Map properties = new HashMap<>(); - properties.put(IsolatedBookieEnsemblePlacementPolicy.ISOLATION_BOOKIE_GROUPS, "*"); - properties.put(IsolatedBookieEnsemblePlacementPolicy - .SECONDARY_ISOLATION_BOOKIE_GROUPS, "*"); - managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyProperties(properties); - } else { - Map properties = new HashMap<>(); - properties.put(IsolatedBookieEnsemblePlacementPolicy.ISOLATION_BOOKIE_GROUPS, ""); - properties.put(IsolatedBookieEnsemblePlacementPolicy.SECONDARY_ISOLATION_BOOKIE_GROUPS, ""); - managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyProperties(properties); - } - } else { - if (localPolicies.isPresent() && localPolicies.get().bookieAffinityGroup != null) { - managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyClassName( - IsolatedBookieEnsemblePlacementPolicy.class); - Map properties = new HashMap<>(); - properties.put(IsolatedBookieEnsemblePlacementPolicy.ISOLATION_BOOKIE_GROUPS, - localPolicies.get().bookieAffinityGroup.getBookkeeperAffinityGroupPrimary()); - properties.put(IsolatedBookieEnsemblePlacementPolicy.SECONDARY_ISOLATION_BOOKIE_GROUPS, - localPolicies.get().bookieAffinityGroup.getBookkeeperAffinityGroupSecondary()); - managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyProperties(properties); - } + Map properties = new HashMap<>(); + properties.put(IsolatedBookieEnsemblePlacementPolicy.ISOLATION_BOOKIE_GROUPS, + localPolicies.get().bookieAffinityGroup.getBookkeeperAffinityGroupPrimary()); + properties.put(IsolatedBookieEnsemblePlacementPolicy.SECONDARY_ISOLATION_BOOKIE_GROUPS, + localPolicies.get().bookieAffinityGroup.getBookkeeperAffinityGroupSecondary()); + managedLedgerConfig.setBookKeeperEnsemblePlacementPolicyProperties(properties); } + } - managedLedgerConfig.setThrottleMarkDelete(persistencePolicies.getManagedLedgerMaxMarkDeleteRate()); - managedLedgerConfig.setDigestType(serviceConfig.getManagedLedgerDigestType()); - managedLedgerConfig.setPassword(serviceConfig.getManagedLedgerPassword()); - - managedLedgerConfig - .setMaxUnackedRangesToPersist(serviceConfig.getManagedLedgerMaxUnackedRangesToPersist()); - managedLedgerConfig.setPersistentUnackedRangesWithMultipleEntriesEnabled( - serviceConfig.isPersistentUnackedRangesWithMultipleEntriesEnabled()); - managedLedgerConfig.setMaxUnackedRangesToPersistInMetadataStore( - serviceConfig.getManagedLedgerMaxUnackedRangesToPersistInMetadataStore()); - managedLedgerConfig.setMaxEntriesPerLedger(serviceConfig.getManagedLedgerMaxEntriesPerLedger()); - managedLedgerConfig - .setMinimumRolloverTime(serviceConfig.getManagedLedgerMinLedgerRolloverTimeMinutes(), - TimeUnit.MINUTES); - managedLedgerConfig - .setMaximumRolloverTime(serviceConfig.getManagedLedgerMaxLedgerRolloverTimeMinutes(), - TimeUnit.MINUTES); - managedLedgerConfig.setMaxSizePerLedgerMb(serviceConfig.getManagedLedgerMaxSizePerLedgerMbytes()); - - managedLedgerConfig.setMetadataOperationsTimeoutSeconds( - serviceConfig.getManagedLedgerMetadataOperationsTimeoutSeconds()); - managedLedgerConfig - .setReadEntryTimeoutSeconds(serviceConfig.getManagedLedgerReadEntryTimeoutSeconds()); - managedLedgerConfig - .setAddEntryTimeoutSeconds(serviceConfig.getManagedLedgerAddEntryTimeoutSeconds()); - managedLedgerConfig.setMetadataEnsembleSize(serviceConfig.getManagedLedgerDefaultEnsembleSize()); - managedLedgerConfig.setUnackedRangesOpenCacheSetEnabled( - serviceConfig.isManagedLedgerUnackedRangesOpenCacheSetEnabled()); - managedLedgerConfig.setMetadataWriteQuorumSize(serviceConfig.getManagedLedgerDefaultWriteQuorum()); - managedLedgerConfig.setMetadataAckQuorumSize(serviceConfig.getManagedLedgerDefaultAckQuorum()); - managedLedgerConfig - .setMetadataMaxEntriesPerLedger(serviceConfig.getManagedLedgerCursorMaxEntriesPerLedger()); + managedLedgerConfig.setThrottleMarkDelete(persistencePolicies.getManagedLedgerMaxMarkDeleteRate()); + managedLedgerConfig.setDigestType(serviceConfig.getManagedLedgerDigestType()); + managedLedgerConfig.setPassword(serviceConfig.getManagedLedgerPassword()); + + managedLedgerConfig + .setMaxUnackedRangesToPersist(serviceConfig.getManagedLedgerMaxUnackedRangesToPersist()); + managedLedgerConfig.setPersistentUnackedRangesWithMultipleEntriesEnabled( + serviceConfig.isPersistentUnackedRangesWithMultipleEntriesEnabled()); + managedLedgerConfig.setMaxUnackedRangesToPersistInMetadataStore( + serviceConfig.getManagedLedgerMaxUnackedRangesToPersistInMetadataStore()); + managedLedgerConfig.setMaxEntriesPerLedger(serviceConfig.getManagedLedgerMaxEntriesPerLedger()); + managedLedgerConfig + .setMinimumRolloverTime(serviceConfig.getManagedLedgerMinLedgerRolloverTimeMinutes(), + TimeUnit.MINUTES); + managedLedgerConfig + .setMaximumRolloverTime(serviceConfig.getManagedLedgerMaxLedgerRolloverTimeMinutes(), + TimeUnit.MINUTES); + managedLedgerConfig.setMaxSizePerLedgerMb(serviceConfig.getManagedLedgerMaxSizePerLedgerMbytes()); + + managedLedgerConfig.setMetadataOperationsTimeoutSeconds( + serviceConfig.getManagedLedgerMetadataOperationsTimeoutSeconds()); + managedLedgerConfig + .setReadEntryTimeoutSeconds(serviceConfig.getManagedLedgerReadEntryTimeoutSeconds()); + managedLedgerConfig + .setAddEntryTimeoutSeconds(serviceConfig.getManagedLedgerAddEntryTimeoutSeconds()); + managedLedgerConfig.setMetadataEnsembleSize(serviceConfig.getManagedLedgerDefaultEnsembleSize()); + managedLedgerConfig.setUnackedRangesOpenCacheSetEnabled( + serviceConfig.isManagedLedgerUnackedRangesOpenCacheSetEnabled()); + managedLedgerConfig.setMetadataWriteQuorumSize(serviceConfig.getManagedLedgerDefaultWriteQuorum()); + managedLedgerConfig.setMetadataAckQuorumSize(serviceConfig.getManagedLedgerDefaultAckQuorum()); + managedLedgerConfig + .setMetadataMaxEntriesPerLedger(serviceConfig.getManagedLedgerCursorMaxEntriesPerLedger()); managedLedgerConfig .setLedgerRolloverTimeout(serviceConfig.getManagedLedgerCursorRolloverTimeInSeconds()); @@ -1906,36 +1933,35 @@ public CompletableFuture getManagedLedgerConfig(@Nonnull To managedLedgerConfig.setMaxBacklogBetweenCursorsForCaching( serviceConfig.getManagedLedgerMaxBacklogBetweenCursorsForCaching()); - OffloadPoliciesImpl nsLevelOffloadPolicies = - (OffloadPoliciesImpl) policies.map(p -> p.offload_policies).orElse(null); - OffloadPoliciesImpl offloadPolicies = OffloadPoliciesImpl.mergeConfiguration( - topicLevelOffloadPolicies, - OffloadPoliciesImpl.oldPoliciesCompatible(nsLevelOffloadPolicies, policies.orElse(null)), - getPulsar().getConfig().getProperties()); - if (NamespaceService.isSystemServiceNamespace(namespace.toString())) { - managedLedgerConfig.setLedgerOffloader(NullLedgerOffloader.INSTANCE); - } else { - if (topicLevelOffloadPolicies != null) { - try { - LedgerOffloader topicLevelLedgerOffLoader = - pulsar().createManagedLedgerOffloader(offloadPolicies); - managedLedgerConfig.setLedgerOffloader(topicLevelLedgerOffLoader); - } catch (PulsarServerException e) { - throw new RuntimeException(e); - } - } else { - //If the topic level policy is null, use the namespace level - managedLedgerConfig - .setLedgerOffloader(pulsar.getManagedLedgerOffloader(namespace, offloadPolicies)); + OffloadPoliciesImpl nsLevelOffloadPolicies = + (OffloadPoliciesImpl) policies.map(p -> p.offload_policies).orElse(null); + OffloadPoliciesImpl offloadPolicies = OffloadPoliciesImpl.mergeConfiguration( + topicLevelOffloadPolicies, + OffloadPoliciesImpl.oldPoliciesCompatible(nsLevelOffloadPolicies, policies.orElse(null)), + getPulsar().getConfig().getProperties()); + if (NamespaceService.isSystemServiceNamespace(namespace.toString())) { + managedLedgerConfig.setLedgerOffloader(NullLedgerOffloader.INSTANCE); + } else { + if (topicLevelOffloadPolicies != null) { + try { + LedgerOffloader topicLevelLedgerOffLoader = + pulsar().createManagedLedgerOffloader(offloadPolicies); + managedLedgerConfig.setLedgerOffloader(topicLevelLedgerOffLoader); + } catch (PulsarServerException e) { + throw new RuntimeException(e); } + } else { + //If the topic level policy is null, use the namespace level + managedLedgerConfig + .setLedgerOffloader(pulsar.getManagedLedgerOffloader(namespace, offloadPolicies)); } + } - managedLedgerConfig.setDeletionAtBatchIndexLevelEnabled( - serviceConfig.isAcknowledgmentAtBatchIndexLevelEnabled()); - managedLedgerConfig.setNewEntriesCheckDelayInMillis( - serviceConfig.getManagedLedgerNewEntriesCheckDelayInMillis()); - return managedLedgerConfig; - }); + managedLedgerConfig.setDeletionAtBatchIndexLevelEnabled( + serviceConfig.isAcknowledgmentAtBatchIndexLevelEnabled()); + managedLedgerConfig.setNewEntriesCheckDelayInMillis( + serviceConfig.getManagedLedgerNewEntriesCheckDelayInMillis()); + return managedLedgerConfig; }); } @@ -3044,7 +3070,7 @@ private void createPendingLoadTopic() { checkOwnershipAndCreatePersistentTopic(topic, pendingTopic.isCreateIfMissing(), pendingFuture, - pendingTopic.getProperties()); + pendingTopic.getProperties(), pendingTopic.getTopicPolicies()); pendingFuture.handle((persistentTopic, ex) -> { // release permit and process next pending topic if (acquiredPermit) { @@ -3605,5 +3631,6 @@ private static class TopicLoadingContext { private final boolean createIfMissing; private final CompletableFuture> topicFuture; private final Map properties; + private final TopicPolicies topicPolicies; } } 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 da31234095446..80fecbe67b646 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 @@ -29,7 +29,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import javax.annotation.Nonnull; import org.apache.commons.lang3.tuple.MutablePair; @@ -43,10 +42,8 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.impl.Backoff; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.client.impl.TopicMessageImpl; -import org.apache.pulsar.client.util.RetryUtil; import org.apache.pulsar.common.events.ActionType; import org.apache.pulsar.common.events.EventType; import org.apache.pulsar.common.events.PulsarEvent; @@ -320,7 +317,7 @@ public CompletableFuture addOwnedNamespaceBundleAsync(NamespaceBundle name requireNonNull(namespace); return policyCacheInitMap.computeIfAbsent(namespace, (k) -> { final CompletableFuture> readerCompletableFuture = - createSystemTopicClientWithRetry(namespace); + createSystemTopicClient(namespace); readerCaches.put(namespace, readerCompletableFuture); ownedBundlesCountPerNamespace.putIfAbsent(namespace, new AtomicInteger(1)); final CompletableFuture initFuture = readerCompletableFuture @@ -346,20 +343,16 @@ public CompletableFuture addOwnedNamespaceBundleAsync(NamespaceBundle name }); } - protected CompletableFuture> createSystemTopicClientWithRetry( + protected CompletableFuture> createSystemTopicClient( NamespaceName namespace) { - CompletableFuture> result = new CompletableFuture<>(); try { createSystemTopicFactoryIfNeeded(); - } catch (PulsarServerException e) { - result.completeExceptionally(e); - return result; + } catch (PulsarServerException ex) { + return FutureUtil.failedFuture(ex); } - SystemTopicClient systemTopicClient = namespaceEventsSystemTopicFactory + final SystemTopicClient systemTopicClient = namespaceEventsSystemTopicFactory .createTopicPoliciesSystemTopicClient(namespace); - Backoff backoff = new Backoff(1, TimeUnit.SECONDS, 3, TimeUnit.SECONDS, 10, TimeUnit.SECONDS); - RetryUtil.retryAsynchronously(systemTopicClient::newReaderAsync, backoff, pulsarService.getExecutor(), result); - return result; + return systemTopicClient.newReaderAsync(); } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java index d447787d1b7cb..590edc2d3f3bb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java @@ -135,6 +135,8 @@ public void testPartitionedTopicAutoCreationForbiddenDuringNamespaceDeletion() new InetSocketAddress(pulsar.getAdvertisedAddress(), pulsar.getBrokerListenPort().get()); return CompletableFuture.completedFuture(Pair.of(brokerAddress, brokerAddress)); }); + final String topicPoliciesServiceInitException + = "Topic creation encountered an exception by initialize topic policies service"; // Creating a producer and creating a Consumer may trigger automatic topic // creation, let's try to create a Producer and a Consumer @@ -145,7 +147,8 @@ public void testPartitionedTopicAutoCreationForbiddenDuringNamespaceDeletion() } catch (PulsarClientException.LookupException expected) { String msg = "Namespace bundle for topic (%s) not served by this instance"; log.info("Expected error", expected); - assertTrue(expected.getMessage().contains(String.format(msg, topic))); + assertTrue(expected.getMessage().contains(String.format(msg, topic)) + || expected.getMessage().contains(topicPoliciesServiceInitException)); } try (Consumer ignored = pulsarClient.newConsumer() @@ -155,7 +158,8 @@ public void testPartitionedTopicAutoCreationForbiddenDuringNamespaceDeletion() } catch (PulsarClientException.LookupException expected) { String msg = "Namespace bundle for topic (%s) not served by this instance"; log.info("Expected error", expected); - assertTrue(expected.getMessage().contains(String.format(msg, topic))); + assertTrue(expected.getMessage().contains(String.format(msg, topic)) + || expected.getMessage().contains(topicPoliciesServiceInitException)); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceUnloadingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceUnloadingTest.java index 0dbfe1760879a..1526611874a62 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceUnloadingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceUnloadingTest.java @@ -22,8 +22,12 @@ import com.google.common.collect.Sets; +import lombok.Cleanup; import org.apache.pulsar.broker.service.BrokerTestBase; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -34,6 +38,9 @@ public class NamespaceUnloadingTest extends BrokerTestBase { @BeforeMethod @Override protected void setup() throws Exception { + conf.setTopicLevelPoliciesEnabled(true); + conf.setForceDeleteNamespaceAllowed(true); + conf.setTopicLoadTimeoutSeconds(Integer.MAX_VALUE); super.baseSetup(); } @@ -68,4 +75,26 @@ public void testUnloadPartiallyLoadedNamespace() throws Exception { producer.close(); } + @Test + public void testUnloadWithTopicCreation() throws PulsarAdminException, PulsarClientException { + final String namespaceName = "prop/ns_unloading"; + final String topicName = "persistent://prop/ns_unloading/with_topic_creation"; + final int partitions = 5; + admin.namespaces().createNamespace(namespaceName, 1); + admin.topics().createPartitionedTopic(topicName, partitions); + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.INT32) + .topic(topicName) + .create(); + + for (int i = 0; i < 100; i++) { + admin.namespaces().unloadNamespaceBundle(namespaceName, "0x00000000_0xffffffff"); + } + + for (int i = 0; i < partitions; i++) { + producer.send(i); + } + admin.namespaces().deleteNamespace(namespaceName, true); + } + } 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 6c0bc6efec758..e0f2fadc2b633 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 @@ -1147,7 +1147,7 @@ public void testTopicLoadingOnDisableNamespaceBundle() throws Exception { // try to create topic which should fail as bundle is disable CompletableFuture> futureResult = pulsar.getBrokerService() - .loadOrCreatePersistentTopic(topicName, true, null); + .loadOrCreatePersistentTopic(topicName, true, null, null); try { futureResult.get(); @@ -1191,7 +1191,7 @@ public void testConcurrentLoadTopicExceedLimitShouldNotBeAutoCreated() throws Ex for (int i = 0; i < 10; i++) { // try to create topic which should fail as bundle is disable CompletableFuture> futureResult = pulsar.getBrokerService() - .loadOrCreatePersistentTopic(topicName + "_" + i, false, null); + .loadOrCreatePersistentTopic(topicName + "_" + i, false, null, null); loadFutures.add(futureResult); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java index 5b70ff996756e..ba5e42867d31f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java @@ -18,9 +18,6 @@ */ package org.apache.pulsar.broker.service; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.testng.AssertJUnit.assertEquals; import static org.testng.AssertJUnit.assertNotNull; @@ -43,11 +40,8 @@ import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.BrokerServiceException.TopicPoliciesCacheNotInitException; -import org.apache.pulsar.broker.systopic.NamespaceEventsSystemTopicFactory; import org.apache.pulsar.broker.systopic.SystemTopicClient; -import org.apache.pulsar.broker.systopic.TopicPoliciesSystemTopicClient; import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.Backoff; import org.apache.pulsar.client.impl.BackoffBuilder; import org.apache.pulsar.common.events.PulsarEvent; @@ -56,7 +50,6 @@ import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicPolicies; -import org.apache.pulsar.common.util.FutureUtil; import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.mockito.Mockito; @@ -321,28 +314,6 @@ public void testGetPolicyTimeout() throws Exception { assertTrue("actual:" + cost, cost >= 5000 - 1000); } - @Test - public void testCreatSystemTopicClientWithRetry() throws Exception { - SystemTopicBasedTopicPoliciesService service = - spy((SystemTopicBasedTopicPoliciesService) pulsar.getTopicPoliciesService()); - Field field = SystemTopicBasedTopicPoliciesService.class - .getDeclaredField("namespaceEventsSystemTopicFactory"); - field.setAccessible(true); - NamespaceEventsSystemTopicFactory factory = spy((NamespaceEventsSystemTopicFactory) field.get(service)); - SystemTopicClient client = mock(TopicPoliciesSystemTopicClient.class); - doReturn(client).when(factory).createTopicPoliciesSystemTopicClient(any()); - field.set(service, factory); - - SystemTopicClient.Reader reader = mock(SystemTopicClient.Reader.class); - // Throw an exception first, create successfully after retrying - doReturn(FutureUtil.failedFuture(new PulsarClientException("test"))) - .doReturn(CompletableFuture.completedFuture(reader)).when(client).newReaderAsync(); - - SystemTopicClient.Reader reader1 = service.createSystemTopicClientWithRetry(null).get(); - - assertEquals(reader1, reader); - } - @Test public void testGetTopicPoliciesWithRetry() throws Exception { Field initMapField = SystemTopicBasedTopicPoliciesService.class.getDeclaredField("policyCacheInitMap"); diff --git a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/client/cli/PulsarClientToolTest.java b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/client/cli/PulsarClientToolTest.java index 8b32ad906eac4..30d6a342f18d1 100644 --- a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/client/cli/PulsarClientToolTest.java +++ b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/client/cli/PulsarClientToolTest.java @@ -120,6 +120,7 @@ public void testNonDurableSubscribe() throws Exception { properties.setProperty("useTls", "false"); final String topicName = getTopicWithRandomSuffix("non-durable"); + admin.topics().createNonPartitionedTopic(topicName); int numberOfMessages = 10; @Cleanup("shutdownNow") @@ -211,6 +212,7 @@ public void testRead() throws Exception { properties.setProperty("useTls", "false"); final String topicName = getTopicWithRandomSuffix("reader"); + admin.topics().createNonPartitionedTopic(topicName); int numberOfMessages = 10; @Cleanup("shutdownNow") @@ -260,6 +262,7 @@ public void testEncryption() throws Exception { properties.setProperty("useTls", "false"); final String topicName = getTopicWithRandomSuffix("encryption"); + admin.topics().createNonPartitionedTopic(topicName); final String keyUriBase = "file:../pulsar-broker/src/test/resources/certificate/"; final int numberOfMessages = 10; From 9d6e33e286cca76c00d6df38222775756a808b39 Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Fri, 10 Nov 2023 13:30:05 +0800 Subject: [PATCH 145/699] [fix][broker] Fix create topic with different auto creation strategies causes race condition (#21545) --- .../pulsar/broker/service/BrokerService.java | 14 ++++- .../pulsar/broker/admin/AdminApi2Test.java | 2 +- .../broker/admin/TopicAutoCreationTest.java | 57 +++++++++++++++++++ .../persistent/PersistentTopicTest.java | 3 +- 4 files changed, 72 insertions(+), 4 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 748536b43cd3d..46cc1bbb87565 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 @@ -1074,10 +1074,22 @@ public CompletableFuture> getTopic(final TopicName topicName, bo return loadOrCreatePersistentTopic(tpName, createIfMissing, properties, topicPolicies); } - return CompletableFuture.completedFuture(Optional.empty()); + final String errorMsg = + String.format("Illegal topic partition name %s with max allowed " + + "%d partitions", topicName, metadata.partitions); + log.warn(errorMsg); + return FutureUtil + .failedFuture(new BrokerServiceException.NotAllowedException(errorMsg)); }); } return loadOrCreatePersistentTopic(tpName, createIfMissing, properties, topicPolicies); + }).thenCompose(optionalTopic -> { + if (!optionalTopic.isPresent() && createIfMissing) { + log.warn("[{}] Try to recreate the topic with createIfMissing=true " + + "but the returned topic is empty", topicName); + return getTopic(topicName, createIfMissing, properties); + } + return CompletableFuture.completedFuture(optionalTopic); }); }); } else { 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 99009a1988206..81ccd39a2e9a4 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 @@ -3198,7 +3198,7 @@ public void testFailedUpdatePartitionedTopic() throws Exception { admin.topics().createSubscription(partitionedTopicName + "-partition-" + startPartitions, subName1, MessageId.earliest); fail("Unexpected behaviour"); - } catch (PulsarAdminException.PreconditionFailedException ex) { + } catch (PulsarAdminException.ConflictException ex) { // OK } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java index 590edc2d3f3bb..c9138beee52d1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java @@ -27,7 +27,10 @@ import java.util.List; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -40,6 +43,7 @@ import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride; import org.apache.pulsar.common.policies.data.TopicType; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -55,6 +59,7 @@ protected void setup() throws Exception { conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); conf.setAllowAutoTopicCreation(true); conf.setDefaultNumPartitions(3); + conf.setForceDeleteNamespaceAllowed(true); super.internalSetup(); super.producerBaseSetup(); } @@ -186,4 +191,56 @@ public void testPartitionedTopicAutoCreationForbiddenDuringNamespaceDeletion() } } + + @Test + public void testClientWithAutoCreationGotNotFoundException() throws PulsarAdminException, PulsarClientException { + final String namespace = "public/test_1"; + final String topicName = "persistent://public/test_1/test_auto_creation_got_not_found" + + System.currentTimeMillis(); + final int retryTimes = 30; + admin.namespaces().createNamespace(namespace); + admin.namespaces().setAutoTopicCreation(namespace, AutoTopicCreationOverride.builder() + .allowAutoTopicCreation(true) + .topicType("non-partitioned") + .build()); + + @Cleanup("shutdown") + final ExecutorService executor1 = Executors.newSingleThreadExecutor(); + + @Cleanup("shutdown") + final ExecutorService executor2 = Executors.newSingleThreadExecutor(); + + for (int i = 0; i < retryTimes; i++) { + final CompletableFuture adminListSub = CompletableFuture.runAsync(() -> { + try { + admin.topics().getSubscriptions(topicName); + } catch (PulsarAdminException e) { + throw new RuntimeException(e); + } + }, executor1); + + final CompletableFuture> consumerSub = CompletableFuture.supplyAsync(() -> { + try { + return pulsarClient.newConsumer() + .topic(topicName) + .subscriptionName("sub-1") + .subscribe(); + } catch (PulsarClientException e) { + throw new RuntimeException(e); + } + }, executor2); + + try { + adminListSub.join(); + } catch (Throwable ex) { + // we don't care the exception. + } + + consumerSub.join().close(); + admin.topics().delete(topicName, true); + } + + admin.namespaces().deleteNamespace(namespace, true); + } + } 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 9995b6a28a903..6f60a13fd4894 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 @@ -457,8 +457,7 @@ public void testCreateNonExistentPartitions() throws PulsarAdminException, Pulsa .topic(partition.toString()) .create(); fail("unexpected behaviour"); - } catch (PulsarClientException.TopicDoesNotExistException ignored) { - + } catch (PulsarClientException.NotAllowedException ex) { } Assert.assertEquals(admin.topics().getPartitionedTopicMetadata(topicName).partitions, 4); } From 821c21271c505768ede5734d581123be340caa51 Mon Sep 17 00:00:00 2001 From: erobot Date: Fri, 10 Nov 2023 18:54:01 +0800 Subject: [PATCH 146/699] [fix][broker] Fix the deadlock when using BookieRackAffinityMapping with rackaware policy (#21481) --- .../BookieRackAffinityMapping.java | 9 +-- .../BookieRackAffinityMappingTest.java | 68 +++++++++++++++++++ 2 files changed, 73 insertions(+), 4 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java index d54ef2a5f4cef..983822f22941b 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java @@ -245,6 +245,7 @@ private void handleUpdates(Notification n) { bookieMappingCache.get(BOOKIE_INFO_ROOT_PATH) .thenAccept(optVal -> { + Set bookieIdSet = new HashSet<>(); synchronized (this) { LOG.info("Bookie rack info updated to {}. Notifying rackaware policy.", optVal); this.updateRacksWithHost(optVal.orElseGet(BookiesRackConfiguration::new)); @@ -259,12 +260,12 @@ private void handleUpdates(Notification n) { LOG.debug("Bookies with rack update from {} to {}", bookieAddressListLastTime, bookieAddressList); } - Set bookieIdSet = new HashSet<>(bookieAddressList); + bookieIdSet.addAll(bookieAddressList); bookieIdSet.addAll(bookieAddressListLastTime); bookieAddressListLastTime = bookieAddressList; - if (rackawarePolicy != null) { - rackawarePolicy.onBookieRackChange(new ArrayList<>(bookieIdSet)); - } + } + if (rackawarePolicy != null) { + rackawarePolicy.onBookieRackChange(new ArrayList<>(bookieIdSet)); } }); } diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMappingTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMappingTest.java index d7df5afb4bebe..9cd8160444249 100644 --- a/pulsar-broker-common/src/test/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMappingTest.java +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMappingTest.java @@ -21,6 +21,7 @@ import static org.apache.bookkeeper.feature.SettableFeatureProvider.DISABLE_ALL; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -28,6 +29,7 @@ import java.lang.reflect.Constructor; import java.lang.reflect.Field; import java.lang.reflect.Method; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -35,7 +37,11 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import lombok.Cleanup; import org.apache.bookkeeper.client.DefaultBookieAddressResolver; import org.apache.bookkeeper.client.EnsemblePlacementPolicy; import org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy; @@ -46,6 +52,7 @@ import org.apache.bookkeeper.net.BookieId; import org.apache.bookkeeper.net.BookieNode; import org.apache.bookkeeper.net.BookieSocketAddress; +import org.apache.bookkeeper.net.NetworkTopology; import org.apache.bookkeeper.proto.BookieAddressResolver; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; @@ -55,6 +62,8 @@ 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.api.Notification; +import org.apache.pulsar.metadata.api.NotificationType; import org.apache.pulsar.metadata.bookkeeper.BookieServiceInfoSerde; import org.apache.pulsar.metadata.bookkeeper.PulsarRegistrationClient; import org.awaitility.Awaitility; @@ -342,4 +351,63 @@ public void testWithPulsarRegistrationClient() throws Exception { timer.stop(); } + + @Test + public void testNoDeadlockWithRackawarePolicy() throws Exception { + ClientConfiguration bkClientConf = new ClientConfiguration(); + bkClientConf.setProperty(BookieRackAffinityMapping.METADATA_STORE_INSTANCE, store); + + BookieRackAffinityMapping mapping = new BookieRackAffinityMapping(); + mapping.setBookieAddressResolver(BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); + mapping.setConf(bkClientConf); + + @Cleanup("stop") + HashedWheelTimer timer = new HashedWheelTimer(new ThreadFactoryBuilder().setNameFormat("TestTimer-%d").build(), + bkClientConf.getTimeoutTimerTickDurationMs(), TimeUnit.MILLISECONDS, + bkClientConf.getTimeoutTimerNumTicks()); + + RackawareEnsemblePlacementPolicy repp = new RackawareEnsemblePlacementPolicy(); + repp.initialize(bkClientConf, Optional.of(mapping), timer, + DISABLE_ALL, NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER); + repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK); + + mapping.registerRackChangeListener(repp); + + @Cleanup("shutdownNow") + ExecutorService executor1 = Executors.newSingleThreadExecutor(); + @Cleanup("shutdownNow") + ExecutorService executor2 = Executors.newSingleThreadExecutor(); + + CountDownLatch count = new CountDownLatch(2); + + executor1.submit(() -> { + try { + Method handleUpdates = + BookieRackAffinityMapping.class.getDeclaredMethod("handleUpdates", Notification.class); + handleUpdates.setAccessible(true); + Notification n = + new Notification(NotificationType.Modified, BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH); + long start = System.currentTimeMillis(); + while (System.currentTimeMillis() - start < 2_000) { + handleUpdates.invoke(mapping, n); + } + count.countDown(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + + executor2.submit(() -> { + Set writableBookies = new HashSet<>(); + writableBookies.add(BOOKIE1.toBookieId()); + long start = System.currentTimeMillis(); + while (System.currentTimeMillis() - start < 2_000) { + repp.onClusterChanged(writableBookies, Collections.emptySet()); + repp.onClusterChanged(Collections.emptySet(), Collections.emptySet()); + } + count.countDown(); + }); + + assertTrue(count.await(3, TimeUnit.SECONDS)); + } } From 2a96c003d588948ce63d168f86cdde9a67406c2a Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Sat, 4 Nov 2023 09:40:31 -0700 Subject: [PATCH 147/699] [fix][broker] Fix failure while creating non-durable cursor with inactive managed-ledger (#21508) --- .../bookkeeper/mledger/ManagedLedger.java | 4 +++- .../mledger/impl/ManagedLedgerImpl.java | 5 +++- .../mledger/impl/ManagedLedgerTest.java | 24 +++++++++++++++++++ .../jcloud/impl/MockManagedLedger.java | 4 ++-- 4 files changed, 33 insertions(+), 4 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java index c7dd8ea9129b7..f91d9ec3f5a02 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedger.java @@ -682,8 +682,10 @@ default void skipNonRecoverableLedger(long ledgerId){} /** * Check current inactive ledger (based on {@link ManagedLedgerConfig#getInactiveLedgerRollOverTimeMs()} and * roll over that ledger if inactive. + * + * @return true if ledger is considered for rolling over */ - void checkInactiveLedgerAndRollOver(); + boolean checkInactiveLedgerAndRollOver(); /** * Check if managed ledger should cache backlog reads. 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 df653a1196b63..fe8aabd94b17e 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 @@ -4447,7 +4447,7 @@ private void cancelScheduledTasks() { } @Override - public void checkInactiveLedgerAndRollOver() { + public boolean checkInactiveLedgerAndRollOver() { long currentTimeMs = System.currentTimeMillis(); if (inactiveLedgerRollOverTimeMs > 0 && currentTimeMs > (lastAddEntryTimeMs + inactiveLedgerRollOverTimeMs)) { log.info("[{}] Closing inactive ledger, last-add entry {}", name, lastAddEntryTimeMs); @@ -4468,10 +4468,13 @@ public void checkInactiveLedgerAndRollOver() { } ledgerClosed(lh); + createLedgerAfterClosed(); // we do not create ledger here, since topic is inactive for a long time. }, null); + return true; } } + return false; } 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 5fc2da22b661e..f9c52ec60b21c 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 @@ -124,6 +124,7 @@ import org.apache.bookkeeper.mledger.util.Futures; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.apache.commons.lang3.exception.ExceptionUtils; +import org.apache.commons.lang3.mutable.MutableBoolean; import org.apache.commons.lang3.mutable.MutableObject; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.common.api.proto.CommandSubscribe.InitialPosition; @@ -4074,4 +4075,27 @@ public void operationFailed(MetaStoreException e) { }); future.join(); } + + @Test + public void testNonDurableCursorCreateForInactiveLedger() throws Exception { + String mlName = "testLedgerInfoMetaCorrectIfAddEntryTimeOut"; + BookKeeper spyBookKeeper = spy(bkc); + ManagedLedgerFactoryImpl factory = new ManagedLedgerFactoryImpl(metadataStore, spyBookKeeper); + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setInactiveLedgerRollOverTime(10, TimeUnit.MILLISECONDS); + ManagedLedgerImpl ml = (ManagedLedgerImpl) factory.open(mlName, config); + + MutableBoolean isRolledOver = new MutableBoolean(false); + retryStrategically((test) -> { + if (isRolledOver.booleanValue()) { + return true; + } + isRolledOver.setValue(ml.checkInactiveLedgerAndRollOver()); + return isRolledOver.booleanValue(); + }, 5, 1000); + assertTrue(isRolledOver.booleanValue()); + + Position Position = new PositionImpl(-1L, -1L); + assertNotNull(ml.newNonDurableCursor(Position)); + } } diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/MockManagedLedger.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/MockManagedLedger.java index 774b0143f956e..66ace69d7cda2 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/MockManagedLedger.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/MockManagedLedger.java @@ -372,8 +372,8 @@ public CompletableFuture getManagedLedgerInternalSta } @Override - public void checkInactiveLedgerAndRollOver() { - + public boolean checkInactiveLedgerAndRollOver() { + return false; } @Override From 46b699dc174e1117656d6d52f76dbf94ed4685c8 Mon Sep 17 00:00:00 2001 From: hanmz Date: Sat, 11 Nov 2023 12:36:29 +0800 Subject: [PATCH 148/699] [fix][client] Fix print error log 'Auto getting partitions failed' when expend partition. (#21485) --- .../org/apache/pulsar/client/impl/PartitionedProducerImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 f780edc95c136..bf7f1066173f6 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 @@ -436,7 +436,7 @@ public CompletableFuture onTopicsExtended(Collection topicsExtende }); // call interceptor with the metadata change onPartitionsChange(topic, currentPartitionNumber); - return null; + return future; } } else { log.error("[{}] not support shrink topic partitions. old: {}, new: {}", From 4c492d13b8cfb6a2e46f1edc417628c43299216a Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Fri, 10 Nov 2023 19:14:39 +0800 Subject: [PATCH 149/699] [fix][ml] Fix unfinished callback when deleting managed ledger (#21530) --- .../bookkeeper/mledger/impl/ManagedLedgerFactoryImpl.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) 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 9107b76c88a28..1bb23912b5e31 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 @@ -880,7 +880,10 @@ public void asyncDelete(String name, CompletableFuture mlCo // If it's open, delete in the normal way ml.asyncDelete(callback, ctx); }).exceptionally(ex -> { - // If it's failing to get open, just delete from metadata + // If it fails to get open, it will be cleaned by managed ledger opening error handling. + // then retry will go to `future=null` branch. + final Throwable rc = FutureUtil.unwrapCompletionException(ex); + callback.deleteLedgerFailed(getManagedLedgerException(rc), ctx); return null; }); } From dcff23a21f978cde94fcf9b1d9e6c08db586b624 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Sat, 11 Nov 2023 15:10:47 +0800 Subject: [PATCH 150/699] [fix] [broker] Fix thousands orphan PersistentTopic caused OOM (#21540) --- .../pulsar/broker/service/BrokerService.java | 37 ++--- .../client/api/OrphanPersistentTopicTest.java | 154 ++++++++++++++++++ .../org/apache/zookeeper/MockZooKeeper.java | 14 +- 3 files changed, 184 insertions(+), 21 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java 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 46cc1bbb87565..382bca68dffc9 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 @@ -1747,16 +1747,14 @@ public void openLedgerComplete(ManagedLedger ledger, Object ctx) { // Check create persistent topic timeout. log.warn("{} future is already completed with failure {}, closing the" + " topic", topic, FutureUtil.getException(topicFuture)); - persistentTopic.getTransactionBuffer() - .closeAsync() - .exceptionally(t -> { - log.error("[{}] Close transactionBuffer failed", topic, t); - return null; - }); - persistentTopic.stopReplProducers() - .whenCompleteAsync((v, exception) -> { - topics.remove(topic, topicFuture); - }, executor()); + executor().submit(() -> { + persistentTopic.close().whenComplete((ignore, ex) -> { + if (ex != null) { + log.warn("[{}] Get an error when closing topic.", + topic, ex); + } + }); + }); } else { addTopicToStatsMaps(topicName, persistentTopic); topicFuture.complete(Optional.of(persistentTopic)); @@ -1765,16 +1763,15 @@ public void openLedgerComplete(ManagedLedger ledger, Object ctx) { .exceptionally((ex) -> { log.warn("Replication or dedup check failed." + " Removing topic from topics list {}, {}", topic, ex); - persistentTopic.getTransactionBuffer() - .closeAsync() - .exceptionally(t -> { - log.error("[{}] Close transactionBuffer failed", topic, t); - return null; - }); - persistentTopic.stopReplProducers().whenCompleteAsync((v, exception) -> { - topics.remove(topic, topicFuture); - topicFuture.completeExceptionally(ex); - }, executor()); + executor().submit(() -> { + persistentTopic.close().whenComplete((ignore, closeEx) -> { + if (closeEx != null) { + log.warn("[{}] Get an error when closing topic.", + topic, closeEx); + } + topicFuture.completeExceptionally(ex); + }); + }); return null; }); } catch (PulsarServerException e) { 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 new file mode 100644 index 0000000000000..7cd9da7574dbb --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/OrphanPersistentTopicTest.java @@ -0,0 +1,154 @@ +/* + * 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.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; +import java.lang.reflect.Field; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.service.BrokerService; +import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.TopicPoliciesService; +import org.apache.pulsar.broker.service.TopicPolicyListener; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.TopicPolicies; +import org.apache.pulsar.compaction.CompactionServiceFactory; +import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-api") +public class OrphanPersistentTopicTest 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 testNoOrphanTopicAfterCreateTimeout() throws Exception { + // Make the topic loading timeout faster. + int topicLoadTimeoutSeconds = 2; + long originalTopicLoadTimeoutSeconds = pulsar.getConfig().getTopicLoadTimeoutSeconds(); + pulsar.getConfig().setTopicLoadTimeoutSeconds(2); + + String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + String mlPath = BrokerService.MANAGED_LEDGER_PATH_ZNODE + "/" + TopicName.get(tpName).getPersistenceNamingEncoding(); + + // Make topic load timeout 5 times. + AtomicInteger timeoutCounter = new AtomicInteger(); + for (int i = 0; i < 5; i++) { + mockZooKeeper.delay(topicLoadTimeoutSeconds * 2 * 1000, (op, path) -> { + if (mlPath.equals(path)) { + log.info("Topic load timeout: " + timeoutCounter.incrementAndGet()); + return true; + } + return false; + }); + } + + // Load topic. + CompletableFuture> consumer = pulsarClient.newConsumer() + .topic(tpName) + .subscriptionName("my-sub") + .subscribeAsync(); + + // After create timeout 5 times, the topic will be created successful. + Awaitility.await().ignoreExceptions().atMost(40, TimeUnit.SECONDS).untilAsserted(() -> { + CompletableFuture> future = pulsar.getBrokerService().getTopic(tpName, false); + assertTrue(future.isDone()); + Optional optional = future.get(); + assertTrue(optional.isPresent()); + }); + + // Assert only one PersistentTopic was not closed. + TopicPoliciesService topicPoliciesService = pulsar.getTopicPoliciesService(); + Map>> listeners = + WhiteboxImpl.getInternalState(topicPoliciesService, "listeners"); + assertEquals(listeners.get(TopicName.get(tpName)).size(), 1); + + // cleanup. + consumer.join().close(); + admin.topics().delete(tpName, false); + pulsar.getConfig().setTopicLoadTimeoutSeconds(originalTopicLoadTimeoutSeconds); + } + + @Test + public void testNoOrphanTopicIfInitFailed() throws Exception { + String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(tpName); + + // Load topic. + Consumer consumer = pulsarClient.newConsumer() + .topic(tpName) + .subscriptionName("my-sub") + .subscribe(); + + // Make the method `PersitentTopic.initialize` fail. + Field fieldCompactionServiceFactory = PulsarService.class.getDeclaredField("compactionServiceFactory"); + fieldCompactionServiceFactory.setAccessible(true); + CompactionServiceFactory compactionServiceFactory = + (CompactionServiceFactory) fieldCompactionServiceFactory.get(pulsar); + fieldCompactionServiceFactory.set(pulsar, null); + admin.topics().unload(tpName); + + // Wait for failed to create topic for several times. + Thread.sleep(5 * 1000); + + // Remove the injected error, the topic will be created successful. + fieldCompactionServiceFactory.set(pulsar, compactionServiceFactory); + // We do not know the next time of consumer reconnection, so wait for 2 minutes to avoid flaky. It will be + // very fast in normal. + Awaitility.await().ignoreExceptions().atMost(120, TimeUnit.SECONDS).untilAsserted(() -> { + CompletableFuture> future = pulsar.getBrokerService().getTopic(tpName, false); + assertTrue(future.isDone()); + Optional optional = future.get(); + assertTrue(optional.isPresent()); + }); + + // Assert only one PersistentTopic was not closed. + TopicPoliciesService topicPoliciesService = pulsar.getTopicPoliciesService(); + Map>> listeners = + WhiteboxImpl.getInternalState(topicPoliciesService, "listeners"); + assertEquals(listeners.get(TopicName.get(tpName)).size(), 1); + + // cleanup. + consumer.close(); + admin.topics().delete(tpName, false); + } +} diff --git a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java index 0c0f7ec9ed1d4..f32036e53f001 100644 --- a/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java +++ b/testmocks/src/main/java/org/apache/zookeeper/MockZooKeeper.java @@ -1114,7 +1114,7 @@ Optional programmedFailure(Op op, String path) { Optional failure = failures.stream().filter(f -> f.predicate.test(op, path)).findFirst(); if (failure.isPresent()) { failures.remove(failure.get()); - return Optional.of(failure.get().failReturnCode); + return Optional.ofNullable(failure.get().failReturnCode); } else { return Optional.empty(); } @@ -1131,6 +1131,18 @@ public void failConditional(KeeperException.Code rc, BiPredicate pre failures.add(new Failure(rc, predicate)); } + public void delay(long millis, BiPredicate predicate) { + failures.add(new Failure(null, (op, s) -> { + if (predicate.test(op, s)) { + try { + Thread.sleep(millis); + } catch (InterruptedException e) {} + return true; + } + return false; + })); + } + public void setAlwaysFail(KeeperException.Code rc) { this.alwaysFail.set(rc); } From e4f86c932b1d5e05645ae2a63029a0eb371fcacb Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Sat, 11 Nov 2023 13:42:27 +0800 Subject: [PATCH 151/699] [fix] [ml] Fix orphan scheduled task for ledger create timeout check (#21542) ### Motivation When an ML tries to create a new ledger, it will create a delay task to check if the ledger create request is timeout[1]. However, we should cancel this delay task after the request to create new ledgers is finished. Otherwise, these tasks will cost unnecessary CPU resources. ### Modifications Cancel the scheduled task after the create ledger request is finished --- .../mledger/impl/ManagedLedgerImpl.java | 31 +++++----- .../mledger/impl/ManagedLedgerTest.java | 59 ++++++++++++++++++- 2 files changed, 72 insertions(+), 18 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 fe8aabd94b17e..ed148f66fc714 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 @@ -59,7 +59,6 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLongFieldUpdater; import java.util.concurrent.atomic.AtomicReference; @@ -4031,7 +4030,7 @@ public static ManagedLedgerException createManagedLedgerException(Throwable t) { */ protected void asyncCreateLedger(BookKeeper bookKeeper, ManagedLedgerConfig config, DigestType digestType, CreateCallback cb, Map metadata) { - AtomicBoolean ledgerCreated = new AtomicBoolean(false); + CompletableFuture ledgerFutureHook = new CompletableFuture<>(); Map finalMetadata = new HashMap<>(); finalMetadata.putAll(ledgerMetadata); finalMetadata.putAll(metadata); @@ -4044,33 +4043,39 @@ protected void asyncCreateLedger(BookKeeper bookKeeper, ManagedLedgerConfig conf )); } catch (EnsemblePlacementPolicyConfig.ParseEnsemblePlacementPolicyConfigException e) { log.error("[{}] Serialize the placement configuration failed", name, e); - cb.createComplete(Code.UnexpectedConditionException, null, ledgerCreated); + cb.createComplete(Code.UnexpectedConditionException, null, ledgerFutureHook); return; } } createdLedgerCustomMetadata = finalMetadata; - try { bookKeeper.asyncCreateLedger(config.getEnsembleSize(), config.getWriteQuorumSize(), - config.getAckQuorumSize(), digestType, config.getPassword(), cb, ledgerCreated, finalMetadata); + config.getAckQuorumSize(), digestType, config.getPassword(), cb, ledgerFutureHook, finalMetadata); } catch (Throwable cause) { log.error("[{}] Encountered unexpected error when creating ledger", name, cause); - cb.createComplete(Code.UnexpectedConditionException, null, ledgerCreated); + ledgerFutureHook.completeExceptionally(cause); + cb.createComplete(Code.UnexpectedConditionException, null, ledgerFutureHook); return; } - scheduledExecutor.schedule(() -> { - if (!ledgerCreated.get()) { + + ScheduledFuture timeoutChecker = scheduledExecutor.schedule(() -> { + if (!ledgerFutureHook.isDone() + && ledgerFutureHook.completeExceptionally(new TimeoutException(name + " Create ledger timeout"))) { if (log.isDebugEnabled()) { log.debug("[{}] Timeout creating ledger", name); } - cb.createComplete(BKException.Code.TimeoutException, null, ledgerCreated); + cb.createComplete(BKException.Code.TimeoutException, null, ledgerFutureHook); } else { if (log.isDebugEnabled()) { log.debug("[{}] Ledger already created when timeout task is triggered", name); } } }, config.getMetadataOperationsTimeoutSeconds(), TimeUnit.SECONDS); + + ledgerFutureHook.whenComplete((ignore, ex) -> { + timeoutChecker.cancel(false); + }); } public Clock getClock() { @@ -4079,16 +4084,12 @@ public Clock getClock() { /** * check if ledger-op task is already completed by timeout-task. If completed then delete the created ledger - * - * @param rc - * @param lh - * @param ctx * @return */ protected boolean checkAndCompleteLedgerOpTask(int rc, LedgerHandle lh, Object ctx) { - if (ctx instanceof AtomicBoolean) { + if (ctx instanceof CompletableFuture) { // ledger-creation is already timed out and callback is already completed so, delete this ledger and return. - if (((AtomicBoolean) (ctx)).compareAndSet(false, true)) { + if (((CompletableFuture) ctx).complete(lh)) { 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 f9c52ec60b21c..ca4e1d10a6c54 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 @@ -61,11 +61,13 @@ import java.util.Optional; import java.util.Set; import java.util.UUID; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.FutureTask; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; @@ -90,6 +92,8 @@ import org.apache.bookkeeper.client.api.LedgerEntries; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.client.api.ReadHandle; +import org.apache.bookkeeper.common.util.BoundedScheduledExecutorService; +import org.apache.bookkeeper.common.util.OrderedScheduler; import org.apache.bookkeeper.conf.ClientConfiguration; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; @@ -136,6 +140,7 @@ import org.apache.pulsar.metadata.api.extended.SessionEvent; import org.apache.pulsar.metadata.impl.FaultInjectionMetadataStore; import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.DataProvider; @@ -3086,9 +3091,9 @@ public void testManagedLedgerWithCreateLedgerTimeOut() throws Exception { latch.await(config.getMetadataOperationsTimeoutSeconds() + 2, TimeUnit.SECONDS); assertEquals(response.get(), BKException.Code.TimeoutException); - assertTrue(ctxHolder.get() instanceof AtomicBoolean); - AtomicBoolean ledgerCreated = (AtomicBoolean) ctxHolder.get(); - assertFalse(ledgerCreated.get()); + assertTrue(ctxHolder.get() instanceof CompletableFuture); + CompletableFuture ledgerCreateHook = (CompletableFuture) ctxHolder.get(); + assertTrue(ledgerCreateHook.isCompletedExceptionally()); ledger.close(); } @@ -4098,4 +4103,52 @@ public void testNonDurableCursorCreateForInactiveLedger() throws Exception { Position Position = new PositionImpl(-1L, -1L); assertNotNull(ml.newNonDurableCursor(Position)); } + + /*** + * 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. + */ + @Test + public void testNoOrphanScheduledTasksAfterCloseML() throws Exception { + String mlName = UUID.randomUUID().toString(); + ManagedLedgerFactoryImpl factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setMetadataOperationsTimeoutSeconds(3600); + + // Calculate pending task count. + long pendingTaskCountBefore = calculatePendingTaskCount(factory.getScheduledExecutor()); + // Trigger create & close ML 1000 times. + for (int i = 0; i < 1000; i++) { + ManagedLedger ml = factory.open(mlName, config); + ml.close(); + } + // Verify there is no orphan scheduled task. + long pendingTaskCountAfter = calculatePendingTaskCount(factory.getScheduledExecutor()); + // Maybe there are other components also appended scheduled tasks, so leave 100 tasks to avoid flaky. + assertTrue(pendingTaskCountAfter - pendingTaskCountBefore < 100); + } + + /** + * Calculate how many pending tasks in {@link OrderedScheduler} + */ + private long calculatePendingTaskCount(OrderedScheduler orderedScheduler) { + ExecutorService[] threads = WhiteboxImpl.getInternalState(orderedScheduler, "threads"); + long taskCounter = 0; + for (ExecutorService thread : threads) { + BoundedScheduledExecutorService boundedScheduledExecutorService = + WhiteboxImpl.getInternalState(thread, "delegate"); + BlockingQueue queue = WhiteboxImpl.getInternalState(boundedScheduledExecutorService, "queue"); + for (Runnable r : queue) { + if (r instanceof FutureTask) { + FutureTask futureTask = (FutureTask) r; + if (!futureTask.isCancelled() && !futureTask.isDone()) { + taskCounter++; + } + } else { + taskCounter++; + } + } + } + return taskCounter; + } } From af3ecfa32d534d03be4c6b0bd9cd04a8aae72b76 Mon Sep 17 00:00:00 2001 From: xiangying <1984997880@qq.com> Date: Mon, 13 Nov 2023 11:41:18 +0800 Subject: [PATCH 152/699] Revert "[fix][client] Avert extensive time consumption during table view construction (#21270)" This reverts commit c46ed2d809df0aab0241afafdc70fab1a7731c53. --- .../pulsar/client/impl/TableViewTest.java | 60 ------------------- .../pulsar/client/impl/TableViewImpl.java | 28 ++------- 2 files changed, 4 insertions(+), 84 deletions(-) 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 523360884c1bf..6c6da5870aed9 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 @@ -20,21 +20,16 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; 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 com.google.common.collect.Sets; -import java.lang.reflect.Method; import java.time.Duration; import java.util.HashSet; -import java.util.List; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -44,7 +39,6 @@ import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; -import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; @@ -52,7 +46,6 @@ import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.TableView; -import org.apache.pulsar.client.api.TopicMessageId; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; @@ -445,57 +438,4 @@ public void testTableViewTailMessageReadRetry() throws Exception { }); verify(consumer, times(msgCnt)).receiveAsync(); } - - @Test - public void testBuildTableViewWithMessagesAlwaysAvailable() throws Exception { - String topic = "persistent://public/default/testBuildTableViewWithMessagesAlwaysAvailable"; - admin.topics().createPartitionedTopic(topic, 10); - @Cleanup - Reader reader = pulsarClient.newReader() - .topic(topic) - .startMessageId(MessageId.earliest) - .create(); - @Cleanup - Producer producer = pulsarClient.newProducer() - .topic(topic) - .create(); - // Prepare real data to do test. - for (int i = 0; i < 1000; i++) { - producer.newMessage().send(); - } - List lastMessageIds = reader.getLastMessageIds(); - - // Use mock reader to build tableview. In the old implementation, the readAllExistingMessages method - // will not be completed because the `mockReader.hasMessageAvailable()` always return ture. - Reader mockReader = spy(reader); - when(mockReader.hasMessageAvailable()).thenReturn(true); - when(mockReader.getLastMessageIdsAsync()).thenReturn(CompletableFuture.completedFuture(lastMessageIds)); - AtomicInteger index = new AtomicInteger(lastMessageIds.size()); - when(mockReader.readNextAsync()).thenAnswer(invocation -> { - Message message = spy(Message.class); - int localIndex = index.decrementAndGet(); - if (localIndex >= 0) { - when(message.getTopicName()).thenReturn(lastMessageIds.get(localIndex).getOwnerTopic()); - when(message.getMessageId()).thenReturn(lastMessageIds.get(localIndex)); - when(message.hasKey()).thenReturn(false); - doNothing().when(message).release(); - } - return CompletableFuture.completedFuture(message); - }); - @Cleanup - TableViewImpl tableView = (TableViewImpl) pulsarClient.newTableView() - .topic(topic) - .createAsync() - .get(); - TableViewImpl mockTableView = spy(tableView); - Method readAllExistingMessagesMethod = TableViewImpl.class - .getDeclaredMethod("readAllExistingMessages", Reader.class); - readAllExistingMessagesMethod.setAccessible(true); - CompletableFuture> future = - (CompletableFuture>) readAllExistingMessagesMethod.invoke(mockTableView, mockReader); - - // The future will complete after receive all the messages from lastMessageIds. - future.get(3, TimeUnit.SECONDS); - assertTrue(index.get() <= 0); - } } 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 151c96d96aa40..560636f94622b 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 @@ -41,7 +41,6 @@ import org.apache.pulsar.client.api.ReaderBuilder; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.TableView; -import org.apache.pulsar.client.api.TopicMessageId; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.topics.TopicCompactionStrategy; @@ -236,40 +235,20 @@ private CompletableFuture> readAllExistingMessages(Reader reader) { AtomicLong messagesRead = new AtomicLong(); CompletableFuture> future = new CompletableFuture<>(); - reader.getLastMessageIdsAsync().thenAccept(lastMessageIds -> { - Map maxMessageIds = new ConcurrentHashMap<>(); - lastMessageIds.forEach(topicMessageId -> { - maxMessageIds.put(topicMessageId.getOwnerTopic(), topicMessageId); - }); - readAllExistingMessages(reader, future, startTime, messagesRead, maxMessageIds); - }).exceptionally(ex -> { - future.completeExceptionally(ex); - return null; - }); - future.thenAccept(__ -> readTailMessages(reader)); + readAllExistingMessages(reader, future, startTime, messagesRead); return future; } private void readAllExistingMessages(Reader reader, CompletableFuture> future, long startTime, - AtomicLong messagesRead, Map maxMessageIds) { + AtomicLong messagesRead) { reader.hasMessageAvailableAsync() .thenAccept(hasMessage -> { if (hasMessage) { reader.readNextAsync() .thenAccept(msg -> { messagesRead.incrementAndGet(); - // We need remove the partition from the maxMessageIds map - // once the partition has been read completely. - TopicMessageId maxMessageId = maxMessageIds.get(msg.getTopicName()); - if (maxMessageId != null && msg.getMessageId().compareTo(maxMessageId) >= 0) { - maxMessageIds.remove(msg.getTopicName()); - } handleMessage(msg); - if (maxMessageIds.isEmpty()) { - future.complete(reader); - } else { - readAllExistingMessages(reader, future, startTime, messagesRead, maxMessageIds); - } + readAllExistingMessages(reader, future, startTime, messagesRead); }).exceptionally(ex -> { if (ex.getCause() instanceof PulsarClientException.AlreadyClosedException) { log.error("Reader {} was closed while reading existing messages.", @@ -290,6 +269,7 @@ private void readAllExistingMessages(Reader reader, CompletableFuture Date: Mon, 13 Nov 2023 18:07:55 +0800 Subject: [PATCH 153/699] Revert "[fix][broker] Fix issue with consumer read uncommitted messages from compacted topic (#21465)" This reverts commit 80f921a45bb023fca36faf98038f3ec687e05f16. --- .../PersistentDispatcherSingleActiveConsumer.java | 6 ++---- .../org/apache/pulsar/compaction/CompactedTopic.java | 5 +---- .../apache/pulsar/compaction/CompactedTopicImpl.java | 3 +-- .../apache/pulsar/compaction/CompactedTopicUtils.java | 10 ++++++---- .../pulsar/compaction/CompactedTopicUtilsTest.java | 4 ++-- 5 files changed, 12 insertions(+), 16 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java index 5e9183df0b1df..d96429693fda8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java @@ -55,7 +55,6 @@ import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.util.Codec; import org.apache.pulsar.compaction.CompactedTopicUtils; -import org.apache.pulsar.compaction.TopicCompactionService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -351,9 +350,8 @@ protected void readMoreEntries(Consumer consumer) { havePendingRead = true; if (consumer.readCompacted()) { boolean readFromEarliest = isFirstRead && MessageId.earliest.equals(consumer.getStartMessageId()); - TopicCompactionService topicCompactionService = topic.getTopicCompactionService(); - CompactedTopicUtils.asyncReadCompactedEntries(topicCompactionService, cursor, messagesToRead, - bytesToRead, topic.getMaxReadPosition(), readFromEarliest, this, true, consumer); + CompactedTopicUtils.asyncReadCompactedEntries(topic.getTopicCompactionService(), cursor, + messagesToRead, bytesToRead, readFromEarliest, this, true, consumer); } else { ReadEntriesCtx readEntriesCtx = ReadEntriesCtx.create(consumer, consumer.getConsumerEpoch()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java index 146ba4327d252..8c17e0f3ca34d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java @@ -24,7 +24,6 @@ import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.Position; -import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.service.Consumer; public interface CompactedTopic { @@ -35,14 +34,12 @@ public interface CompactedTopic { * Read entries from compacted topic. * * @deprecated Use {@link CompactedTopicUtils#asyncReadCompactedEntries(TopicCompactionService, ManagedCursor, - * int, long, org.apache.bookkeeper.mledger.impl.PositionImpl, boolean, ReadEntriesCallback, boolean, Consumer)} - * instead. + * int, long, boolean, ReadEntriesCallback, boolean, Consumer)} instead. */ @Deprecated void asyncReadEntriesOrWait(ManagedCursor cursor, int maxEntries, long bytesToRead, - PositionImpl maxReadPosition, boolean isFirstRead, ReadEntriesCallback callback, Consumer consumer); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java index 8794e2736d4d4..b028b708c49e4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java @@ -96,7 +96,6 @@ public CompletableFuture deleteCompactedLedger(long compactedLedgerId) { public void asyncReadEntriesOrWait(ManagedCursor cursor, int maxEntries, long bytesToRead, - PositionImpl maxReadPosition, boolean isFirstRead, ReadEntriesCallback callback, Consumer consumer) { PositionImpl cursorPosition; @@ -113,7 +112,7 @@ public void asyncReadEntriesOrWait(ManagedCursor cursor, if (currentCompactionHorizon == null || currentCompactionHorizon.compareTo(cursorPosition) < 0) { - cursor.asyncReadEntriesOrWait(maxEntries, bytesToRead, callback, readEntriesCtx, maxReadPosition); + cursor.asyncReadEntriesOrWait(maxEntries, bytesToRead, callback, readEntriesCtx, PositionImpl.LATEST); } else { ManagedCursorImpl managedCursor = (ManagedCursorImpl) cursor; int numberOfEntriesToRead = managedCursor.applyMaxSizeCap(maxEntries, bytesToRead); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java index d3464d402e9c6..66bcf4c3002bd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java @@ -42,8 +42,8 @@ public class CompactedTopicUtils { @Beta public static void asyncReadCompactedEntries(TopicCompactionService topicCompactionService, ManagedCursor cursor, int maxEntries, - long bytesToRead, PositionImpl maxReadPosition, - boolean readFromEarliest, AsyncCallbacks.ReadEntriesCallback callback, + long bytesToRead, boolean readFromEarliest, + AsyncCallbacks.ReadEntriesCallback callback, boolean wait, @Nullable Consumer consumer) { Objects.requireNonNull(topicCompactionService); Objects.requireNonNull(cursor); @@ -68,9 +68,11 @@ public static void asyncReadCompactedEntries(TopicCompactionService topicCompact || readPosition.compareTo( lastCompactedPosition.getLedgerId(), lastCompactedPosition.getEntryId()) > 0) { if (wait) { - cursor.asyncReadEntriesOrWait(maxEntries, bytesToRead, callback, readEntriesCtx, maxReadPosition); + cursor.asyncReadEntriesOrWait(maxEntries, bytesToRead, callback, readEntriesCtx, + PositionImpl.LATEST); } else { - cursor.asyncReadEntries(maxEntries, bytesToRead, callback, readEntriesCtx, maxReadPosition); + cursor.asyncReadEntries(maxEntries, bytesToRead, callback, readEntriesCtx, + PositionImpl.LATEST); } return CompletableFuture.completedFuture(null); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicUtilsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicUtilsTest.java index 2545c0362e82a..94f2a17a2a3f4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicUtilsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicUtilsTest.java @@ -69,8 +69,8 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { } }; - CompactedTopicUtils.asyncReadCompactedEntries(service, cursor, 1, 100, - PositionImpl.LATEST, false, readEntriesCallback, false, null); + CompactedTopicUtils.asyncReadCompactedEntries(service, cursor, 1, 100, false, + readEntriesCallback, false, null); List entries = completableFuture.get(); Assert.assertTrue(entries.isEmpty()); From b26a52fd05697fc13ce693862a126dc343fb8bb5 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Mon, 13 Nov 2023 21:08:00 +0800 Subject: [PATCH 154/699] [fix][broker][branch-3.1] Fix issue with consumer read uncommitted messages from compacted topic (#21465) (#21571) --- ...ersistentDispatcherSingleActiveConsumer.java | 6 ++++-- .../pulsar/compaction/CompactedTopic.java | 5 ++++- .../pulsar/compaction/CompactedTopicImpl.java | 3 ++- .../pulsar/compaction/CompactedTopicUtils.java | 17 ++++++++++++----- .../compaction/CompactedTopicUtilsTest.java | 4 ++-- 5 files changed, 24 insertions(+), 11 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java index d96429693fda8..5e9183df0b1df 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java @@ -55,6 +55,7 @@ import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.util.Codec; import org.apache.pulsar.compaction.CompactedTopicUtils; +import org.apache.pulsar.compaction.TopicCompactionService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -350,8 +351,9 @@ protected void readMoreEntries(Consumer consumer) { havePendingRead = true; if (consumer.readCompacted()) { boolean readFromEarliest = isFirstRead && MessageId.earliest.equals(consumer.getStartMessageId()); - CompactedTopicUtils.asyncReadCompactedEntries(topic.getTopicCompactionService(), cursor, - messagesToRead, bytesToRead, readFromEarliest, this, true, consumer); + TopicCompactionService topicCompactionService = topic.getTopicCompactionService(); + CompactedTopicUtils.asyncReadCompactedEntries(topicCompactionService, cursor, messagesToRead, + bytesToRead, topic.getMaxReadPosition(), readFromEarliest, this, true, consumer); } else { ReadEntriesCtx readEntriesCtx = ReadEntriesCtx.create(consumer, consumer.getConsumerEpoch()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java index 8c17e0f3ca34d..146ba4327d252 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopic.java @@ -24,6 +24,7 @@ import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.service.Consumer; public interface CompactedTopic { @@ -34,12 +35,14 @@ public interface CompactedTopic { * Read entries from compacted topic. * * @deprecated Use {@link CompactedTopicUtils#asyncReadCompactedEntries(TopicCompactionService, ManagedCursor, - * int, long, boolean, ReadEntriesCallback, boolean, Consumer)} instead. + * int, long, org.apache.bookkeeper.mledger.impl.PositionImpl, boolean, ReadEntriesCallback, boolean, Consumer)} + * instead. */ @Deprecated void asyncReadEntriesOrWait(ManagedCursor cursor, int maxEntries, long bytesToRead, + PositionImpl maxReadPosition, boolean isFirstRead, ReadEntriesCallback callback, Consumer consumer); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java index b028b708c49e4..8794e2736d4d4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java @@ -96,6 +96,7 @@ public CompletableFuture deleteCompactedLedger(long compactedLedgerId) { public void asyncReadEntriesOrWait(ManagedCursor cursor, int maxEntries, long bytesToRead, + PositionImpl maxReadPosition, boolean isFirstRead, ReadEntriesCallback callback, Consumer consumer) { PositionImpl cursorPosition; @@ -112,7 +113,7 @@ public void asyncReadEntriesOrWait(ManagedCursor cursor, if (currentCompactionHorizon == null || currentCompactionHorizon.compareTo(cursorPosition) < 0) { - cursor.asyncReadEntriesOrWait(maxEntries, bytesToRead, callback, readEntriesCtx, PositionImpl.LATEST); + cursor.asyncReadEntriesOrWait(maxEntries, bytesToRead, callback, readEntriesCtx, maxReadPosition); } else { ManagedCursorImpl managedCursor = (ManagedCursorImpl) cursor; int numberOfEntriesToRead = managedCursor.applyMaxSizeCap(maxEntries, bytesToRead); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java index 66bcf4c3002bd..c084593b0fa37 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicUtils.java @@ -39,12 +39,21 @@ public class CompactedTopicUtils { - @Beta public static void asyncReadCompactedEntries(TopicCompactionService topicCompactionService, ManagedCursor cursor, int maxEntries, long bytesToRead, boolean readFromEarliest, AsyncCallbacks.ReadEntriesCallback callback, boolean wait, @Nullable Consumer consumer) { + asyncReadCompactedEntries(topicCompactionService, cursor, maxEntries, bytesToRead, PositionImpl.LATEST, + readFromEarliest, callback, wait, consumer); + } + + @Beta + public static void asyncReadCompactedEntries(TopicCompactionService topicCompactionService, + ManagedCursor cursor, int maxEntries, + long bytesToRead, PositionImpl maxReadPosition, + boolean readFromEarliest, AsyncCallbacks.ReadEntriesCallback callback, + boolean wait, @Nullable Consumer consumer) { Objects.requireNonNull(topicCompactionService); Objects.requireNonNull(cursor); checkArgument(maxEntries > 0); @@ -68,11 +77,9 @@ public static void asyncReadCompactedEntries(TopicCompactionService topicCompact || readPosition.compareTo( lastCompactedPosition.getLedgerId(), lastCompactedPosition.getEntryId()) > 0) { if (wait) { - cursor.asyncReadEntriesOrWait(maxEntries, bytesToRead, callback, readEntriesCtx, - PositionImpl.LATEST); + cursor.asyncReadEntriesOrWait(maxEntries, bytesToRead, callback, readEntriesCtx, maxReadPosition); } else { - cursor.asyncReadEntries(maxEntries, bytesToRead, callback, readEntriesCtx, - PositionImpl.LATEST); + cursor.asyncReadEntries(maxEntries, bytesToRead, callback, readEntriesCtx, maxReadPosition); } return CompletableFuture.completedFuture(null); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicUtilsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicUtilsTest.java index 94f2a17a2a3f4..2545c0362e82a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicUtilsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactedTopicUtilsTest.java @@ -69,8 +69,8 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { } }; - CompactedTopicUtils.asyncReadCompactedEntries(service, cursor, 1, 100, false, - readEntriesCallback, false, null); + CompactedTopicUtils.asyncReadCompactedEntries(service, cursor, 1, 100, + PositionImpl.LATEST, false, readEntriesCallback, false, null); List entries = completableFuture.get(); Assert.assertTrue(entries.isEmpty()); From d79e0de3dd7f76d0829e66e6101aa2e43d3440f4 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Mon, 20 Nov 2023 17:17:28 +0800 Subject: [PATCH 155/699] =?UTF-8?q?[fix][broker]=20Duplicate=20LedgerOfflo?= =?UTF-8?q?ader=20creation=20when=20namespace/topic=E2=80=A6=20(#21591)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit (cherry picked from commit 98bf9dd72910e1b02dea17148a4199e3b26d7147) --- .../policies/data/OffloadPoliciesImpl.java | 96 +++++-------------- .../policies/data/OffloadPoliciesTest.java | 31 +++++- 2 files changed, 55 insertions(+), 72 deletions(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java index f9148ba8699fd..51e181811c228 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java @@ -30,6 +30,7 @@ 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.Properties; @@ -86,6 +87,7 @@ public class OffloadPoliciesImpl implements Serializable, OffloadPolicies { public static final Long DEFAULT_OFFLOAD_THRESHOLD_IN_BYTES = null; public static final Long DEFAULT_OFFLOAD_THRESHOLD_IN_SECONDS = null; public static final Long DEFAULT_OFFLOAD_DELETION_LAG_IN_MILLIS = null; + public static final String EXTRA_CONFIG_PREFIX = "managedLedgerOffloadExtraConfig"; public static final String OFFLOAD_THRESHOLD_NAME_IN_CONF_FILE = "managedLedgerOffloadAutoTriggerSizeThresholdBytes"; @@ -121,8 +123,7 @@ public class OffloadPoliciesImpl implements Serializable, OffloadPolicies { private OffloadedReadPriority managedLedgerOffloadedReadPriority = DEFAULT_OFFLOADED_READ_PRIORITY; @Configuration @JsonProperty(access = JsonProperty.Access.READ_WRITE) - private Map managedLedgerExtraConfigurations = null; - + private Map managedLedgerExtraConfigurations = new HashMap<>(); // s3 config, set by service configuration or cli @Configuration @JsonProperty(access = JsonProperty.Access.READ_WRITE) @@ -248,8 +249,7 @@ public static OffloadPoliciesImpl create(String driver, String region, String bu public static OffloadPoliciesImpl create(Properties properties) { OffloadPoliciesImpl data = new OffloadPoliciesImpl(); - Field[] fields = OffloadPoliciesImpl.class.getDeclaredFields(); - Arrays.stream(fields).forEach(f -> { + for (Field f : CONFIGURATION_FIELDS) { if (properties.containsKey(f.getName())) { try { f.setAccessible(true); @@ -260,14 +260,15 @@ public static OffloadPoliciesImpl create(Properties properties) { f.getName(), properties.get(f.getName())), e); } } - }); + } + Map extraConfigurations = properties.entrySet().stream() - .filter(entry -> entry.getKey().toString().startsWith("managedLedgerOffloadExtraConfig")) - .collect(Collectors.toMap( - entry -> entry.getKey().toString().replaceFirst("managedLedgerOffloadExtraConfig", ""), - entry -> entry.getValue().toString())); + .filter(entry -> entry.getKey().toString().startsWith(EXTRA_CONFIG_PREFIX)) + .collect(Collectors.toMap( + entry -> entry.getKey().toString().replaceFirst(EXTRA_CONFIG_PREFIX, ""), + entry -> entry.getValue().toString())); - data.setManagedLedgerExtraConfigurations(extraConfigurations); + data.getManagedLedgerExtraConfigurations().putAll(extraConfigurations); data.compatibleWithBrokerConfigFile(properties); return data; @@ -346,66 +347,21 @@ public boolean bucketValid() { public Properties toProperties() { Properties properties = new Properties(); - setProperty(properties, "managedLedgerOffloadedReadPriority", this.getManagedLedgerOffloadedReadPriority()); - setProperty(properties, "offloadersDirectory", this.getOffloadersDirectory()); - setProperty(properties, "managedLedgerOffloadDriver", this.getManagedLedgerOffloadDriver()); - setProperty(properties, "managedLedgerOffloadMaxThreads", - this.getManagedLedgerOffloadMaxThreads()); - setProperty(properties, "managedLedgerOffloadPrefetchRounds", - this.getManagedLedgerOffloadPrefetchRounds()); - setProperty(properties, "managedLedgerOffloadThresholdInBytes", - this.getManagedLedgerOffloadThresholdInBytes()); - setProperty(properties, "managedLedgerOffloadThresholdInSeconds", - this.getManagedLedgerOffloadThresholdInSeconds()); - setProperty(properties, "managedLedgerOffloadDeletionLagInMillis", - this.getManagedLedgerOffloadDeletionLagInMillis()); - setProperty(properties, "managedLedgerOffloadExtraConfigurations", - this.getManagedLedgerExtraConfigurations()); - - if (this.isS3Driver()) { - setProperty(properties, "s3ManagedLedgerOffloadRegion", - this.getS3ManagedLedgerOffloadRegion()); - setProperty(properties, "s3ManagedLedgerOffloadBucket", - this.getS3ManagedLedgerOffloadBucket()); - setProperty(properties, "s3ManagedLedgerOffloadServiceEndpoint", - this.getS3ManagedLedgerOffloadServiceEndpoint()); - setProperty(properties, "s3ManagedLedgerOffloadMaxBlockSizeInBytes", - this.getS3ManagedLedgerOffloadMaxBlockSizeInBytes()); - setProperty(properties, "s3ManagedLedgerOffloadCredentialId", - this.getS3ManagedLedgerOffloadCredentialId()); - setProperty(properties, "s3ManagedLedgerOffloadCredentialSecret", - this.getS3ManagedLedgerOffloadCredentialSecret()); - setProperty(properties, "s3ManagedLedgerOffloadRole", - this.getS3ManagedLedgerOffloadRole()); - setProperty(properties, "s3ManagedLedgerOffloadRoleSessionName", - this.getS3ManagedLedgerOffloadRoleSessionName()); - setProperty(properties, "s3ManagedLedgerOffloadReadBufferSizeInBytes", - this.getS3ManagedLedgerOffloadReadBufferSizeInBytes()); - } else if (this.isGcsDriver()) { - setProperty(properties, "gcsManagedLedgerOffloadRegion", - this.getGcsManagedLedgerOffloadRegion()); - setProperty(properties, "gcsManagedLedgerOffloadBucket", - this.getGcsManagedLedgerOffloadBucket()); - setProperty(properties, "gcsManagedLedgerOffloadMaxBlockSizeInBytes", - this.getGcsManagedLedgerOffloadMaxBlockSizeInBytes()); - setProperty(properties, "gcsManagedLedgerOffloadReadBufferSizeInBytes", - this.getGcsManagedLedgerOffloadReadBufferSizeInBytes()); - setProperty(properties, "gcsManagedLedgerOffloadServiceAccountKeyFile", - this.getGcsManagedLedgerOffloadServiceAccountKeyFile()); - } else if (this.isFileSystemDriver()) { - setProperty(properties, "fileSystemProfilePath", this.getFileSystemProfilePath()); - setProperty(properties, "fileSystemURI", this.getFileSystemURI()); - } - - setProperty(properties, "managedLedgerOffloadBucket", this.getManagedLedgerOffloadBucket()); - setProperty(properties, "managedLedgerOffloadRegion", this.getManagedLedgerOffloadRegion()); - setProperty(properties, "managedLedgerOffloadServiceEndpoint", - this.getManagedLedgerOffloadServiceEndpoint()); - setProperty(properties, "managedLedgerOffloadMaxBlockSizeInBytes", - this.getManagedLedgerOffloadMaxBlockSizeInBytes()); - setProperty(properties, "managedLedgerOffloadReadBufferSizeInBytes", - this.getManagedLedgerOffloadReadBufferSizeInBytes()); - + for (Field f : CONFIGURATION_FIELDS) { + try { + f.setAccessible(true); + if ("managedLedgerExtraConfigurations".equals(f.getName())) { + Map extraConfig = (Map) f.get(this); + extraConfig.forEach((key, value) -> { + setProperty(properties, EXTRA_CONFIG_PREFIX + key, value); + }); + } else { + setProperty(properties, f.getName(), f.get(this)); + } + } catch (Exception e) { + throw new IllegalArgumentException("An error occurred while processing the field: " + f.getName(), e); + } + } return properties; } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/OffloadPoliciesTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/OffloadPoliciesTest.java index d79d2c32ffa7f..bbede4e982044 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/OffloadPoliciesTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/policies/data/OffloadPoliciesTest.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.common.policies.data; +import static org.apache.pulsar.common.policies.data.OffloadPoliciesImpl.EXTRA_CONFIG_PREFIX; +import static org.testng.Assert.assertEquals; import java.io.DataInputStream; import java.io.File; import java.io.IOException; @@ -26,6 +28,7 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; +import java.util.HashMap; import java.util.Map; import java.util.Properties; import org.testng.Assert; @@ -436,8 +439,8 @@ private byte[] loadClassData(String name) throws IOException { @Test public void testCreateOffloadPoliciesWithExtraConfiguration() { Properties properties = new Properties(); - properties.put("managedLedgerOffloadExtraConfigKey1", "value1"); - properties.put("managedLedgerOffloadExtraConfigKey2", "value2"); + properties.put(EXTRA_CONFIG_PREFIX + "Key1", "value1"); + properties.put(EXTRA_CONFIG_PREFIX + "Key2", "value2"); OffloadPoliciesImpl policies = OffloadPoliciesImpl.create(properties); Map extraConfigurations = policies.getManagedLedgerExtraConfigurations(); @@ -445,4 +448,28 @@ public void testCreateOffloadPoliciesWithExtraConfiguration() { Assert.assertEquals(extraConfigurations.get("Key1"), "value1"); Assert.assertEquals(extraConfigurations.get("Key2"), "value2"); } + + /** + * Test toProperties as well as create from properties. + * @throws Exception + */ + @Test + public void testToProperties() throws Exception { + // Base information convert. + OffloadPoliciesImpl offloadPolicies = OffloadPoliciesImpl.create("aws-s3", "test-region", "test-bucket", + "http://test.endpoint", null, null, null, null, 32 * 1024 * 1024, 5 * 1024 * 1024, + 10 * 1024 * 1024L, 100L, 10000L, OffloadedReadPriority.TIERED_STORAGE_FIRST); + assertEquals(offloadPolicies, OffloadPoliciesImpl.create(offloadPolicies.toProperties())); + + // Set useless config to offload policies. Make sure convert conversion result is the same. + offloadPolicies.setFileSystemProfilePath("/test/file"); + assertEquals(offloadPolicies, OffloadPoliciesImpl.create(offloadPolicies.toProperties())); + + // Set extra config to offload policies. Make sure convert conversion result is the same. + Map extraConfiguration = new HashMap<>(); + extraConfiguration.put("key1", "value1"); + extraConfiguration.put("key2", "value2"); + offloadPolicies.setManagedLedgerExtraConfigurations(extraConfiguration); + assertEquals(offloadPolicies, OffloadPoliciesImpl.create(offloadPolicies.toProperties())); + } } From 7b24dd9aad884ddc7825fdd21bc4a14540201e94 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Wed, 22 Nov 2023 11:22:48 +0800 Subject: [PATCH 156/699] [improve][broker] Support not retaining null-key message during topic compaction (#21578) (#21601) --- conf/broker.conf | 3 + conf/standalone.conf | 5 +- .../pulsar/broker/ServiceConfiguration.java | 6 ++ .../pulsar/client/impl/RawBatchConverter.java | 19 +++++-- .../pulsar/compaction/TwoPhaseCompactor.java | 32 ++++++++--- .../pulsar/compaction/CompactionTest.java | 56 ++++++++++--------- 6 files changed, 84 insertions(+), 37 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index 94c89414504f9..811862aaa5034 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -538,6 +538,9 @@ brokerServiceCompactionThresholdInBytes=0 # If the execution time of the compaction phase one loop exceeds this time, the compaction will not proceed. brokerServiceCompactionPhaseOneLoopTimeInSeconds=30 +# Whether retain null-key message during topic compaction +topicCompactionRemainNullKey=true + # Whether to enable the delayed delivery for messages. # If disabled, messages will be immediately delivered and there will # be no tracking overhead. diff --git a/conf/standalone.conf b/conf/standalone.conf index 76223c5933e45..b730bbc1290e2 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -1277,4 +1277,7 @@ brokerInterceptorsDirectory=./interceptors brokerInterceptors= # Enable or disable the broker interceptor, which is only used for testing for now -disableBrokerInterceptors=true \ No newline at end of file +disableBrokerInterceptors=true + +# Whether retain null-key message during topic compaction +topicCompactionRemainNullKey=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 5b6b3acad3fac..f3255a83e29f3 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 @@ -2777,6 +2777,12 @@ The delayed message index time step(in seconds) in per bucket snapshot segment, ) private long brokerServiceCompactionPhaseOneLoopTimeInSeconds = 30; + @FieldContext( + category = CATEGORY_SERVER, + doc = "Whether retain null-key message during topic compaction." + ) + private boolean topicCompactionRemainNullKey = true; + @FieldContext( category = CATEGORY_SERVER, doc = "Interval between checks to see if cluster is migrated and marks topic migrated " diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java index 167cc1b699c39..b6be228788a7f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java @@ -92,6 +92,11 @@ public static List> extractIdsAndKey return idsAndKeysAndSize; } + public static Optional rebatchMessage(RawMessage msg, + BiPredicate filter) throws IOException { + return rebatchMessage(msg, filter, true); + } + /** * Take a batched message and a filter, and returns a message with the only the sub-messages * which match the filter. Returns an empty optional if no messages match. @@ -99,7 +104,8 @@ public static List> extractIdsAndKey * NOTE: this message does not alter the reference count of the RawMessage argument. */ public static Optional rebatchMessage(RawMessage msg, - BiPredicate filter) + BiPredicate filter, + boolean retainNullKey) throws IOException { checkArgument(msg.getMessageIdData().getBatchIndex() == -1); @@ -135,9 +141,14 @@ public static Optional rebatchMessage(RawMessage msg, msg.getMessageIdData().getPartition(), i); if (!singleMessageMetadata.hasPartitionKey()) { - messagesRetained++; - Commands.serializeSingleMessageInBatchWithPayload(singleMessageMetadata, - singleMessagePayload, batchBuffer); + if (retainNullKey) { + messagesRetained++; + Commands.serializeSingleMessageInBatchWithPayload(singleMessageMetadata, + singleMessagePayload, batchBuffer); + } else { + Commands.serializeSingleMessageInBatchWithPayload(emptyMetadata, + Unpooled.EMPTY_BUFFER, batchBuffer); + } } else if (filter.test(singleMessageMetadata.getPartitionKey(), id) && singleMessagePayload.readableBytes() > 0) { messagesRetained++; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java index e4e067ad6117e..cb39cc93154fb 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java @@ -62,6 +62,7 @@ public class TwoPhaseCompactor extends Compactor { private static final Logger log = LoggerFactory.getLogger(TwoPhaseCompactor.class); private static final int MAX_OUTSTANDING = 500; private final Duration phaseOneLoopReadTimeout; + private final boolean topicCompactionRemainNullKey; public TwoPhaseCompactor(ServiceConfiguration conf, PulsarClient pulsar, @@ -69,6 +70,7 @@ public TwoPhaseCompactor(ServiceConfiguration conf, ScheduledExecutorService scheduler) { super(conf, pulsar, bk, scheduler); phaseOneLoopReadTimeout = Duration.ofSeconds(conf.getBrokerServiceCompactionPhaseOneLoopTimeInSeconds()); + topicCompactionRemainNullKey = conf.isTopicCompactionRemainNullKey(); } @Override @@ -134,6 +136,14 @@ private void phaseOneLoop(RawReader reader, int deleteCnt = 0; for (ImmutableTriple e : extractIdsAndKeysAndSizeFromBatch(m)) { if (e != null) { + if (e.getMiddle() == null) { + if (!topicCompactionRemainNullKey) { + // record delete null-key message event + deleteCnt++; + mxBean.addCompactionRemovedEvent(reader.getTopic()); + } + continue; + } if (e.getRight() > 0) { MessageId old = latestForKey.put(e.getMiddle(), e.getLeft()); if (old != null) { @@ -163,6 +173,10 @@ private void phaseOneLoop(RawReader reader, deletedMessage = true; latestForKey.remove(keyAndSize.getLeft()); } + } else { + if (!topicCompactionRemainNullKey) { + deletedMessage = true; + } } if (replaceMessage || deletedMessage) { mxBean.addCompactionRemovedEvent(reader.getTopic()); @@ -249,8 +263,8 @@ private void phaseTwoLoop(RawReader reader, MessageId to, Map mxBean.addCompactionReadOp(reader.getTopic(), m.getHeadersAndPayload().readableBytes()); if (RawBatchConverter.isReadableBatch(m)) { try { - messageToAdd = rebatchMessage( - m, (key, subid) -> subid.equals(latestForKey.get(key))); + messageToAdd = rebatchMessage(reader.getTopic(), + m, (key, subid) -> subid.equals(latestForKey.get(key)), topicCompactionRemainNullKey); } catch (IOException ioe) { log.info("Error decoding batch for message {}. Whole batch will be included in output", id, ioe); @@ -259,8 +273,8 @@ private void phaseTwoLoop(RawReader reader, MessageId to, Map } else { Pair keyAndSize = extractKeyAndSize(m); MessageId msg; - if (keyAndSize == null) { // pass through messages without a key - messageToAdd = Optional.of(m); + if (keyAndSize == null) { + messageToAdd = topicCompactionRemainNullKey ? Optional.of(m) : Optional.empty(); } else if ((msg = latestForKey.get(keyAndSize.getLeft())) != null && msg.equals(id)) { // consider message only if present into latestForKey map if (keyAndSize.getRight() <= 0) { @@ -416,12 +430,16 @@ protected Pair extractKeyAndSize(RawMessage m) { protected List> extractIdsAndKeysAndSizeFromBatch(RawMessage msg) throws IOException { - return RawBatchConverter.extractIdsAndKeysAndSize(msg, false); + return RawBatchConverter.extractIdsAndKeysAndSize(msg); } - protected Optional rebatchMessage(RawMessage msg, BiPredicate filter) + protected Optional rebatchMessage(String topic, RawMessage msg, BiPredicate filter, + boolean retainNullKey) throws IOException { - return RawBatchConverter.rebatchMessage(msg, filter); + if (log.isDebugEnabled()) { + log.debug("Rebatching message {} for topic {}", msg.getMessageId(), topic); + } + return RawBatchConverter.rebatchMessage(msg, filter, retainNullKey); } private static class PhaseOneResult { 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 52837cbdcd56a..5ee12d660e031 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 @@ -26,6 +26,7 @@ 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 com.google.common.util.concurrent.ThreadFactoryBuilder; import io.netty.buffer.ByteBuf; @@ -640,8 +641,17 @@ public void testWholeBatchCompactedOut() throws Exception { } } - @Test - public void testKeyLessMessagesPassThrough() throws Exception { + @DataProvider(name = "retainNullKey") + public static Object[][] retainNullKey() { + return new Object[][] {{true}, {false}}; + } + + @Test(dataProvider = "retainNullKey") + public void testKeyLessMessagesPassThrough(boolean retainNullKey) throws Exception { + conf.setTopicCompactionRemainNullKey(retainNullKey); + restartBroker(); + FieldUtils.writeDeclaredField(compactor, "topicCompactionRemainNullKey", retainNullKey, true); + String topic = "persistent://my-property/use/my-ns/my-topic1"; // subscribe before sending anything, so that we get all messages @@ -682,29 +692,25 @@ public void testKeyLessMessagesPassThrough() throws Exception { Message m = consumer.receive(2, TimeUnit.SECONDS); assertNull(m); } else { - Message message1 = consumer.receive(); - Assert.assertFalse(message1.hasKey()); - Assert.assertEquals(new String(message1.getData()), "my-message-1"); - - Message message2 = consumer.receive(); - Assert.assertFalse(message2.hasKey()); - Assert.assertEquals(new String(message2.getData()), "my-message-2"); - - Message message3 = consumer.receive(); - Assert.assertEquals(message3.getKey(), "key1"); - Assert.assertEquals(new String(message3.getData()), "my-message-4"); - - Message message4 = consumer.receive(); - Assert.assertEquals(message4.getKey(), "key2"); - Assert.assertEquals(new String(message4.getData()), "my-message-6"); - - Message message5 = consumer.receive(); - Assert.assertFalse(message5.hasKey()); - Assert.assertEquals(new String(message5.getData()), "my-message-7"); + List> result = new ArrayList<>(); + while (true) { + Message message = consumer.receive(10, TimeUnit.SECONDS); + if (message == null) { + break; + } + result.add(Pair.of(message.getKey(), message.getData() == null ? null : new String(message.getData()))); + } - Message message6 = consumer.receive(); - Assert.assertFalse(message6.hasKey()); - Assert.assertEquals(new String(message6.getData()), "my-message-8"); + List> expectList; + if (retainNullKey) { + expectList = List.of( + Pair.of(null, "my-message-1"), Pair.of(null, "my-message-2"), + Pair.of("key1", "my-message-4"), Pair.of("key2", "my-message-6"), + Pair.of(null, "my-message-7"), Pair.of(null, "my-message-8")); + } else { + expectList = List.of(Pair.of("key1", "my-message-4"), Pair.of("key2", "my-message-6")); + } + Assert.assertEquals(result, expectList); } } } @@ -1885,7 +1891,7 @@ public void testDispatcherMaxReadSizeBytes() throws Exception { .topic(topicName).create(); for (int i = 0; i < 10; i+=2) { - producer.newMessage().key(null).value(new byte[4*1024*1024]).send(); + producer.newMessage().key(UUID.randomUUID().toString()).value(new byte[4*1024*1024]).send(); } producer.flush(); From 855dfc110a2744b5077cb59ac1b71c9c0acda2ff Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 21 Nov 2023 17:21:51 +0800 Subject: [PATCH 157/699] [fix] [broker] Delete topic timeout due to NPE (#21595) ### Issue: There is an NPE that causes the Future of Delay message indexes bucket deletion to be no longer complete, which leads to the topic deletion timeout. You can reproduce this issue by the test `testDeletePartitionedTopicIfCursorPropsEmpty` and `testDeleteTopicIfCursorPropsEmpty` ### Modifications Fix the NPE. (cherry picked from commit b2f2b53907e43d0eb6757bfc4b77bf3db027f251) --- .../mledger/impl/ManagedCursorImpl.java | 2 +- .../BucketDelayedDeliveryTrackerFactory.java | 4 + .../bucket/BucketDelayedDeliveryTracker.java | 7 +- .../persistent/BucketDelayedDeliveryTest.java | 123 ++++++++++++++++++ 4 files changed, 134 insertions(+), 2 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 ff8e0655d03be..ea013d2da7dd7 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 @@ -690,7 +690,7 @@ private void recoveredCursor(PositionImpl position, Map properties position = ledger.getLastPosition(); } log.info("[{}] Cursor {} recovered to position {}", ledger.getName(), name, position); - this.cursorProperties = cursorProperties; + this.cursorProperties = cursorProperties == null ? Collections.emptyMap() : cursorProperties; messagesConsumedCounter = -getNumberOfEntries(Range.openClosed(position, ledger.getLastPosition())); markDeletePosition = position; persistentMarkDeletePosition = position; 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 6a00bfd199584..33076fd51a8e9 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 @@ -27,6 +27,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.commons.collections4.MapUtils; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.delayed.bucket.BookkeeperBucketSnapshotStorage; @@ -85,6 +86,9 @@ public DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers d */ public CompletableFuture cleanResidualSnapshots(ManagedCursor cursor) { Map cursorProperties = cursor.getCursorProperties(); + if (MapUtils.isEmpty(cursorProperties)) { + return CompletableFuture.completedFuture(null); + } List> futures = new ArrayList<>(); FutureUtil.Sequencer sequencer = FutureUtil.Sequencer.create(); cursorProperties.forEach((k, v) -> { 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 67a7de1f01339..d7a3e80f086d2 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 @@ -50,6 +50,7 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.collections4.MapUtils; import org.apache.commons.lang3.mutable.MutableLong; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.delayed.AbstractDelayedDeliveryTracker; @@ -137,9 +138,13 @@ public BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispat private synchronized long recoverBucketSnapshot() throws RuntimeException { ManagedCursor cursor = this.lastMutableBucket.getCursor(); + Map cursorProperties = cursor.getCursorProperties(); + if (MapUtils.isEmpty(cursorProperties)) { + return 0; + } FutureUtil.Sequencer sequencer = this.lastMutableBucket.getSequencer(); Map, ImmutableBucket> toBeDeletedBucketMap = new HashMap<>(); - cursor.getCursorProperties().keySet().forEach(key -> { + cursorProperties.keySet().forEach(key -> { if (key.startsWith(DELAYED_BUCKET_KEY_PREFIX)) { String[] keys = key.split(DELIMITER); checkArgument(keys.length == 3); 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 0a82b2b4c3cb0..54fec3934ddbc 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 @@ -20,10 +20,13 @@ import static org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.CURSOR_INTERNAL_PROPERTY_PREFIX; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import com.google.common.collect.Multimap; import java.io.ByteArrayOutputStream; import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Optional; @@ -32,6 +35,7 @@ import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerHandle; +import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.pulsar.broker.BrokerTestUtil; @@ -40,6 +44,7 @@ import org.apache.pulsar.broker.stats.PrometheusMetricsTest; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; 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.Schema; import org.apache.pulsar.client.api.SubscriptionType; @@ -47,6 +52,7 @@ import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Test(groups = "broker") @@ -353,4 +359,121 @@ public void testDelete() throws Exception { } } } + + @DataProvider(name = "subscriptionTypes") + public Object[][] subscriptionTypes() { + return new Object[][]{ + {SubscriptionType.Shared}, + {SubscriptionType.Key_Shared}, + {SubscriptionType.Failover}, + {SubscriptionType.Exclusive}, + }; + } + + /** + * see: https://github.com/apache/pulsar/pull/21595. + */ + @Test(dataProvider = "subscriptionTypes") + public void testDeleteTopicIfCursorPropsEmpty(SubscriptionType subscriptionType) throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/tp_"); + final String subscriptionName = "s1"; + // create a topic. + admin.topics().createNonPartitionedTopic(topic); + // create a subscription without props. + admin.topics().createSubscription(topic, subscriptionName, MessageId.earliest); + pulsarClient.newConsumer().topic(topic).subscriptionName(subscriptionName) + .subscriptionType(subscriptionType).subscribe().close(); + ManagedCursorImpl cursor = findCursor(topic, subscriptionName); + assertNotNull(cursor); + assertTrue(cursor.getCursorProperties() == null || cursor.getCursorProperties().isEmpty()); + // Test topic deletion is successful. + admin.topics().delete(topic); + } + + /** + * see: https://github.com/apache/pulsar/pull/21595. + */ + @Test(dataProvider = "subscriptionTypes") + public void testDeletePartitionedTopicIfCursorPropsEmpty(SubscriptionType subscriptionType) throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/tp_"); + final String subscriptionName = "s1"; + // create a topic. + admin.topics().createPartitionedTopic(topic, 2); + // create a subscription without props. + admin.topics().createSubscription(topic, subscriptionName, MessageId.earliest); + pulsarClient.newConsumer().topic(topic).subscriptionName(subscriptionName) + .subscriptionType(subscriptionType).subscribe().close(); + ManagedCursorImpl cursor = findCursor(topic + "-partition-0", subscriptionName); + assertNotNull(cursor); + assertTrue(cursor.getCursorProperties() == null || cursor.getCursorProperties().isEmpty()); + // Test topic deletion is successful. + admin.topics().deletePartitionedTopic(topic); + } + + /** + * see: https://github.com/apache/pulsar/pull/21595. + */ + @Test(dataProvider = "subscriptionTypes") + public void testDeleteTopicIfCursorPropsNotEmpty(SubscriptionType subscriptionType) throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/tp_"); + final String subscriptionName = "s1"; + // create a topic. + admin.topics().createNonPartitionedTopic(topic); + // create a subscription without props. + admin.topics().createSubscription(topic, subscriptionName, MessageId.earliest); + pulsarClient.newConsumer().topic(topic).subscriptionName(subscriptionName) + .subscriptionType(subscriptionType).subscribe().close(); + ManagedCursorImpl cursor = findCursor(topic, subscriptionName); + assertNotNull(cursor); + assertTrue(cursor.getCursorProperties() == null || cursor.getCursorProperties().isEmpty()); + // Put a subscription prop. + Map properties = new HashMap<>(); + properties.put("ignore", "ignore"); + admin.topics().updateSubscriptionProperties(topic, subscriptionName, properties); + assertTrue(cursor.getCursorProperties() != null && !cursor.getCursorProperties().isEmpty()); + // Test topic deletion is successful. + admin.topics().delete(topic); + } + + /** + * see: https://github.com/apache/pulsar/pull/21595. + */ + @Test(dataProvider = "subscriptionTypes") + public void testDeletePartitionedTopicIfCursorPropsNotEmpty(SubscriptionType subscriptionType) throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/tp_"); + final String subscriptionName = "s1"; + // create a topic. + admin.topics().createPartitionedTopic(topic, 2); + pulsarClient.newProducer().topic(topic).create().close(); + // create a subscription without props. + admin.topics().createSubscription(topic, subscriptionName, MessageId.earliest); + pulsarClient.newConsumer().topic(topic).subscriptionName(subscriptionName) + .subscriptionType(subscriptionType).subscribe().close(); + + ManagedCursorImpl cursor = findCursor(topic + "-partition-0", subscriptionName); + assertNotNull(cursor); + assertTrue(cursor.getCursorProperties() == null || cursor.getCursorProperties().isEmpty()); + // Put a subscription prop. + Map properties = new HashMap<>(); + properties.put("ignore", "ignore"); + admin.topics().updateSubscriptionProperties(topic, subscriptionName, properties); + assertTrue(cursor.getCursorProperties() != null && !cursor.getCursorProperties().isEmpty()); + // Test topic deletion is successful. + admin.topics().deletePartitionedTopic(topic); + } + + + private ManagedCursorImpl findCursor(String topic, String subscriptionName) { + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + Iterator cursorIterator = persistentTopic.getManagedLedger().getCursors().iterator(); + while (cursorIterator.hasNext()) { + ManagedCursor managedCursor = cursorIterator.next(); + if (managedCursor == null || !managedCursor.getName().equals(subscriptionName)) { + continue; + } + return (ManagedCursorImpl) managedCursor; + } + return null; + } } From 3dcd986a13b5f4222720fcad2fc15fd10823f4c6 Mon Sep 17 00:00:00 2001 From: Dezhi LIiu <33149602+liudezhi2098@users.noreply.github.com> Date: Sun, 12 Nov 2023 15:15:43 +0800 Subject: [PATCH 158/699] [fix][broker] Fix setReplicatedSubscriptionStatus incorrect behavior (#21510) --- .../persistent/PersistentSubscription.java | 7 ++- .../service/ReplicatorSubscriptionTest.java | 24 ++++++++ .../broker/service/ReplicatorTestBase.java | 58 ++++++++++++++++++- 3 files changed, 87 insertions(+), 2 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 1d8f78385778c..cf4af49437b8a 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 @@ -199,7 +199,12 @@ public boolean setReplicated(boolean replicated) { if (this.cursor != null) { if (replicated) { - return this.cursor.putProperty(REPLICATED_SUBSCRIPTION_PROPERTY, 1L); + if (!config.isEnableReplicatedSubscriptions()) { + log.warn("[{}][{}] Failed set replicated subscription status to {}, please enable the " + + "configuration enableReplicatedSubscriptions", topicName, subName, replicated); + } else { + return this.cursor.putProperty(REPLICATED_SUBSCRIPTION_PROPERTY, 1L); + } } else { return this.cursor.removeProperty(REPLICATED_SUBSCRIPTION_PROPERTY); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java index 2816a973c92da..f816aa2dd244a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java @@ -41,6 +41,7 @@ import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.persistent.ReplicatedSubscriptionsController; +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.MessageId; @@ -50,11 +51,13 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.policies.data.PartitionedTopicStats; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; 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.BeforeClass; import org.testng.annotations.Test; @@ -621,6 +624,27 @@ public void testReplicatedSubscriptionRestApi2() throws Exception { String.format("numReceivedMessages2 (%d) should be less than %d", numReceivedMessages2, numMessages)); } + @Test(timeOut = 30000) + public void testReplicatedSubscriptionRestApi3() throws Exception { + final String namespace = BrokerTestUtil.newUniqueName("geo/replicatedsubscription"); + final String topicName = "persistent://" + namespace + "/topic-rest-api3"; + final String subName = "sub"; + admin4.tenants().createTenant("geo", + new TenantInfoImpl(Sets.newHashSet("appid1", "appid4"), Sets.newHashSet(cluster1, cluster4))); + admin4.namespaces().createNamespace(namespace); + admin4.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet(cluster1, cluster4)); + admin4.topics().createPartitionedTopic(topicName, 2); + + @Cleanup + final PulsarClient client4 = PulsarClient.builder().serviceUrl(url4.toString()) + .statsInterval(0, TimeUnit.SECONDS).build(); + + Consumer consumer4 = client4.newConsumer().topic(topicName).subscriptionName(subName).subscribe(); + Assert.expectThrows(PulsarAdminException.class, () -> + admin4.topics().setReplicatedSubscriptionStatus(topicName, subName, true)); + consumer4.close(); + } + /** * Tests replicated subscriptions when replicator producer is closed */ diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java index b83e8ac9d2dbf..beb1a3c4b9309 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java @@ -82,6 +82,13 @@ public abstract class ReplicatorTestBase extends TestRetrySupport { PulsarAdmin admin3; LocalBookkeeperEnsemble bkEnsemble3; + URL url4; + URL urlTls4; + ServiceConfiguration config4 = new ServiceConfiguration(); + PulsarService pulsar4; + PulsarAdmin admin4; + LocalBookkeeperEnsemble bkEnsemble4; + ZookeeperServerTest globalZkS; ExecutorService executor; @@ -111,6 +118,7 @@ public abstract class ReplicatorTestBase extends TestRetrySupport { protected final String cluster1 = "r1"; protected final String cluster2 = "r2"; protected final String cluster3 = "r3"; + protected final String cluster4 = "r4"; // Default frequency public int getBrokerServicePurgeInactiveFrequency() { @@ -178,6 +186,21 @@ protected void setup() throws Exception { urlTls3 = new URL(pulsar3.getWebServiceAddressTls()); admin3 = PulsarAdmin.builder().serviceHttpUrl(url3.toString()).build(); + // Start region 4 + + // Start zk & bks + bkEnsemble4 = new LocalBookkeeperEnsemble(3, 0, () -> 0); + bkEnsemble4.start(); + + setConfig4DefaultValue(); + pulsar4 = new PulsarService(config4); + pulsar4.start(); + + url4 = new URL(pulsar4.getWebServiceAddress()); + urlTls4 = new URL(pulsar4.getWebServiceAddressTls()); + admin4 = PulsarAdmin.builder().serviceHttpUrl(url4.toString()).build(); + + // Provision the global namespace admin1.clusters().createCluster(cluster1, ClusterData.builder() .serviceUrl(url1.toString()) @@ -230,6 +253,23 @@ protected void setup() throws Exception { .brokerClientTlsTrustStorePassword(keyStorePassword) .brokerClientTlsTrustStoreType(keyStoreType) .build()); + admin4.clusters().createCluster(cluster4, ClusterData.builder() + .serviceUrl(url4.toString()) + .serviceUrlTls(urlTls4.toString()) + .brokerServiceUrl(pulsar4.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar4.getBrokerServiceUrlTls()) + .brokerClientTlsEnabled(true) + .brokerClientCertificateFilePath(clientCertFilePath) + .brokerClientKeyFilePath(clientKeyFilePath) + .brokerClientTrustCertsFilePath(caCertFilePath) + .brokerClientTlsEnabledWithKeyStore(tlsWithKeyStore) + .brokerClientTlsKeyStore(clientKeyStorePath) + .brokerClientTlsKeyStorePassword(keyStorePassword) + .brokerClientTlsKeyStoreType(keyStoreType) + .brokerClientTlsTrustStore(clientTrustStorePath) + .brokerClientTlsTrustStorePassword(keyStorePassword) + .brokerClientTlsTrustStoreType(keyStoreType) + .build()); admin1.tenants().createTenant("pulsar", new TenantInfoImpl(Sets.newHashSet("appid1", "appid2", "appid3"), Sets.newHashSet("r1", "r2", "r3"))); @@ -257,7 +297,7 @@ protected void setup() throws Exception { } public void setConfig3DefaultValue() { - setConfigDefaults(config3, "r3", bkEnsemble3); + setConfigDefaults(config3, cluster3, bkEnsemble3); config3.setTlsEnabled(true); } @@ -269,6 +309,11 @@ public void setConfig2DefaultValue() { setConfigDefaults(config2, cluster2, bkEnsemble2); } + public void setConfig4DefaultValue() { + setConfigDefaults(config4, cluster4, bkEnsemble4); + config4.setEnableReplicatedSubscriptions(false); + } + private void setConfigDefaults(ServiceConfiguration config, String clusterName, LocalBookkeeperEnsemble bookkeeperEnsemble) { config.setClusterName(clusterName); @@ -316,6 +361,11 @@ public void resetConfig3() { setConfig3DefaultValue(); } + public void resetConfig4() { + config4 = new ServiceConfiguration(); + setConfig4DefaultValue(); + } + private int inSec(int time, TimeUnit unit) { return (int) TimeUnit.SECONDS.convert(time, unit); } @@ -332,7 +382,11 @@ protected void cleanup() throws Exception { admin1.close(); admin2.close(); admin3.close(); + admin4.close(); + if (pulsar4 != null) { + pulsar4.close(); + } if (pulsar3 != null) { pulsar3.close(); } @@ -346,11 +400,13 @@ protected void cleanup() throws Exception { bkEnsemble1.stop(); bkEnsemble2.stop(); bkEnsemble3.stop(); + bkEnsemble4.stop(); globalZkS.stop(); resetConfig1(); resetConfig2(); resetConfig3(); + resetConfig4(); } static class MessageProducer implements AutoCloseable { From 0bc30a294052bd346ca00349012880fe41aabe96 Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Tue, 14 Nov 2023 20:35:29 +0800 Subject: [PATCH 159/699] [fix][broker] Do not write replicated snapshot marker when the topic which is not enable replication (#21495) ### Motivation [PIP 33](https://github.com/apache/pulsar/wiki/PIP-33%3A-Replicated-subscriptions) introduces a new concept ` Replicated subscriptions`. When a topic has a consumer (subscription) that enables replicated subscriptions, it will write markers into the original topic. Even if there is no replicated cluster configured for this topic, the mark will still be written. And that will make the backlog of the topic keep increasing. --- The mark will be written in the following two ways: 1. A scheduled task writes a marker at a fixed time interval if there are new messages published. https://github.com/apache/pulsar/blob/ea1fc0f20138bc35f54f55d32dabf3c3a3309c8e/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java#L78-L86 https://github.com/apache/pulsar/blob/ea1fc0f20138bc35f54f55d32dabf3c3a3309c8e/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsSnapshotBuilder.java#L77-L86 2. Acknowledging message will trigger a check if the first snapshot is written and the mark delete position moves, if true, It will write a marker. https://github.com/apache/pulsar/blob/ea1fc0f20138bc35f54f55d32dabf3c3a3309c8e/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/ReplicatedSubscriptionsController.java#L114-L150 ### Modifications According to the topic policy to create or remove `ReplicatedSubscriptionsController` of this topic. --- .../service/persistent/PersistentTopic.java | 9 +- .../service/ReplicatorSubscriptionTest.java | 211 ++++++++++++++++++ 2 files changed, 217 insertions(+), 3 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 a090aa1bf2565..d0de36c624211 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 @@ -2819,7 +2819,7 @@ public CompletableFuture onPoliciesUpdate(@Nonnull Policies data) { } updateTopicPolicyByNamespacePolicy(data); - + checkReplicatedSubscriptionControllerState(); isEncryptionRequired = data.encryption_required; isAllowAutoUpdateSchema = data.is_allow_auto_update_schema; @@ -3306,12 +3306,14 @@ private synchronized void checkReplicatedSubscriptionControllerState(boolean sho boolean isCurrentlyEnabled = replicatedSubscriptionsController.isPresent(); boolean isEnableReplicatedSubscriptions = brokerService.pulsar().getConfiguration().isEnableReplicatedSubscriptions(); + boolean replicationEnabled = this.topicPolicies.getReplicationClusters().get().size() > 1; - if (shouldBeEnabled && !isCurrentlyEnabled && isEnableReplicatedSubscriptions) { + if (shouldBeEnabled && !isCurrentlyEnabled && isEnableReplicatedSubscriptions && replicationEnabled) { log.info("[{}] Enabling replicated subscriptions controller", topic); replicatedSubscriptionsController = Optional.of(new ReplicatedSubscriptionsController(this, brokerService.pulsar().getConfiguration().getClusterName())); - } else if (isCurrentlyEnabled && !shouldBeEnabled || !isEnableReplicatedSubscriptions) { + } else if (isCurrentlyEnabled && !shouldBeEnabled || !isEnableReplicatedSubscriptions + || !replicationEnabled) { log.info("[{}] Disabled replicated subscriptions controller", topic); replicatedSubscriptionsController.ifPresent(ReplicatedSubscriptionsController::close); replicatedSubscriptionsController = Optional.empty(); @@ -3494,6 +3496,7 @@ public void onUpdate(TopicPolicies policies) { updateTopicPolicy(policies); shadowTopics = policies.getShadowTopics(); updateDispatchRateLimiter(); + checkReplicatedSubscriptionControllerState(); updateSubscriptionsDispatcherRateLimiter().thenRun(() -> { updatePublishDispatcher(); updateSubscribeRateLimiter(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java index f816aa2dd244a..529fb923f5918 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java @@ -32,6 +32,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.LinkedHashSet; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -50,7 +51,9 @@ import org.apache.pulsar.client.api.PulsarClient; 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.policies.data.PartitionedTopicStats; +import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; @@ -60,6 +63,7 @@ import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; /** @@ -728,6 +732,213 @@ public void testReplicatedSubscriptionWhenReplicatorProducerIsClosed() throws Ex Awaitility.await().untilAsserted(() -> assertNotNull(topic2.getSubscription(subscriptionName))); } + @DataProvider(name = "isTopicPolicyEnabled") + private Object[][] isTopicPolicyEnabled() { + // Todo: fix replication can not be enabled at topic level. + return new Object[][] { { Boolean.FALSE } }; + } + + /** + * Test the replication subscription can work normal in the following cases: + *

+ * 1. Do not write data into the original topic when the topic does not configure a remote cluster. {topic1} + * 1. Publish message to the topic and then wait a moment, + * the backlog will not increase after publishing completely. + * 2. Acknowledge the messages, the last confirm entry does not change. + * 2. Snapshot and mark will be written after topic configure a remote cluster. {topic2} + * 1. publish message to topic. After publishing completely, the backlog of the topic keep increase. + * 2. Wait the snapshot complete, the backlog stop changing. + * 3. Publish messages to wait another snapshot complete. + * 4. Ack messages to move the mark delete position after the position record in the first snapshot. + * 5. Check new entry (a mark) appending to the original topic. + * 3. Stopping writing snapshot and mark after remove the remote cluster of the topic. {topic2} + * similar to step 1. + *

+ */ + @Test(dataProvider = "isTopicPolicyEnabled") + public void testWriteMarkerTaskOfReplicateSubscriptions(boolean isTopicPolicyEnabled) throws Exception { + // 1. Prepare resource and use proper configuration. + String namespace = BrokerTestUtil.newUniqueName("pulsar/testReplicateSubBackLog"); + String topic1 = "persistent://" + namespace + "/replication-enable"; + String topic2 = "persistent://" + namespace + "/replication-disable"; + String subName = "sub"; + + admin1.namespaces().createNamespace(namespace); + pulsar1.getConfiguration().setTopicLevelPoliciesEnabled(isTopicPolicyEnabled); + pulsar1.getConfiguration().setReplicationPolicyCheckDurationSeconds(1); + pulsar1.getConfiguration().setReplicatedSubscriptionsSnapshotFrequencyMillis(1000); + // 2. Build Producer and Consumer. + @Cleanup + PulsarClient client1 = PulsarClient.builder().serviceUrl(url1.toString()) + .statsInterval(0, TimeUnit.SECONDS) + .build(); + @Cleanup + Consumer consumer1 = client1.newConsumer() + .topic(topic1) + .subscriptionName(subName) + .ackTimeout(5, TimeUnit.SECONDS) + .subscriptionType(SubscriptionType.Shared) + .replicateSubscriptionState(true) + .subscribe(); + @Cleanup + Producer producer1 = client1.newProducer() + .topic(topic1) + .create(); + // 3. Test replication subscription work as expected. + // Test case 1: disable replication, backlog will not increase. + testReplicatedSubscriptionWhenDisableReplication(producer1, consumer1, topic1); + + // Test case 2: enable replication, mark and snapshot work as expected. + if (isTopicPolicyEnabled) { + admin1.topics().createNonPartitionedTopic(topic2); + admin1.topics().setReplicationClusters(topic2, List.of("r1", "r2")); + } else { + admin1.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("r1", "r2")); + } + @Cleanup + Consumer consumer2 = client1.newConsumer() + .topic(topic2) + .subscriptionName(subName) + .ackTimeout(5, TimeUnit.SECONDS) + .subscriptionType(SubscriptionType.Shared) + .replicateSubscriptionState(true) + .subscribe(); + @Cleanup + Producer producer2 = client1.newProducer() + .topic(topic2) + .create(); + testReplicatedSubscriptionWhenEnableReplication(producer2, consumer2, topic2); + + // Test case 3: enable replication, mark and snapshot work as expected. + if (isTopicPolicyEnabled) { + admin1.topics().setReplicationClusters(topic2, List.of("r1")); + } else { + admin1.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("r1")); + } + testReplicatedSubscriptionWhenDisableReplication(producer2, consumer2, topic2); + // 4. Clear resource. + pulsar1.getConfiguration().setForceDeleteNamespaceAllowed(true); + admin1.namespaces().deleteNamespace(namespace, true); + pulsar1.getConfiguration().setForceDeleteNamespaceAllowed(false); + } + + /** + * Disable replication subscription. + * Test scheduled task case. + * 1. Send three messages |1:0|1:1|1:2|. + * 2. Get topic backlog, as backlog1. + * 3. Wait a moment. + * 4. Get the topic backlog again, the backlog will not increase. + * Test acknowledge messages case. + * 1. Get the last confirm entry, as LAC1. + * 2. Acknowledge these messages |1:0|1:1|. + * 3. wait a moment. + * 4. Get the last confirm entry, as LAC2. LAC1 is equal to LAC2. + * Clear environment. + * 1. Ack all the retained messages. |1:2| + * 2. Wait for the backlog to return to zero. + */ + private void testReplicatedSubscriptionWhenDisableReplication(Producer producer, Consumer consumer, + String topic) throws Exception { + final int messageSum = 3; + // Test scheduled task case. + for (int i = 0; i < messageSum; i++) { + producer.newMessage().send(); + } + long backlog1 = admin1.topics().getStats(topic, false).getBacklogSize(); + Thread.sleep(3000); + long backlog2 = admin1.topics().getStats(topic, false).getBacklogSize(); + assertEquals(backlog1, backlog2); + // Test acknowledge messages case. + String lastConfirmEntry1 = admin1.topics().getInternalStats(topic).lastConfirmedEntry; + for (int i = 0; i < messageSum - 1; i++) { + consumer.acknowledge(consumer.receive(5, TimeUnit.SECONDS)); + } + Awaitility.await().untilAsserted(() -> { + String lastConfirmEntry2 = admin1.topics().getInternalStats(topic).lastConfirmedEntry; + assertEquals(lastConfirmEntry1, lastConfirmEntry2); + }); + // Clear environment. + consumer.acknowledge(consumer.receive(5, TimeUnit.SECONDS)); + Awaitility.await().untilAsserted(() -> { + long backlog4 = admin1.topics().getStats(topic, false).getBacklogSize(); + assertEquals(backlog4, 0); + }); + } + + /** + * Enable replication subscription. + * Test scheduled task case. + * 1. Wait replicator connected. + * 2. Send three messages |1:0|1:1|1:2|. + * 3. Get topic backlog, as backlog1. + * 4. Wait a moment. + * 5. Get the topic backlog again, as backlog2. The backlog2 is bigger than backlog1. |1:0|1:1|1:2|mark|. + * 6. Wait the snapshot complete. + * Test acknowledge messages case. + * 1. Write messages and wait another snapshot complete. |1:0|1:1|1:2|mark|1:3|1:4|1:5|mark| + * 2. Ack message |1:0|1:1|1:2|1:3|1:4|. + * 3. Get last confirm entry, as LAC1. + * 2. Wait a moment. + * 3. Get Last confirm entry, as LAC2. LAC2 different to LAC1. |1:5|mark|mark| + * Clear environment. + * 1. Ack all the retained message |1:5|. + * 2. Wait for the backlog to return to zero. + */ + private void testReplicatedSubscriptionWhenEnableReplication(Producer producer, Consumer consumer, + String topic) throws Exception { + final int messageSum = 3; + Awaitility.await().untilAsserted(() -> { + List keys = pulsar1.getBrokerService() + .getTopic(topic, false).get().get() + .getReplicators().keys(); + assertEquals(keys.size(), 1); + assertTrue(pulsar1.getBrokerService() + .getTopic(topic, false).get().get() + .getReplicators().get(keys.get(0)).isConnected()); + }); + // Test scheduled task case. + sendMessageAndWaitSnapshotComplete(producer, topic, messageSum); + // Test acknowledge messages case. + // After snapshot write completely, acknowledging message to move the mark delete position + // after the position recorded in the snapshot will trigger to write a new marker. + sendMessageAndWaitSnapshotComplete(producer, topic, messageSum); + String lastConfirmedEntry3 = admin1.topics().getInternalStats(topic, false).lastConfirmedEntry; + for (int i = 0; i < messageSum * 2 - 1; i++) { + consumer.acknowledge(consumer.receive(5, TimeUnit.SECONDS)); + } + Awaitility.await().untilAsserted(() -> { + String lastConfirmedEntry4 = admin1.topics().getInternalStats(topic, false).lastConfirmedEntry; + assertNotEquals(lastConfirmedEntry3, lastConfirmedEntry4); + }); + // Clear environment. + consumer.acknowledge(consumer.receive(5, TimeUnit.SECONDS)); + Awaitility.await().untilAsserted(() -> { + long backlog4 = admin1.topics().getStats(topic, false).getBacklogSize(); + assertEquals(backlog4, 0); + }); + } + + private void sendMessageAndWaitSnapshotComplete(Producer producer, String topic, + int messageSum) throws Exception { + for (int i = 0; i < messageSum; i++) { + producer.newMessage().send(); + } + long backlog1 = admin1.topics().getStats(topic, false).getBacklogSize(); + Awaitility.await().untilAsserted(() -> { + long backlog2 = admin1.topics().getStats(topic, false).getBacklogSize(); + assertTrue(backlog2 > backlog1); + }); + // Wait snapshot write completely, stop writing marker into topic. + Awaitility.await().untilAsserted(() -> { + String lastConfirmedEntry1 = admin1.topics().getInternalStats(topic, false).lastConfirmedEntry; + PersistentTopicInternalStats persistentTopicInternalStats = admin1.topics().getInternalStats(topic, false); + Thread.sleep(1000); + String lastConfirmedEntry2 = admin1.topics().getInternalStats(topic, false).lastConfirmedEntry; + assertEquals(lastConfirmedEntry1, lastConfirmedEntry2); + }); + } + void publishMessages(Producer producer, int startIndex, int numMessages, Set sentMessages) throws PulsarClientException { for (int i = startIndex; i < startIndex + numMessages; i++) { From 6b6fd09d859fb16eac454774c226e5a4ffe0fb97 Mon Sep 17 00:00:00 2001 From: houxiaoyu Date: Thu, 16 Nov 2023 11:08:47 +0800 Subject: [PATCH 160/699] [fix][broker] Fix resource_quota_zpath (#21461) --- .../impl/ModularLoadManagerImpl.java | 2 +- .../impl/ModularLoadManagerImplTest.java | 52 +++++++++++++++++++ .../testclient/LoadSimulationController.java | 2 +- 3 files changed, 54 insertions(+), 2 deletions(-) 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 0d5dbf489e90f..022f2fcbe39f4 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 @@ -108,7 +108,7 @@ public class ModularLoadManagerImpl implements ModularLoadManager { public static final int NUM_SHORT_SAMPLES = 10; // Path to ZNode whose children contain ResourceQuota jsons. - public static final String RESOURCE_QUOTA_ZPATH = "/loadbalance/resource-quota/namespace"; + public static final String RESOURCE_QUOTA_ZPATH = "/loadbalance/resource-quota"; // Path to ZNode containing TimeAverageBrokerData jsons for each broker. public static final String TIME_AVERAGE_BROKER_ZPATH = "/loadbalance/broker-time-average"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java index 786c9027c944d..e3206ba5167dd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java @@ -76,8 +76,10 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.ServiceUnitId; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.BundlesData; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.NamespaceIsolationDataImpl; +import org.apache.pulsar.common.policies.data.ResourceQuota; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.metadata.api.MetadataCache; @@ -98,6 +100,7 @@ import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Slf4j @@ -790,6 +793,55 @@ public void testRemoveDeadBrokerTimeAverageData() throws Exception { assertEquals(data.size(), 1); } + @DataProvider(name = "isV1") + public Object[][] isV1() { + return new Object[][] {{true}, {false}}; + } + + @Test(dataProvider = "isV1") + public void testBundleDataDefaultValue(boolean isV1) throws Exception { + final String cluster = "use"; + final String tenant = "my-tenant"; + final String namespace = "my-ns"; + NamespaceName ns = isV1 ? NamespaceName.get(tenant, cluster, namespace) : NamespaceName.get(tenant, namespace); + admin1.clusters().createCluster(cluster, ClusterData.builder().serviceUrl("http://" + pulsar1.getAdvertisedAddress()).build()); + admin1.tenants().createTenant(tenant, + new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet(cluster))); + admin1.namespaces().createNamespace(ns.toString(), 16); + + // set resourceQuota to the first bundle range. + BundlesData bundlesData = admin1.namespaces().getBundles(ns.toString()); + NamespaceBundle namespaceBundle = nsFactory.getBundle(ns, + Range.range(Long.decode(bundlesData.getBoundaries().get(0)), BoundType.CLOSED, Long.decode(bundlesData.getBoundaries().get(1)), + BoundType.OPEN)); + ResourceQuota quota = new ResourceQuota(); + quota.setMsgRateIn(1024.1); + quota.setMsgRateOut(1024.2); + quota.setBandwidthIn(1024.3); + quota.setBandwidthOut(1024.4); + quota.setMemory(1024.0); + admin1.resourceQuotas().setNamespaceBundleResourceQuota(ns.toString(), namespaceBundle.getBundleRange(), quota); + + ModularLoadManagerWrapper loadManagerWrapper = (ModularLoadManagerWrapper) pulsar1.getLoadManager().get(); + ModularLoadManagerImpl lm = (ModularLoadManagerImpl) loadManagerWrapper.getLoadManager(); + + // get the bundleData of the first bundle range. + // The default value of the bundleData be the same as resourceQuota because the resourceQuota is present. + BundleData defaultBundleData = lm.getBundleDataOrDefault(namespaceBundle.toString()); + + TimeAverageMessageData shortTermData = defaultBundleData.getShortTermData(); + TimeAverageMessageData longTermData = defaultBundleData.getLongTermData(); + assertEquals(shortTermData.getMsgRateIn(), 1024.1); + assertEquals(shortTermData.getMsgRateOut(), 1024.2); + assertEquals(shortTermData.getMsgThroughputIn(), 1024.3); + assertEquals(shortTermData.getMsgThroughputOut(), 1024.4); + + assertEquals(longTermData.getMsgRateIn(), 1024.1); + assertEquals(longTermData.getMsgRateOut(), 1024.2); + assertEquals(longTermData.getMsgThroughputIn(), 1024.3); + assertEquals(longTermData.getMsgThroughputOut(), 1024.4); + } + @Test public void testRemoveNonExistBundleData() diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationController.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationController.java index bbe535df5e289..3dc7008a41a49 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationController.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/LoadSimulationController.java @@ -60,7 +60,7 @@ */ public class LoadSimulationController { private static final Logger log = LoggerFactory.getLogger(LoadSimulationController.class); - private static final String QUOTA_ROOT = "/loadbalance/resource-quota/namespace"; + private static final String QUOTA_ROOT = "/loadbalance/resource-quota"; private static final String BUNDLE_DATA_ROOT = "/loadbalance/bundle-data"; // Input streams for each client to send commands through. From 705f8b6c2974b773ba9b927087adf1e14bc59a6c Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Mon, 27 Nov 2023 13:38:07 +0800 Subject: [PATCH 161/699] [cleanup][client] Fix inconsistent API annotations of `getTopicName` (#21620) --- .../java/org/apache/pulsar/client/impl/TopicMessageImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicMessageImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicMessageImpl.java index 1b6cba2f7234d..1fec08a43f137 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicMessageImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicMessageImpl.java @@ -47,7 +47,7 @@ public class TopicMessageImpl implements Message { } /** - * Get the topic name without partition part of this message. + * Get the topic name with partition part of this message. * @return the name of the topic on which this message was published */ @Override From 028b49176784342de7292a039187e4e14034a934 Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Mon, 27 Nov 2023 18:20:09 +0800 Subject: [PATCH 162/699] [fix][sec] Upgrade rabbitmq client to address CVE-2023-46120 (#21619) Co-authored-by: Jiwe Guo --- distribution/server/pom.xml | 6 ++++++ distribution/server/src/assemble/LICENSE.bin.txt | 2 -- managed-ledger/pom.xml | 6 ++++++ pom.xml | 8 +++++++- pulsar-sql/presto-distribution/LICENSE | 2 -- 5 files changed, 19 insertions(+), 5 deletions(-) diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index b302792b4160e..257a1ac085c81 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -155,6 +155,12 @@ io.dropwizard.metrics metrics-graphite + + + amqp-client + com.rabbitmq + + diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 258ba8e9bafe8..a3aa54a9e1756 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -499,8 +499,6 @@ The Apache Software License, Version 2.0 - com.github.seancfoley-ipaddress-5.3.3.jar * RxJava - io.reactivex.rxjava3-rxjava-3.0.1.jar - * RabbitMQ Java Client - - com.rabbitmq-amqp-client-5.5.3.jar * RoaringBitmap - org.roaringbitmap-RoaringBitmap-0.9.44.jar diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index 83ba015ea4b7a..76055aeccd2d7 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -47,6 +47,12 @@ org.apache.bookkeeper.stats codahale-metrics-provider ${bookkeeper.version} + + + amqp-client + com.rabbitmq + + diff --git a/pom.xml b/pom.xml index 1c95513ef4e20..6b97ff31df31a 100644 --- a/pom.xml +++ b/pom.xml @@ -175,7 +175,7 @@ flexible messaging model and an intuitive client API. 3.11.2 4.4.20 3.4.0 - 5.5.3 + 5.18.0 1.12.262 1.10.2 2.10.10 @@ -406,6 +406,12 @@ flexible messaging model and an intuitive client API. io.dropwizard.metrics metrics-graphite ${dropwizardmetrics.version} + + + com.rabbitmq + amqp-client + + io.dropwizard.metrics diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 0f61055260dcd..e9dcabd5bb32c 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -478,8 +478,6 @@ The Apache Software License, Version 2.0 - audience-annotations-0.12.0.jar * Perfmark - perfmark-api-0.26.0.jar - * RabbitMQ Java Client - - amqp-client-5.5.3.jar * Stream Lib - stream-2.9.5.jar * High Performance Primitive Collections for Java From 15af3d0eef85c7475a10b282a8995cbfb839988b Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 27 Oct 2023 14:48:30 +0800 Subject: [PATCH 163/699] [fix] [build] rename schema_example.conf to schema_example.json (#21447) ### Motivation Since the files will be checked for `LICENSE` whose name is a suffix with `.conf`. But we can not add a LICENSE definition for the file `schema_example.conf` because it is a `JSON` file. Note: the file `schema_example.conf` is only used for the integration test `CLITest.java`. ### Modifications Rename `schema_example.conf` to `schema_example.json`. --- conf/{schema_example.conf => schema_example.json} | 0 pom.xml | 4 ++-- .../java/org/apache/pulsar/tests/integration/cli/CLITest.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) rename conf/{schema_example.conf => schema_example.json} (100%) diff --git a/conf/schema_example.conf b/conf/schema_example.json similarity index 100% rename from conf/schema_example.conf rename to conf/schema_example.json diff --git a/pom.xml b/pom.xml index 6b97ff31df31a..b6260eabc7caf 100644 --- a/pom.xml +++ b/pom.xml @@ -1683,7 +1683,7 @@ flexible messaging model and an intuitive client API. **/ByteBufCodedOutputStream.java **/ahc.properties bin/proto/* - conf/schema_example.conf + conf/schema_example.json data/** logs/** **/circe/** @@ -1808,7 +1808,7 @@ flexible messaging model and an intuitive client API. **/requirements.txt - conf/schema_example.conf + conf/schema_example.json **/templates/*.tpl diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/cli/CLITest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/cli/CLITest.java index 7e8f55429244b..f13a4dcfbdceb 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/cli/CLITest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/cli/CLITest.java @@ -402,7 +402,7 @@ public void testSchemaCLI() throws Exception { "upload", topicName, "-f", - "/pulsar/conf/schema_example.conf" + "/pulsar/conf/schema_example.json" ); result.assertNoOutput(); From 4b0713d91c458f7ecf0fab85937036fff8b0fc30 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 2 Nov 2023 13:58:31 +0800 Subject: [PATCH 164/699] [improve] [broker] Let the producer request success at the first time if the previous one is inactive (#21220) If a producer establishes a new connection when it is reconnecting, while the previous connection is now inactive, the initial request made on the new connection will fail. This failure will trigger the topic of cleaning up the inactive producers. However, upon making a second request, the producer will be able to successfully establish a connection and proceed with the operation. Make the initial request made on the new connection success. (cherry picked from commit 711b621bc2609be6fc207fedbe646d7cd14eadc2) --- .../pulsar/broker/service/AbstractTopic.java | 56 ++++--- .../pulsar/broker/service/ServerCnxTest.java | 147 ++++++++++++++++-- .../service/utils/ClientChannelHelper.java | 12 ++ 3 files changed, 183 insertions(+), 32 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 31e37d0f176d7..90ca196792b5f 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 @@ -703,15 +703,14 @@ public CompletableFuture> addProducer(Producer producer, log.warn("[{}] Attempting to add producer to a terminated topic", topic); throw new TopicTerminatedException("Topic was already terminated"); } - internalAddProducer(producer); - - USAGE_COUNT_UPDATER.incrementAndGet(this); - if (log.isDebugEnabled()) { - log.debug("[{}] [{}] Added producer -- count: {}", topic, producer.getProducerName(), - USAGE_COUNT_UPDATER.get(this)); - } - - return CompletableFuture.completedFuture(producerEpoch); + return internalAddProducer(producer).thenApply(ignore -> { + USAGE_COUNT_UPDATER.incrementAndGet(this); + if (log.isDebugEnabled()) { + log.debug("[{}] [{}] Added producer -- count: {}", topic, producer.getProducerName(), + USAGE_COUNT_UPDATER.get(this)); + } + return producerEpoch; + }); } catch (BrokerServiceException e) { return FutureUtil.failedFuture(e); } finally { @@ -957,15 +956,17 @@ protected void checkTopicFenced() throws BrokerServiceException { } } - protected void internalAddProducer(Producer producer) throws BrokerServiceException { + protected CompletableFuture internalAddProducer(Producer producer) { if (isProducersExceeded(producer)) { log.warn("[{}] Attempting to add producer to topic which reached max producers limit", topic); - throw new BrokerServiceException.ProducerBusyException("Topic reached max producers limit"); + return CompletableFuture.failedFuture( + new BrokerServiceException.ProducerBusyException("Topic reached max producers limit")); } if (isSameAddressProducersExceeded(producer)) { log.warn("[{}] Attempting to add producer to topic which reached max same address producers limit", topic); - throw new BrokerServiceException.ProducerBusyException("Topic reached max same address producers limit"); + return CompletableFuture.failedFuture( + new BrokerServiceException.ProducerBusyException("Topic reached max same address producers limit")); } if (log.isDebugEnabled()) { @@ -974,31 +975,46 @@ protected void internalAddProducer(Producer producer) throws BrokerServiceExcept Producer existProducer = producers.putIfAbsent(producer.getProducerName(), producer); if (existProducer != null) { - tryOverwriteOldProducer(existProducer, producer); + return tryOverwriteOldProducer(existProducer, producer); } else if (!producer.isRemote()) { USER_CREATED_PRODUCER_COUNTER_UPDATER.incrementAndGet(this); } + return CompletableFuture.completedFuture(null); } - private void tryOverwriteOldProducer(Producer oldProducer, Producer newProducer) - throws BrokerServiceException { + private CompletableFuture tryOverwriteOldProducer(Producer oldProducer, Producer newProducer) { if (newProducer.isSuccessorTo(oldProducer)) { oldProducer.close(false); if (!producers.replace(newProducer.getProducerName(), oldProducer, newProducer)) { // Met concurrent update, throw exception here so that client can try reconnect later. - throw new BrokerServiceException.NamingException("Producer with name '" + newProducer.getProducerName() - + "' replace concurrency error"); + return CompletableFuture.failedFuture(new BrokerServiceException.NamingException("Producer with name '" + + newProducer.getProducerName() + "' replace concurrency error")); } else { handleProducerRemoved(oldProducer); + return CompletableFuture.completedFuture(null); } } else { // If a producer with the same name tries to use a new connection, async check the old connection is // available. The producers related the connection that not available are automatically cleaned up. if (!Objects.equals(oldProducer.getCnx(), newProducer.getCnx())) { - oldProducer.getCnx().checkConnectionLiveness(); + return oldProducer.getCnx().checkConnectionLiveness().thenCompose(previousIsActive -> { + if (previousIsActive) { + return CompletableFuture.failedFuture(new BrokerServiceException.NamingException( + "Producer with name '" + newProducer.getProducerName() + + "' is already connected to topic")); + } else { + // If the connection of the previous producer is not active, the method + // "cnx().checkConnectionLiveness()" will trigger the close for it and kick off the previous + // producer. So try to add current producer again. + // The recursive call will be stopped by these two case(This prevents infinite call): + // 1. add current producer success. + // 2. once another same name producer registered. + return internalAddProducer(newProducer); + } + }); } - throw new BrokerServiceException.NamingException( - "Producer with name '" + newProducer.getProducerName() + "' is already connected to topic"); + return CompletableFuture.failedFuture(new BrokerServiceException.NamingException( + "Producer with name '" + newProducer.getProducerName() + "' is already connected to topic")); } } 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 5fd4881981365..8abd6dcff8de4 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 @@ -45,6 +45,7 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import io.netty.channel.ChannelHandler; +import io.netty.channel.DefaultChannelId; import io.netty.channel.embedded.EmbeddedChannel; import io.netty.handler.codec.LengthFieldBasedFrameDecoder; import io.vertx.core.impl.ConcurrentHashSet; @@ -62,10 +63,14 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; import java.util.function.Supplier; +import lombok.AllArgsConstructor; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.CloseCallback; @@ -973,7 +978,7 @@ public void testVerifyOriginalPrincipalWithAuthDataForwardedFromProxy() throws E } @Test - public void testHandleProducerAfterClientChannelInactive() throws Exception { + public void testDuplicateProducer() throws Exception { final String tName = successTopicName; final long producerId = 1; final MutableInt requestId = new MutableInt(1); @@ -993,33 +998,131 @@ public void testHandleProducerAfterClientChannelInactive() throws Exception { assertNotNull(topicRef); assertEquals(topicRef.getProducers().size(), 1); - // Verify the second producer using a new connection will override the producer who using a stopped channel. - channelsStoppedAnswerHealthCheck.add(channel); + // Verify the second producer will be reject due to the previous one still is active. + // Every second try once, total 10 times, all requests should fail. ClientChannel channel2 = new ClientChannel(); + BackGroundExecutor backGroundExecutor1 = startBackgroundExecutorForEmbeddedChannel(channel); + BackGroundExecutor autoResponseForHeartBeat = autoResponseForHeartBeat(channel, clientChannelHelper); + BackGroundExecutor backGroundExecutor2 = startBackgroundExecutorForEmbeddedChannel(channel2.channel); setChannelConnected(channel2.serverCnx); - Awaitility.await().untilAsserted(() -> { + + for (int i = 0; i < 10; i++) { ByteBuf cmdProducer2 = Commands.newProducer(tName, producerId, requestId.incrementAndGet(), pName, false, metadata, null, epoch.incrementAndGet(), false, ProducerAccessMode.Shared, Optional.empty(), false); channel2.channel.writeInbound(cmdProducer2); - assertTrue(getResponse(channel2.channel, channel2.clientChannelHelper) instanceof CommandProducerSuccess); + Object response2 = getResponse(channel2.channel, channel2.clientChannelHelper); + assertTrue(response2 instanceof CommandError); assertEquals(topicRef.getProducers().size(), 1); - }); + assertTrue(channel.isActive()); + Thread.sleep(500); + } // cleanup. + autoResponseForHeartBeat.close(); + backGroundExecutor1.close(); + backGroundExecutor2.close(); channel.finish(); channel2.close(); } + @Test + public void testProducerChangeSocket() throws Exception { + final String tName = successTopicName; + final long producerId = 1; + final MutableInt requestId = new MutableInt(1); + final MutableInt epoch = new MutableInt(1); + final Map metadata = Collections.emptyMap(); + final String pName = "p1"; + resetChannel(); + setChannelConnected(); + + // The producer register using the first connection. + ByteBuf cmdProducer1 = Commands.newProducer(tName, producerId, requestId.incrementAndGet(), + pName, false, metadata, null, epoch.incrementAndGet(), false, + ProducerAccessMode.Shared, Optional.empty(), false); + channel.writeInbound(cmdProducer1); + assertTrue(getResponse() instanceof CommandProducerSuccess); + PersistentTopic topicRef = (PersistentTopic) brokerService.getTopicReference(tName).get(); + assertNotNull(topicRef); + assertEquals(topicRef.getProducers().size(), 1); + + // Verify the second producer using a new connection will override the producer who using a stopped channel. + channelsStoppedAnswerHealthCheck.add(channel); + ClientChannel channel2 = new ClientChannel(); + BackGroundExecutor backGroundExecutor1 = startBackgroundExecutorForEmbeddedChannel(channel); + BackGroundExecutor backGroundExecutor2 = startBackgroundExecutorForEmbeddedChannel(channel2.channel); + setChannelConnected(channel2.serverCnx); + + ByteBuf cmdProducer2 = Commands.newProducer(tName, producerId, requestId.incrementAndGet(), + pName, false, metadata, null, epoch.incrementAndGet(), false, + ProducerAccessMode.Shared, Optional.empty(), false); + channel2.channel.writeInbound(cmdProducer2); + Object response2 = getResponse(channel2.channel, channel2.clientChannelHelper); + assertTrue(response2 instanceof CommandProducerSuccess); + assertEquals(topicRef.getProducers().size(), 1); + + // cleanup. + channelsStoppedAnswerHealthCheck.clear(); + backGroundExecutor1.close(); + backGroundExecutor2.close(); + channel.finish(); + channel2.close(); + } + + /** + * When a channel typed "EmbeddedChannel", once we call channel.execute(runnable), there is no background thread + * to run it. + * So starting a background thread to trigger the tasks in the queue. + */ + private BackGroundExecutor startBackgroundExecutorForEmbeddedChannel(final EmbeddedChannel channel) { + ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); + ScheduledFuture scheduledFuture = executor.scheduleWithFixedDelay(() -> { + channel.runPendingTasks(); + }, 100, 100, TimeUnit.MILLISECONDS); + return new BackGroundExecutor(executor, scheduledFuture); + } + + /** + * Auto answer `Pong` for the `Cmd-Ping`. + * Node: This will result in additional threads pop Command from the Command queue, so do not call this + * method if the channel needs to accept other Command. + */ + private BackGroundExecutor autoResponseForHeartBeat(EmbeddedChannel channel, + ClientChannelHelper clientChannelHelper) { + ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); + ScheduledFuture scheduledFuture = executor.scheduleWithFixedDelay(() -> { + tryPeekResponse(channel, clientChannelHelper); + }, 100, 100, TimeUnit.MILLISECONDS); + return new BackGroundExecutor(executor, scheduledFuture); + } + + @AllArgsConstructor + private static class BackGroundExecutor implements Closeable { + + private ScheduledExecutorService executor; + + private ScheduledFuture scheduledFuture; + + @Override + public void close() throws IOException { + if (scheduledFuture != null) { + scheduledFuture.cancel(true); + } + executor.shutdown(); + } + } + private class ClientChannel implements Closeable { private ClientChannelHelper clientChannelHelper = new ClientChannelHelper(); private ServerCnx serverCnx = new ServerCnx(pulsar); - private EmbeddedChannel channel = new EmbeddedChannel(new LengthFieldBasedFrameDecoder( - 5 * 1024 * 1024, - 0, - 4, - 0, - 4), + private EmbeddedChannel channel = new EmbeddedChannel(DefaultChannelId.newInstance(), + new LengthFieldBasedFrameDecoder( + 5 * 1024 * 1024, + 0, + 4, + 0, + 4), serverCnx); public ClientChannel() { serverCnx.setAuthRole(""); @@ -2694,6 +2797,26 @@ protected Object getResponse(EmbeddedChannel channel, ClientChannelHelper client throw new IOException("Failed to get response from socket within 10s"); } + protected Object tryPeekResponse(EmbeddedChannel channel, ClientChannelHelper clientChannelHelper) { + while (true) { + if (channel.outboundMessages().isEmpty()) { + return null; + } else { + Object outObject = channel.outboundMessages().peek(); + Object cmd = clientChannelHelper.getCommand(outObject); + if (cmd instanceof CommandPing) { + if (channelsStoppedAnswerHealthCheck.contains(channel)) { + continue; + } + channel.writeInbound(Commands.newPong()); + channel.outboundMessages().remove(); + continue; + } + return cmd; + } + } + } + private void setupMLAsyncCallbackMocks() { ledgerMock = mock(ManagedLedger.class); cursorMock = mock(ManagedCursor.class); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/utils/ClientChannelHelper.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/utils/ClientChannelHelper.java index bf0dd3aa9c1c5..c8fce32efc5f0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/utils/ClientChannelHelper.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/utils/ClientChannelHelper.java @@ -27,6 +27,8 @@ import org.apache.pulsar.common.api.proto.CommandEndTxnResponse; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespaceResponse; import org.apache.pulsar.common.api.proto.CommandPartitionedTopicMetadataResponse; +import org.apache.pulsar.common.api.proto.CommandPing; +import org.apache.pulsar.common.api.proto.CommandPong; import org.apache.pulsar.common.api.proto.CommandWatchTopicListSuccess; import org.apache.pulsar.common.protocol.PulsarDecoder; import org.apache.pulsar.common.api.proto.CommandAck; @@ -207,6 +209,16 @@ protected void handleEndTxnOnSubscriptionResponse( CommandEndTxnOnSubscriptionResponse commandEndTxnOnSubscriptionResponse) { queue.offer(new CommandEndTxnOnSubscriptionResponse().copyFrom(commandEndTxnOnSubscriptionResponse)); } + + @Override + protected void handlePing(CommandPing ping) { + queue.offer(new CommandPing().copyFrom(ping)); + } + + @Override + protected void handlePong(CommandPong pong) { + return; + } }; } From 23e64d90ceef35a8f57cf5f059e11f9d777dff78 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 29 Nov 2023 01:55:43 +0800 Subject: [PATCH 165/699] [fix] [log] fix the vague response if topic not found (#20932) ### Motivation When I did this test below and got the error "Topic not found". ```java pulsarAdmin.topics().createNonPartitionedTopic("persistent://public/default/tp_1"); Consumer consumer = null; Consumer consumer = pulsarClient.newConsumer() .topic("persistent://public/default/tp_1") .subscriptionName("s1") .enableRetry(true) .subscribe(); ``` I do create the topic `persistent://public/default/tp_1` first but got a response "Topic not found", it is confusing. The root cause is the retry letter topic `persistent://public/default/tp_1-sub1-RETRY` was not created. ### Modifications clear the vague response if the topic is not founded. (cherry picked from commit 1a024bc6aa0999669ef8b67bdbe0dc80eeff9f8f) --- .../broker/admin/impl/TransactionsBase.java | 6 +- .../broker/admin/v1/NonPersistentTopics.java | 5 +- .../broker/admin/v2/NonPersistentTopics.java | 3 +- .../pulsar/broker/lookup/TopicLookupBase.java | 5 +- .../pulsar/broker/service/ServerCnx.java | 10 +-- .../admin/AdminApiGetLastMessageIdTest.java | 2 +- .../admin/v3/AdminApiTransactionTest.java | 20 ++--- ...erConsumerDisallowAutoCreateTopicTest.java | 79 +++++++++++++++++++ 8 files changed, 107 insertions(+), 23 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerDisallowAutoCreateTopicTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TransactionsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TransactionsBase.java index 3921334cff30a..470cdc3e74ba1 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TransactionsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/TransactionsBase.java @@ -501,11 +501,13 @@ protected CompletableFuture getExistingPersistentTopicAsync(boo CompletableFuture> topicFuture = pulsar().getBrokerService() .getTopics().get(topicName.toString()); if (topicFuture == null) { - return FutureUtil.failedFuture(new RestException(NOT_FOUND, "Topic not found")); + return FutureUtil.failedFuture(new RestException(NOT_FOUND, + String.format("Topic not found %s", topicName.toString()))); } return topicFuture.thenCompose(optionalTopic -> { if (!optionalTopic.isPresent()) { - return FutureUtil.failedFuture(new RestException(NOT_FOUND, "Topic not found")); + return FutureUtil.failedFuture(new RestException(NOT_FOUND, + String.format("Topic not found %s", topicName.toString()))); } return CompletableFuture.completedFuture((PersistentTopic) optionalTopic.get()); }); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java index 0d857f2211f41..1c1dd74719641 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v1/NonPersistentTopics.java @@ -284,11 +284,12 @@ public List getListFromBundle(@PathParam("property") String property, @P } } - private Topic getTopicReference(TopicName topicName) { + private Topic getTopicReference(final TopicName topicName) { try { return pulsar().getBrokerService().getTopicIfExists(topicName.toString()) .get(config().getMetadataStoreOperationTimeoutSeconds(), TimeUnit.SECONDS) - .orElseThrow(() -> new RestException(Status.NOT_FOUND, "Topic not found")); + .orElseThrow(() -> new RestException(Status.NOT_FOUND, + String.format("Topic not found %s", topicName.toString()))); } catch (ExecutionException e) { throw new RuntimeException(e.getCause()); } catch (InterruptedException | TimeoutException e) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java index c360eeabb5838..386b9749ef959 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java @@ -233,7 +233,8 @@ public void getPartitionedStats( getPartitionedTopicMetadataAsync(topicName, authoritative, false).thenAccept(partitionMetadata -> { if (partitionMetadata.partitions == 0) { - asyncResponse.resume(new RestException(Status.NOT_FOUND, "Partitioned Topic not found")); + asyncResponse.resume(new RestException(Status.NOT_FOUND, + String.format("Partitioned topic not found %s", topicName.toString()))); return; } NonPersistentPartitionedTopicStatsImpl stats = 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 bd70201cba55d..9c9c522dd98ab 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 @@ -56,7 +56,7 @@ public class TopicLookupBase extends PulsarWebResource { private static final String LOOKUP_PATH_V1 = "/lookup/v2/destination/"; private static final String LOOKUP_PATH_V2 = "/lookup/v2/topic/"; - protected CompletableFuture internalLookupTopicAsync(TopicName topicName, boolean authoritative, + protected CompletableFuture internalLookupTopicAsync(final TopicName topicName, boolean authoritative, String listenerName) { if (!pulsar().getBrokerService().getLookupRequestSemaphore().tryAcquire()) { log.warn("No broker was found available for topic {}", topicName); @@ -79,7 +79,8 @@ protected CompletableFuture internalLookupTopicAsync(TopicName topic }) .thenCompose(exist -> { if (!exist) { - throw new RestException(Response.Status.NOT_FOUND, "Topic not found."); + throw new RestException(Response.Status.NOT_FOUND, + String.format("Topic not found %s", topicName.toString())); } CompletableFuture> lookupFuture = pulsar().getNamespaceService() .getBrokerServiceUrlAsync(topicName, 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 b819a99309365..4e61a3228fe59 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 @@ -1371,7 +1371,7 @@ protected void handleProducer(final CommandProducer cmdProducer) { cmdProducer.hasInitialSubscriptionName() ? cmdProducer.getInitialSubscriptionName() : null; final boolean supportsPartialProducer = supportsPartialProducer(); - TopicName topicName = validateTopicName(cmdProducer.getTopic(), requestId, cmdProducer); + final TopicName topicName = validateTopicName(cmdProducer.getTopic(), requestId, cmdProducer); if (topicName == null) { return; } @@ -1564,7 +1564,7 @@ protected void handleProducer(final CommandProducer cmdProducer) { // Do not print stack traces for expected exceptions if (cause instanceof NoSuchElementException) { - cause = new TopicNotFoundException("Topic Not Found."); + cause = new TopicNotFoundException(String.format("Topic not found %s", topicName.toString())); log.warn("[{}] Failed to load topic {}, producerId={}: Topic not found", remoteAddress, topicName, producerId); } else if (!Exceptions.areExceptionsPresentInChain(cause, @@ -2377,7 +2377,7 @@ remoteAddress, new String(commandGetSchema.getSchemaVersion()), schemaService.getSchema(schemaName, schemaVersion).thenAccept(schemaAndMetadata -> { if (schemaAndMetadata == null) { commandSender.sendGetSchemaErrorResponse(requestId, ServerError.TopicNotFound, - "Topic not found or no-schema"); + String.format("Topic not found or no-schema %s", commandGetSchema.getTopic())); } else { commandSender.sendGetSchemaResponse(requestId, SchemaInfoUtil.newSchemaInfo(schemaName, schemaAndMetadata.schema), schemaAndMetadata.version); @@ -2395,7 +2395,7 @@ protected void handleGetOrCreateSchema(CommandGetOrCreateSchema commandGetOrCrea log.debug("Received CommandGetOrCreateSchema call from {}", remoteAddress); } long requestId = commandGetOrCreateSchema.getRequestId(); - String topicName = commandGetOrCreateSchema.getTopic(); + final String topicName = commandGetOrCreateSchema.getTopic(); SchemaData schemaData = getSchema(commandGetOrCreateSchema.getSchema()); SchemaData schema = schemaData.getType() == SchemaType.NONE ? null : schemaData; service.getTopicIfExists(topicName).thenAccept(topicOpt -> { @@ -2415,7 +2415,7 @@ protected void handleGetOrCreateSchema(CommandGetOrCreateSchema commandGetOrCrea }); } else { commandSender.sendGetOrCreateSchemaErrorResponse(requestId, ServerError.TopicNotFound, - "Topic not found"); + String.format("Topic not found %s", topicName)); } }).exceptionally(ex -> { ServerError errorCode = BrokerServiceException.getClientErrorCode(ex); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiGetLastMessageIdTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiGetLastMessageIdTest.java index 3d2a6b934f847..27d72f98c2c49 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiGetLastMessageIdTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiGetLastMessageIdTest.java @@ -168,7 +168,7 @@ public Map, Collection>> register(Object callback, Object... c testNamespace, "my-topic", true); } catch (Exception e) { //System.out.println(e.getMessage()); - Assert.assertEquals("Topic not found", e.getMessage()); + Assert.assertTrue(e.getMessage().contains("Topic not found")); } String key = "legendtkl"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java index 1e5f4679492ad..049fd0f5f4400 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java @@ -163,7 +163,7 @@ public void testGetTransactionInBufferStats() throws Exception { } catch (ExecutionException ex) { assertTrue(ex.getCause() instanceof PulsarAdminException.NotFoundException); PulsarAdminException.NotFoundException cause = (PulsarAdminException.NotFoundException)ex.getCause(); - assertEquals(cause.getMessage(), "Topic not found"); + assertTrue(cause.getMessage().contains("Topic not found")); } try { pulsar.getBrokerService().getTopic(topic, false); @@ -173,7 +173,7 @@ public void testGetTransactionInBufferStats() throws Exception { } catch (ExecutionException ex) { assertTrue(ex.getCause() instanceof PulsarAdminException.NotFoundException); PulsarAdminException.NotFoundException cause = (PulsarAdminException.NotFoundException)ex.getCause(); - assertEquals(cause.getMessage(), "Topic not found"); + assertTrue(cause.getMessage().contains("Topic not found")); } admin.topics().createNonPartitionedTopic(topic); Producer producer = pulsarClient.newProducer(Schema.BYTES).topic(topic).sendTimeout(0, TimeUnit.SECONDS).create(); @@ -208,7 +208,7 @@ public void testGetTransactionInPendingAckStats() throws Exception { } catch (ExecutionException ex) { assertTrue(ex.getCause() instanceof PulsarAdminException.NotFoundException); PulsarAdminException.NotFoundException cause = (PulsarAdminException.NotFoundException)ex.getCause(); - assertEquals(cause.getMessage(), "Topic not found"); + assertTrue(cause.getMessage().contains("Topic not found")); } try { pulsar.getBrokerService().getTopic(topic, false); @@ -219,7 +219,7 @@ public void testGetTransactionInPendingAckStats() throws Exception { } catch (ExecutionException ex) { assertTrue(ex.getCause() instanceof PulsarAdminException.NotFoundException); PulsarAdminException.NotFoundException cause = (PulsarAdminException.NotFoundException)ex.getCause(); - assertEquals(cause.getMessage(), "Topic not found"); + assertTrue(cause.getMessage().contains("Topic not found")); } admin.topics().createNonPartitionedTopic(topic); Producer producer = pulsarClient.newProducer(Schema.BYTES).topic(topic).create(); @@ -334,7 +334,7 @@ public void testGetTransactionBufferStats() throws Exception { } catch (ExecutionException ex) { assertTrue(ex.getCause() instanceof PulsarAdminException.NotFoundException); PulsarAdminException.NotFoundException cause = (PulsarAdminException.NotFoundException)ex.getCause(); - assertEquals(cause.getMessage(), "Topic not found"); + assertTrue(cause.getMessage().contains("Topic not found")); } try { pulsar.getBrokerService().getTopic(topic, false); @@ -344,7 +344,7 @@ public void testGetTransactionBufferStats() throws Exception { } catch (ExecutionException ex) { assertTrue(ex.getCause() instanceof PulsarAdminException.NotFoundException); PulsarAdminException.NotFoundException cause = (PulsarAdminException.NotFoundException)ex.getCause(); - assertEquals(cause.getMessage(), "Topic not found"); + assertTrue(cause.getMessage().contains("Topic not found")); } admin.topics().createNonPartitionedTopic(topic); Producer producer = pulsarClient.newProducer(Schema.BYTES) @@ -392,7 +392,7 @@ public void testGetPendingAckStats(String ackType) throws Exception { } catch (ExecutionException ex) { assertTrue(ex.getCause() instanceof PulsarAdminException.NotFoundException); PulsarAdminException.NotFoundException cause = (PulsarAdminException.NotFoundException)ex.getCause(); - assertEquals(cause.getMessage(), "Topic not found"); + assertTrue(cause.getMessage().contains("Topic not found")); } try { pulsar.getBrokerService().getTopic(topic, false); @@ -402,7 +402,7 @@ public void testGetPendingAckStats(String ackType) throws Exception { } catch (ExecutionException ex) { assertTrue(ex.getCause() instanceof PulsarAdminException.NotFoundException); PulsarAdminException.NotFoundException cause = (PulsarAdminException.NotFoundException)ex.getCause(); - assertEquals(cause.getMessage(), "Topic not found"); + assertTrue(cause.getMessage().contains("Topic not found")); } admin.topics().createNonPartitionedTopic(topic); @@ -541,7 +541,7 @@ public void testGetPendingAckInternalStats() throws Exception { } catch (ExecutionException ex) { assertTrue(ex.getCause() instanceof PulsarAdminException.NotFoundException); PulsarAdminException.NotFoundException cause = (PulsarAdminException.NotFoundException)ex.getCause(); - assertEquals(cause.getMessage(), "Topic not found"); + assertTrue(cause.getMessage().contains("Topic not found")); } try { pulsar.getBrokerService().getTopic(topic, false); @@ -551,7 +551,7 @@ public void testGetPendingAckInternalStats() throws Exception { } catch (ExecutionException ex) { assertTrue(ex.getCause() instanceof PulsarAdminException.NotFoundException); PulsarAdminException.NotFoundException cause = (PulsarAdminException.NotFoundException)ex.getCause(); - assertEquals(cause.getMessage(), "Topic not found"); + assertTrue(cause.getMessage().contains("Topic not found")); } admin.topics().createNonPartitionedTopic(topic); Producer producer = pulsarClient.newProducer(Schema.BYTES).topic(topic).create(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerDisallowAutoCreateTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerDisallowAutoCreateTopicTest.java new file mode 100644 index 0000000000000..728e556f0224a --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerDisallowAutoCreateTopicTest.java @@ -0,0 +1,79 @@ +/* + * 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.apache.pulsar.client.util.RetryMessageUtil.RETRY_GROUP_TOPIC_SUFFIX; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-api") +public class SimpleProducerConsumerDisallowAutoCreateTopicTest 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(); + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + conf.setAllowAutoTopicCreation(false); + } + + @Test + public void testClearErrorIfRetryTopicNotExists() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp_"); + final String subName = "sub"; + final String retryTopicName = topicName + "-" + subName + RETRY_GROUP_TOPIC_SUFFIX; + admin.topics().createNonPartitionedTopic(topicName); + Consumer consumer = null; + try { + consumer = pulsarClient.newConsumer() + .topic(topicName) + .subscriptionName(subName) + .enableRetry(true) + .subscribe(); + fail(""); + } catch (Exception ex) { + log.info("got an expected error", ex); + assertTrue(ex.getMessage().contains("Not found:")); + assertTrue(ex.getMessage().contains(retryTopicName)); + } finally { + // cleanup. + if (consumer != null) { + consumer.close(); + } + admin.topics().delete(topicName); + } + } +} From 278a9054f5353a6e41fb1256543b4b930a1ab83b Mon Sep 17 00:00:00 2001 From: houxiaoyu Date: Thu, 30 Nov 2023 23:08:00 +0800 Subject: [PATCH 166/699] Release 3.1.2 --- 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 +- managed-ledger/pom.xml | 2 +- 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-client-1x-base/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-1x/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-2x-shaded/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/aws/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/hdfs2/pom.xml | 2 +- pulsar-io/hdfs3/pom.xml | 2 +- pulsar-io/http/pom.xml | 2 +- pulsar-io/jdbc/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/twitter/pom.xml | 2 +- pulsar-metadata/pom.xml | 2 +- pulsar-proxy/pom.xml | 2 +- pulsar-sql/pom.xml | 2 +- pulsar-sql/presto-distribution/pom.xml | 2 +- pulsar-sql/presto-pulsar-plugin/pom.xml | 2 +- pulsar-sql/presto-pulsar/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 +- 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 +- 117 files changed, 119 insertions(+), 119 deletions(-) diff --git a/bouncy-castle/bc/pom.xml b/bouncy-castle/bc/pom.xml index 7e57a206bf3d9..6058c28076814 100644 --- a/bouncy-castle/bc/pom.xml +++ b/bouncy-castle/bc/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.2-SNAPSHOT + 3.1.2 .. diff --git a/bouncy-castle/bcfips-include-test/pom.xml b/bouncy-castle/bcfips-include-test/pom.xml index 27f8e1dfef9e9..92641bdd2c49a 100644 --- a/bouncy-castle/bcfips-include-test/pom.xml +++ b/bouncy-castle/bcfips-include-test/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.2-SNAPSHOT + 3.1.2 .. diff --git a/bouncy-castle/bcfips/pom.xml b/bouncy-castle/bcfips/pom.xml index c526abb1d650d..2280474524cee 100644 --- a/bouncy-castle/bcfips/pom.xml +++ b/bouncy-castle/bcfips/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.2-SNAPSHOT + 3.1.2 .. diff --git a/bouncy-castle/pom.xml b/bouncy-castle/pom.xml index bd603050a41b6..dfb955a354459 100644 --- a/bouncy-castle/pom.xml +++ b/bouncy-castle/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.2-SNAPSHOT + 3.1.2 .. diff --git a/buildtools/pom.xml b/buildtools/pom.xml index b5a78470c0952..393a69fd1edb5 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -31,12 +31,12 @@ com.datastax.oss buildtools - 3.1.2-SNAPSHOT + 3.1.2 jar Pulsar Build Tools - 2023-09-27T08:29:08Z + 2023-11-30T15:05:59Z 1.8 1.8 3.1.0 diff --git a/distribution/io/pom.xml b/distribution/io/pom.xml index 12d8a5be1bf9d..710373712427c 100644 --- a/distribution/io/pom.xml +++ b/distribution/io/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.2-SNAPSHOT + 3.1.2 .. diff --git a/distribution/offloaders/pom.xml b/distribution/offloaders/pom.xml index 21e8fce1acf5d..b6ea86580489e 100644 --- a/distribution/offloaders/pom.xml +++ b/distribution/offloaders/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.2-SNAPSHOT + 3.1.2 .. diff --git a/distribution/pom.xml b/distribution/pom.xml index 5dadef0b48986..01354a0eaac69 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.2-SNAPSHOT + 3.1.2 .. diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index 257a1ac085c81..6177782aad3bd 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.2-SNAPSHOT + 3.1.2 .. diff --git a/distribution/shell/pom.xml b/distribution/shell/pom.xml index 27d1ef5fa00ec..c4e6118f8bdd2 100644 --- a/distribution/shell/pom.xml +++ b/distribution/shell/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.2-SNAPSHOT + 3.1.2 .. diff --git a/docker/pom.xml b/docker/pom.xml index 70291bfa9cdb6..477cfa3f221d6 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.2-SNAPSHOT + 3.1.2 docker-images Apache Pulsar :: Docker Images diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index 64f7d8953ca21..977253c228028 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.2-SNAPSHOT + 3.1.2 4.0.0 pulsar-all-docker-image diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index aacf4a09cd695..9c55ff898d913 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.2-SNAPSHOT + 3.1.2 4.0.0 pulsar-docker-image diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml index 980f204b1c275..9d3f6961fe550 100644 --- a/jclouds-shaded/pom.xml +++ b/jclouds-shaded/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.2-SNAPSHOT + 3.1.2 .. diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index 76055aeccd2d7..fca2d04548f43 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.2-SNAPSHOT + 3.1.2 .. diff --git a/pom.xml b/pom.xml index b6260eabc7caf..494ea2278a695 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ com.datastax.oss pulsar - 3.1.2-SNAPSHOT + 3.1.2 Pulsar Pulsar is a distributed pub-sub messaging platform with a very @@ -92,7 +92,7 @@ flexible messaging model and an intuitive client API. UTF-8 UTF-8 - 2023-09-27T08:29:08Z + 2023-11-30T15:05:59Z true diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/TestZKServer.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/TestZKServer.java index 726f5ae312d19..33034ddb3fe0f 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/TestZKServer.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/TestZKServer.java @@ -19,39 +19,33 @@ package org.apache.pulsar.metadata; import static org.testng.Assert.assertTrue; - import java.io.BufferedReader; import java.io.File; import java.io.IOException; import java.io.InputStreamReader; import java.io.OutputStream; -import java.net.InetSocketAddress; +import java.lang.reflect.Field; import java.net.Socket; - -import java.nio.charset.StandardCharsets; - +import java.util.Properties; +import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; - import org.apache.commons.io.FileUtils; -import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.server.ContainerManager; -import org.apache.zookeeper.server.NIOServerCnxnFactory; -import org.apache.zookeeper.server.Request; -import org.apache.zookeeper.server.RequestProcessor; import org.apache.zookeeper.server.ServerCnxnFactory; -import org.apache.zookeeper.server.SessionTracker; import org.apache.zookeeper.server.ZooKeeperServer; +import org.apache.zookeeper.server.ZooKeeperServerMain; +import org.apache.zookeeper.server.embedded.ExitHandler; +import org.apache.zookeeper.server.embedded.ZooKeeperServerEmbedded; import org.assertj.core.util.Files; @Slf4j public class TestZKServer implements AutoCloseable { + public static final int TICK_TIME = 1000; - protected ZooKeeperServer zks; - private final File zkDataDir; - private ServerCnxnFactory serverFactory; - private ContainerManager containerManager; - private int zkPort = 0; + private final File zkDataDir; + private int zkPort; // initially this is zero + private ZooKeeperServerEmbedded zooKeeperServerEmbedded; public TestZKServer() throws Exception { this.zkDataDir = Files.newTemporaryFolder(); @@ -64,86 +58,86 @@ public TestZKServer() throws Exception { } public void start() throws Exception { - this.zks = new ZooKeeperServer(zkDataDir, zkDataDir, TICK_TIME); - this.zks.setMaxSessionTimeout(300_000); - this.serverFactory = new NIOServerCnxnFactory(); - this.serverFactory.configure(new InetSocketAddress(zkPort), 1000); - this.serverFactory.startup(zks, true); - - this.zkPort = serverFactory.getLocalPort(); - log.info("Started test ZK server on port {}", zkPort); + final Properties configZookeeper = new Properties(); + configZookeeper.put("clientPort", zkPort + ""); + configZookeeper.put("host", "127.0.0.1"); + configZookeeper.put("ticktime", TICK_TIME + ""); + zooKeeperServerEmbedded = ZooKeeperServerEmbedded + .builder() + .baseDir(zkDataDir.toPath()) + .configuration(configZookeeper) + .exitHandler(ExitHandler.LOG_ONLY) + .build(); + + zooKeeperServerEmbedded.start(60_000); + log.info("Started test ZK server on at {}", zooKeeperServerEmbedded.getConnectionString()); + + ZooKeeperServerMain zooKeeperServerMain = getZooKeeperServerMain(zooKeeperServerEmbedded); + ServerCnxnFactory serverCnxnFactory = getServerCnxnFactory(zooKeeperServerMain); + // save the port, in order to allow restarting on the same port + zkPort = serverCnxnFactory.getLocalPort(); boolean zkServerReady = waitForServerUp(this.getConnectionString(), 30_000); assertTrue(zkServerReady); + } - this.containerManager = new ContainerManager(zks.getZKDatabase(), new RequestProcessor() { - @Override - public void processRequest(Request request) throws RequestProcessorException { - String path = StandardCharsets.UTF_8.decode(request.request).toString(); - try { - zks.getZKDatabase().getDataTree().deleteNode(path, -1); - } catch (KeeperException.NoNodeException e) { - // Ok - } - } + @SneakyThrows + private static ZooKeeperServerMain getZooKeeperServerMain(ZooKeeperServerEmbedded zooKeeperServerEmbedded) { + ZooKeeperServerMain zooKeeperServerMain = readField(zooKeeperServerEmbedded.getClass(), + "mainsingle", zooKeeperServerEmbedded); + return zooKeeperServerMain; + } - @Override - public void shutdown() { + @SneakyThrows + private static ContainerManager getContainerManager(ZooKeeperServerMain zooKeeperServerMain) { + ContainerManager containerManager = readField(ZooKeeperServerMain.class, "containerManager", zooKeeperServerMain); + return containerManager; + } - } - }, 10, 10000, 0L); + @SneakyThrows + private static ZooKeeperServer getZooKeeperServer(ZooKeeperServerMain zooKeeperServerMain) { + ServerCnxnFactory serverCnxnFactory = getServerCnxnFactory(zooKeeperServerMain); + ZooKeeperServer zkServer = readField(ServerCnxnFactory.class, "zkServer", serverCnxnFactory); + return zkServer; + } + + @SneakyThrows + private static T readField(Class clazz, String field, Object object) { + Field declaredField = clazz.getDeclaredField(field); + boolean accessible = declaredField.isAccessible(); + if (!accessible) { + declaredField.setAccessible(true); + } + try { + return (T) declaredField.get(object); + } finally { + declaredField.setAccessible(accessible); + } + } + + private static ServerCnxnFactory getServerCnxnFactory(ZooKeeperServerMain zooKeeperServerMain) throws Exception { + ServerCnxnFactory serverCnxnFactory = readField(ZooKeeperServerMain.class, "cnxnFactory", zooKeeperServerMain); + return serverCnxnFactory; } public void checkContainers() throws Exception { // Make sure the container nodes are actually deleted Thread.sleep(1000); + ContainerManager containerManager = getContainerManager(getZooKeeperServerMain(zooKeeperServerEmbedded)); containerManager.checkContainers(); } public void stop() throws Exception { - if (containerManager != null) { - containerManager.stop(); - containerManager = null; - } - - if (serverFactory != null) { - serverFactory.shutdown(); - serverFactory = null; - } - - if (zks != null) { - SessionTracker sessionTracker = zks.getSessionTracker(); - zks.shutdown(); - zks.getZKDatabase().close(); - if (sessionTracker instanceof Thread) { - Thread sessionTrackerThread = (Thread) sessionTracker; - sessionTrackerThread.interrupt(); - sessionTrackerThread.join(); - } - zks = null; + if (zooKeeperServerEmbedded != null) { + zooKeeperServerEmbedded.close(); } - log.info("Stopped test ZK server"); } public void expireSession(long sessionId) { - zks.expire(new SessionTracker.Session() { - @Override - public long getSessionId() { - return sessionId; - } - - @Override - public int getTimeout() { - return 10_000; - } - - @Override - public boolean isClosing() { - return false; - } - }); + getZooKeeperServer(getZooKeeperServerMain(zooKeeperServerEmbedded)) + .expire(sessionId); } @Override @@ -152,12 +146,9 @@ public void close() throws Exception { FileUtils.deleteDirectory(zkDataDir); } - public int getPort() { - return zkPort; - } - + @SneakyThrows public String getConnectionString() { - return "127.0.0.1:" + getPort(); + return zooKeeperServerEmbedded.getConnectionString(); } public static boolean waitForServerUp(String hp, long timeout) { diff --git a/pulsar-package-management/bookkeeper-storage/pom.xml b/pulsar-package-management/bookkeeper-storage/pom.xml index 621b13b2a5c5a..b4c3d6376cf19 100644 --- a/pulsar-package-management/bookkeeper-storage/pom.xml +++ b/pulsar-package-management/bookkeeper-storage/pom.xml @@ -71,6 +71,12 @@ + + org.hamcrest + hamcrest + test + + io.dropwizard.metrics diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index e9dcabd5bb32c..49f4b38a1361c 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -472,8 +472,8 @@ The Apache Software License, Version 2.0 - memory-0.8.3.jar - sketches-core-0.8.3.jar * Apache Zookeeper - - zookeeper-3.8.3.jar - - zookeeper-jute-3.8.3.jar + - zookeeper-3.9.1.jar + - zookeeper-jute-3.9.1.jar * Apache Yetus Audience Annotations - audience-annotations-0.12.0.jar * Perfmark From 75fa40db248472e5dc30d4c25a7664fb5d0b1ffb Mon Sep 17 00:00:00 2001 From: Yong Zhang Date: Fri, 8 Dec 2023 04:20:55 +0800 Subject: [PATCH 175/699] [fix][offload] Don't cleanup data when offload met MetaStore exception (#21686) --- .../mledger/impl/ManagedLedgerImpl.java | 3 +- .../mledger/impl/ManagedLedgerTest.java | 55 +++++++++++++++++++ 2 files changed, 57 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 ed148f66fc714..146693f661071 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 @@ -3208,7 +3208,7 @@ public void asyncOffloadPrefix(Position pos, OffloadCallback callback, Object ct } } - private void offloadLoop(CompletableFuture promise, Queue ledgersToOffload, + void offloadLoop(CompletableFuture promise, Queue ledgersToOffload, PositionImpl firstUnoffloaded, Optional firstError) { State currentState = getState(); if (currentState == State.Closed) { @@ -3256,6 +3256,7 @@ private void offloadLoop(CompletableFuture promise, Queue key.equals(FaultInjectionMetadataStore.OperationType.PUT) && + metadataPutCallCount.incrementAndGet() == 2); + + // prepare the arguments for the offloadLoop method + CompletableFuture future = new CompletableFuture<>(); + Queue ledgersToOffload = new LinkedList<>(); + LedgerInfo ledgerInfo = LedgerInfo.getDefaultInstance().toBuilder().setLedgerId(1).setEntries(10).build(); + ledgersToOffload.add(ledgerInfo); + PositionImpl firstUnoffloaded = new PositionImpl(1, 0); + Optional firstError = Optional.empty(); + + // mock the read handle to make the offload successful + CompletableFuture readHandle = new CompletableFuture<>(); + readHandle.complete(mock(ReadHandle.class)); + when(ml.getLedgerHandle(eq(ledgerInfo.getLedgerId()))).thenReturn(readHandle); + when(ledgerOffloader.offload(any(), any(), anyMap())).thenReturn(CompletableFuture.completedFuture(null)); + + ml.ledgers.put(ledgerInfo.getLedgerId(), ledgerInfo); + + // do the offload + ml.offloadLoop(future, ledgersToOffload, firstUnoffloaded, firstError); + + // waiting for the offload complete + try { + future.join(); + fail("The offload should fail"); + } catch (Exception e) { + // the offload should fail + assertTrue(e.getCause().getMessage().contains("mock completion error")); + } + + // the ledger deletion shouldn't happen + verify(ledgerOffloader, times(0)) + .deleteOffloaded(eq(ledgerInfo.getLedgerId()), any(), anyMap()); + } } From 627de283e7137e1ab95feb97f458e2cf20aeab04 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Fri, 8 Dec 2023 04:23:30 +0800 Subject: [PATCH 176/699] [fix][broker] Fix typo in the config key (#21690) --- conf/broker.conf | 2 +- conf/standalone.conf | 2 +- .../apache/pulsar/broker/ServiceConfiguration.java | 2 +- .../apache/pulsar/compaction/TwoPhaseCompactor.java | 12 ++++++------ .../org/apache/pulsar/compaction/CompactionTest.java | 5 ++--- 5 files changed, 11 insertions(+), 12 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index 811862aaa5034..73a42171fc1d5 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -539,7 +539,7 @@ brokerServiceCompactionThresholdInBytes=0 brokerServiceCompactionPhaseOneLoopTimeInSeconds=30 # Whether retain null-key message during topic compaction -topicCompactionRemainNullKey=true +topicCompactionRetainNullKey=true # Whether to enable the delayed delivery for messages. # If disabled, messages will be immediately delivered and there will diff --git a/conf/standalone.conf b/conf/standalone.conf index b730bbc1290e2..0b486bdaf0481 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -1280,4 +1280,4 @@ brokerInterceptors= disableBrokerInterceptors=true # Whether retain null-key message during topic compaction -topicCompactionRemainNullKey=true +topicCompactionRetainNullKey=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 f3255a83e29f3..307943ea37523 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 @@ -2781,7 +2781,7 @@ The delayed message index time step(in seconds) in per bucket snapshot segment, category = CATEGORY_SERVER, doc = "Whether retain null-key message during topic compaction." ) - private boolean topicCompactionRemainNullKey = true; + private boolean topicCompactionRetainNullKey = true; @FieldContext( category = CATEGORY_SERVER, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java index 5fa64e9f067cc..a78323a9cfe6d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java @@ -62,7 +62,7 @@ public class TwoPhaseCompactor extends Compactor { private static final Logger log = LoggerFactory.getLogger(TwoPhaseCompactor.class); private static final int MAX_OUTSTANDING = 500; private final Duration phaseOneLoopReadTimeout; - private final boolean topicCompactionRemainNullKey; + private final boolean topicCompactionRetainNullKey; public TwoPhaseCompactor(ServiceConfiguration conf, PulsarClient pulsar, @@ -70,7 +70,7 @@ public TwoPhaseCompactor(ServiceConfiguration conf, ScheduledExecutorService scheduler) { super(conf, pulsar, bk, scheduler); phaseOneLoopReadTimeout = Duration.ofSeconds(conf.getBrokerServiceCompactionPhaseOneLoopTimeInSeconds()); - topicCompactionRemainNullKey = conf.isTopicCompactionRemainNullKey(); + topicCompactionRetainNullKey = conf.isTopicCompactionRetainNullKey(); } @Override @@ -137,7 +137,7 @@ private void phaseOneLoop(RawReader reader, for (ImmutableTriple e : extractIdsAndKeysAndSizeFromBatch(m)) { if (e != null) { if (e.getMiddle() == null) { - if (!topicCompactionRemainNullKey) { + if (!topicCompactionRetainNullKey) { // record delete null-key message event deleteCnt++; mxBean.addCompactionRemovedEvent(reader.getTopic()); @@ -174,7 +174,7 @@ private void phaseOneLoop(RawReader reader, latestForKey.remove(keyAndSize.getLeft()); } } else { - if (!topicCompactionRemainNullKey) { + if (!topicCompactionRetainNullKey) { deletedMessage = true; } } @@ -265,7 +265,7 @@ private void phaseTwoLoop(RawReader reader, MessageId to, Map if (RawBatchConverter.isReadableBatch(m)) { try { messageToAdd = rebatchMessage(reader.getTopic(), - m, (key, subid) -> subid.equals(latestForKey.get(key)), topicCompactionRemainNullKey); + m, (key, subid) -> subid.equals(latestForKey.get(key)), topicCompactionRetainNullKey); } catch (IOException ioe) { log.info("Error decoding batch for message {}. Whole batch will be included in output", id, ioe); @@ -275,7 +275,7 @@ private void phaseTwoLoop(RawReader reader, MessageId to, Map Pair keyAndSize = extractKeyAndSize(m); MessageId msg; if (keyAndSize == null) { - messageToAdd = topicCompactionRemainNullKey ? Optional.of(m) : Optional.empty(); + messageToAdd = topicCompactionRetainNullKey ? Optional.of(m) : Optional.empty(); } else if ((msg = latestForKey.get(keyAndSize.getLeft())) != null && msg.equals(id)) { // consider message only if present into latestForKey map if (keyAndSize.getRight() <= 0) { 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 5ee12d660e031..69baca3abc31c 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 @@ -26,7 +26,6 @@ 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 com.google.common.util.concurrent.ThreadFactoryBuilder; import io.netty.buffer.ByteBuf; @@ -648,9 +647,9 @@ public static Object[][] retainNullKey() { @Test(dataProvider = "retainNullKey") public void testKeyLessMessagesPassThrough(boolean retainNullKey) throws Exception { - conf.setTopicCompactionRemainNullKey(retainNullKey); + conf.setTopicCompactionRetainNullKey(retainNullKey); restartBroker(); - FieldUtils.writeDeclaredField(compactor, "topicCompactionRemainNullKey", retainNullKey, true); + FieldUtils.writeDeclaredField(compactor, "topicCompactionRetainNullKey", retainNullKey, true); String topic = "persistent://my-property/use/my-ns/my-topic1"; From 06584c4a24b49d10bea3008a492ca46c0c6e310c Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Mon, 4 Dec 2023 10:23:00 +0800 Subject: [PATCH 177/699] [improve][broker] Print recoverBucketSnapshot log if cursorProperties are empty (#21651) --- .../broker/delayed/bucket/BucketDelayedDeliveryTracker.java | 2 ++ 1 file changed, 2 insertions(+) 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 d7a3e80f086d2..f98c9e000f150 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 @@ -140,6 +140,8 @@ private synchronized long recoverBucketSnapshot() throws RuntimeException { ManagedCursor cursor = this.lastMutableBucket.getCursor(); Map cursorProperties = cursor.getCursorProperties(); if (MapUtils.isEmpty(cursorProperties)) { + log.info("[{}] Recover delayed message index bucket snapshot finish, don't find bucket snapshot", + dispatcher.getName()); return 0; } FutureUtil.Sequencer sequencer = this.lastMutableBucket.getSequencer(); From 3b117edde32c4d1cf4175c28ae369633696fa5da Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Fri, 8 Dec 2023 10:32:58 +0800 Subject: [PATCH 178/699] [fix][broker] Record GeoPersistentReplicator.msgOut before producer#sendAsync (#21673) Signed-off-by: Zixuan Liu --- .../broker/service/persistent/GeoPersistentReplicator.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java index 08882982297ab..b8287dd2c141a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java @@ -149,9 +149,6 @@ protected boolean replicateEntries(List entries) { } dispatchRateLimiter.ifPresent(rateLimiter -> rateLimiter.tryDispatchPermit(1, entry.getLength())); - - msgOut.recordEvent(headersAndPayload.readableBytes()); - msg.setReplicatedFrom(localCluster); headersAndPayload.retain(); @@ -181,6 +178,7 @@ protected boolean replicateEntries(List entries) { msg.setSchemaInfoForReplicator(schemaFuture.get()); msg.getMessageBuilder().clearTxnidMostBits(); msg.getMessageBuilder().clearTxnidLeastBits(); + msgOut.recordEvent(headersAndPayload.readableBytes()); // Increment pending messages for messages produced locally PENDING_MESSAGES_UPDATER.incrementAndGet(this); producer.sendAsync(msg, ProducerSendCallback.create(this, entry, msg)); From 50eb091bec42be9cfdfe2c109624910224b4c7b5 Mon Sep 17 00:00:00 2001 From: tison Date: Tue, 17 Oct 2023 09:38:02 +0800 Subject: [PATCH 179/699] [fix][sec] Bump avro version to 1.11.3 for CVE-2023-39410 (#21341) Signed-off-by: tison --- distribution/server/src/assemble/LICENSE.bin.txt | 4 ++-- distribution/shell/src/assemble/LICENSE.bin.txt | 4 ++-- pom.xml | 2 +- .../schema/compatibility/SchemaCompatibilityCheckTest.java | 2 +- .../pulsar/client/impl/schema/ProtobufSchemaTest.java | 6 +++--- pulsar-io/kafka-connect-adaptor/pom.xml | 6 ++++++ pulsar-sql/presto-distribution/LICENSE | 4 ++-- 7 files changed, 17 insertions(+), 11 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 6233a835997cf..287fe8e680338 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -446,8 +446,8 @@ The Apache Software License, Version 2.0 - net.jodah-typetools-0.5.0.jar - net.jodah-failsafe-2.4.4.jar * Apache Avro - - org.apache.avro-avro-1.10.2.jar - - org.apache.avro-avro-protobuf-1.10.2.jar + - org.apache.avro-avro-1.11.3.jar + - org.apache.avro-avro-protobuf-1.11.3.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 b2df59482b979..91324dc4a7175 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -407,8 +407,8 @@ The Apache Software License, Version 2.0 * Google Error Prone Annotations - error_prone_annotations-2.5.1.jar * Javassist -- javassist-3.25.0-GA.jar * Apache Avro - - avro-1.10.2.jar - - avro-protobuf-1.10.2.jar + - avro-1.11.3.jar + - avro-protobuf-1.11.3.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 a3870e732573e..48e572e1372af 100644 --- a/pom.xml +++ b/pom.xml @@ -177,7 +177,7 @@ flexible messaging model and an intuitive client API. 3.4.0 5.18.0 1.12.262 - 1.10.2 + 1.11.3 2.10.10 2.5.0 5.1.0 diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/SchemaCompatibilityCheckTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/SchemaCompatibilityCheckTest.java index 140dea9e7ebc7..49517a424b936 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/SchemaCompatibilityCheckTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/schema/compatibility/SchemaCompatibilityCheckTest.java @@ -407,7 +407,7 @@ public void testSchemaComparison() throws Exception { assertEquals(admin.namespaces().getSchemaCompatibilityStrategy(namespaceName.toString()), SchemaCompatibilityStrategy.UNDEFINED); byte[] changeSchemaBytes = (new String(Schema.AVRO(Schemas.PersonOne.class) - .getSchemaInfo().getSchema(), UTF_8) + "/n /n /n").getBytes(); + .getSchemaInfo().getSchema(), UTF_8) + "\n \n \n").getBytes(); SchemaInfo schemaInfo = SchemaInfo.builder().type(SchemaType.AVRO).schema(changeSchemaBytes).build(); admin.schemas().createSchema(fqtn, schemaInfo); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/ProtobufSchemaTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/ProtobufSchemaTest.java index 3fcd6f12b982d..85012276d5af1 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/ProtobufSchemaTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/ProtobufSchemaTest.java @@ -41,20 +41,20 @@ public class ProtobufSchemaTest { "\"namespace\":\"org.apache.pulsar.client.schema.proto.Test\"," + "\"fields\":[{\"name\":\"stringField\",\"type\":{\"type\":\"string\"," + "\"avro.java.string\":\"String\"},\"default\":\"\"},{\"name\":\"doubleField\"," + - "\"type\":\"double\",\"default\":0},{\"name\":\"intField\",\"type\":\"int\"," + + "\"type\":\"double\",\"default\":0.0},{\"name\":\"intField\",\"type\":\"int\"," + "\"default\":0},{\"name\":\"testEnum\",\"type\":{\"type\":\"enum\"," + "\"name\":\"TestEnum\",\"symbols\":[\"SHARED\",\"FAILOVER\"]}," + "\"default\":\"SHARED\"},{\"name\":\"nestedField\"," + "\"type\":[\"null\",{\"type\":\"record\",\"name\":\"SubMessage\"," + "\"fields\":[{\"name\":\"foo\",\"type\":{\"type\":\"string\"," + "\"avro.java.string\":\"String\"},\"default\":\"\"}" + - ",{\"name\":\"bar\",\"type\":\"double\",\"default\":0}]}]" + + ",{\"name\":\"bar\",\"type\":\"double\",\"default\":0.0}]}]" + ",\"default\":null},{\"name\":\"repeatedField\",\"type\":{\"type\":\"array\"" + ",\"items\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},\"default\":[]}" + ",{\"name\":\"externalMessage\",\"type\":[\"null\",{\"type\":\"record\"" + ",\"name\":\"ExternalMessage\",\"namespace\":\"org.apache.pulsar.client.schema.proto.ExternalTest\"" + ",\"fields\":[{\"name\":\"stringField\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}," + - "\"default\":\"\"},{\"name\":\"doubleField\",\"type\":\"double\",\"default\":0}]}],\"default\":null}]}"; + "\"default\":\"\"},{\"name\":\"doubleField\",\"type\":\"double\",\"default\":0.0}]}],\"default\":null}]}"; private static final String EXPECTED_PARSING_INFO = "{\"__alwaysAllowNull\":\"true\",\"__jsr310ConversionEnabled\":\"false\"," + "\"__PARSING_INFO__\":\"[{\\\"number\\\":1,\\\"name\\\":\\\"stringField\\\",\\\"type\\\":\\\"STRING\\\"," + diff --git a/pulsar-io/kafka-connect-adaptor/pom.xml b/pulsar-io/kafka-connect-adaptor/pom.xml index 361dd5ae76387..27b57583dfe82 100644 --- a/pulsar-io/kafka-connect-adaptor/pom.xml +++ b/pulsar-io/kafka-connect-adaptor/pom.xml @@ -116,6 +116,12 @@ io.confluent kafka-connect-avro-converter ${confluent.version} + + + org.apache.avro + avro + + diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 49f4b38a1361c..0e0c99f85a117 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -372,8 +372,8 @@ The Apache Software License, Version 2.0 * OpenCSV - opencsv-2.3.jar * Avro - - avro-1.10.2.jar - - avro-protobuf-1.10.2.jar + - avro-1.11.3.jar + - avro-protobuf-1.11.3.jar * Caffeine - caffeine-2.9.1.jar * Javax From 4bcef6a3158d411980f946a1a1ecaa93b0fba81c Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Wed, 13 Dec 2023 22:55:59 +0530 Subject: [PATCH 180/699] Update groupID of dependencies to com.datastax.oss --- pom.xml | 2 +- pulsar-client-admin-shaded/pom.xml | 12 ++++++------ pulsar-client-all/pom.xml | 10 +++++----- .../pom.xml | 2 +- pulsar-client-tools/pom.xml | 2 +- pulsar-functions/localrun-shaded/pom.xml | 2 +- pulsar-io/http/pom.xml | 2 +- pulsar-io/kinesis/pom.xml | 2 +- pulsar-metadata/pom.xml | 2 +- pulsar-sql/presto-pulsar/pom.xml | 14 +++++++------- tests/docker-images/java-test-functions/pom.xml | 8 ++++---- tests/docker-images/java-test-image/pom.xml | 4 ++-- tests/docker-images/java-test-plugins/pom.xml | 2 +- tests/docker-images/latest-version-image/pom.xml | 2 +- tests/integration/pom.xml | 16 ++++++++-------- tests/pulsar-client-admin-shade-test/pom.xml | 6 +++--- tests/pulsar-client-all-shade-test/pom.xml | 6 +++--- tests/pulsar-client-shade-test/pom.xml | 4 ++-- 18 files changed, 49 insertions(+), 49 deletions(-) diff --git a/pom.xml b/pom.xml index 48e572e1372af..fbcc86f61d9e6 100644 --- a/pom.xml +++ b/pom.xml @@ -1429,7 +1429,7 @@ flexible messaging model and an intuitive client API. - org.apache.pulsar + com.datastax.oss buildtools ${project.version} test diff --git a/pulsar-client-admin-shaded/pom.xml b/pulsar-client-admin-shaded/pom.xml index eb97e49637b3e..53c6e8f7dd7a3 100644 --- a/pulsar-client-admin-shaded/pom.xml +++ b/pulsar-client-admin-shaded/pom.xml @@ -111,8 +111,8 @@ - org.apache.pulsar:pulsar-client-original - org.apache.pulsar:pulsar-client-admin-original + com.datastax.oss:pulsar-client-original + com.datastax.oss:pulsar-client-admin-original org.apache.commons:commons-lang3 commons-codec:commons-codec commons-collections:commons-collections @@ -126,7 +126,7 @@ com.fasterxml.jackson.*:* io.netty:* io.netty.incubator:* - org.apache.pulsar:pulsar-common + com.datastax.oss:pulsar-common org.apache.bookkeeper:* com.yahoo.datasketches:sketches-core org.glassfish.jersey*:* @@ -148,7 +148,7 @@ org.yaml:snakeyaml io.swagger:* - org.apache.pulsar:pulsar-client-messagecrypto-bc + com.datastax.oss:pulsar-client-messagecrypto-bc com.fasterxml.jackson.core:jackson-annotations @@ -156,7 +156,7 @@ - org.apache.pulsar:pulsar-client-original + com.datastax.oss:pulsar-client-original ** @@ -166,7 +166,7 @@ - org.apache.pulsar:pulsar-client-admin-original + com.datastax.oss:pulsar-client-admin-original ** diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index 3935135a5a25b..4e456e622806b 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -149,8 +149,8 @@ - org.apache.pulsar:pulsar-client-original - org.apache.pulsar:pulsar-client-admin-original + com.datastax.oss:pulsar-client-original + com.datastax.com:pulsar-client-admin-original org.apache.commons:commons-lang3 commons-codec:commons-codec commons-collections:commons-collections @@ -177,7 +177,7 @@ io.swagger:* io.airlift:* - org.apache.pulsar:pulsar-common + com.datastax.oss:pulsar-common org.apache.bookkeeper:* com.yahoo.datasketches:sketches-core org.glassfish.jersey*:* @@ -205,7 +205,7 @@ org.apache.commons:commons-compress org.tukaani:xz - org.apache.pulsar:pulsar-client-messagecrypto-bc + com.datastax.oss:pulsar-client-messagecrypto-bc com.fasterxml.jackson.core:jackson-annotations @@ -213,7 +213,7 @@ - org.apache.pulsar:pulsar-client-original + com.datastax.oss:pulsar-client-original ** diff --git a/pulsar-client-tools-customcommand-example/pom.xml b/pulsar-client-tools-customcommand-example/pom.xml index b54638f379165..9c298e49537e4 100644 --- a/pulsar-client-tools-customcommand-example/pom.xml +++ b/pulsar-client-tools-customcommand-example/pom.xml @@ -31,7 +31,7 @@ Pulsar CLI Custom command example - org.apache.pulsar + com.datastax.oss pulsar-client-tools-api ${project.version} provided diff --git a/pulsar-client-tools/pom.xml b/pulsar-client-tools/pom.xml index 55806af6b41dd..76ebec55c49cd 100644 --- a/pulsar-client-tools/pom.xml +++ b/pulsar-client-tools/pom.xml @@ -119,7 +119,7 @@ - org.apache.pulsar + com.datastax.oss pulsar-io-batch-discovery-triggerers ${project.version} test diff --git a/pulsar-functions/localrun-shaded/pom.xml b/pulsar-functions/localrun-shaded/pom.xml index 6bc2c06e1938c..40bc261ed9d42 100644 --- a/pulsar-functions/localrun-shaded/pom.xml +++ b/pulsar-functions/localrun-shaded/pom.xml @@ -142,7 +142,7 @@ - org.apache.pulsar:pulsar-client-original + com.datastax.oss:pulsar-client-original ** diff --git a/pulsar-io/http/pom.xml b/pulsar-io/http/pom.xml index b99db00e604ef..cf84f8bb89344 100644 --- a/pulsar-io/http/pom.xml +++ b/pulsar-io/http/pom.xml @@ -60,7 +60,7 @@ - org.apache.pulsar + ${project.groupId} pulsar-client-original ${project.version} test diff --git a/pulsar-io/kinesis/pom.xml b/pulsar-io/kinesis/pom.xml index d4df05b4037b9..9b2fe19410d3d 100644 --- a/pulsar-io/kinesis/pom.xml +++ b/pulsar-io/kinesis/pom.xml @@ -58,7 +58,7 @@ - org.apache.pulsar + ${project.groupId} pulsar-functions-instance ${project.version} test diff --git a/pulsar-metadata/pom.xml b/pulsar-metadata/pom.xml index 88aa6daf9dca0..1da5389eadbcb 100644 --- a/pulsar-metadata/pom.xml +++ b/pulsar-metadata/pom.xml @@ -33,7 +33,7 @@ Pulsar Metadata - org.apache.pulsar + com.datastax.oss pulsar-common ${project.version} diff --git a/pulsar-sql/presto-pulsar/pom.xml b/pulsar-sql/presto-pulsar/pom.xml index c8f5be55ff188..87d79802f2abd 100644 --- a/pulsar-sql/presto-pulsar/pom.xml +++ b/pulsar-sql/presto-pulsar/pom.xml @@ -137,14 +137,14 @@ - org.apache.pulsar + com.datastax.oss pulsar-broker ${project.version} test - org.apache.pulsar + com.datastax.oss testmocks ${project.version} test @@ -176,10 +176,10 @@ - org.apache.pulsar:pulsar-client-original - org.apache.pulsar:pulsar-client-admin-original - org.apache.pulsar:managed-ledger - org.apache.pulsar:pulsar-metadata + com.datastax.oss:pulsar-client-original + com.datastax.oss:pulsar-client-admin-original + com.datastax.oss:managed-ledger + com.datastax.oss:pulsar-metadata org.glassfish.jersey*:* javax.ws.rs:* @@ -192,7 +192,7 @@ - org.apache.pulsar:pulsar-client-original + com.datastax.oss:pulsar-client-original ** diff --git a/tests/docker-images/java-test-functions/pom.xml b/tests/docker-images/java-test-functions/pom.xml index 61719af01178b..5dceae4b308ab 100644 --- a/tests/docker-images/java-test-functions/pom.xml +++ b/tests/docker-images/java-test-functions/pom.xml @@ -30,12 +30,12 @@ Apache Pulsar :: Tests :: Docker Images :: Java Test Functions - org.apache.pulsar + com.datastax.oss pulsar-io-core ${project.version} - org.apache.pulsar + com.datastax.oss pulsar-functions-api ${project.version} @@ -71,7 +71,7 @@ - org.apache.pulsar + com.datastax.oss pulsar-functions-api-examples ${project.version} @@ -91,7 +91,7 @@ - org.apache.pulsar:pulsar-functions-api-examples + com.datastax.oss:pulsar-functions-api-examples diff --git a/tests/docker-images/java-test-image/pom.xml b/tests/docker-images/java-test-image/pom.xml index 175a185f41422..3f21dd8617f67 100644 --- a/tests/docker-images/java-test-image/pom.xml +++ b/tests/docker-images/java-test-image/pom.xml @@ -50,7 +50,7 @@ ${project.parent.version} - org.apache.pulsar + com.datastax.oss pulsar-server-distribution ${project.parent.version} bin @@ -87,7 +87,7 @@ java-test-functions.jar - org.apache.pulsar + com.datastax.oss pulsar-server-distribution ${project.parent.version} bin diff --git a/tests/docker-images/java-test-plugins/pom.xml b/tests/docker-images/java-test-plugins/pom.xml index b8b1b0012b3b4..64e8860a5ef97 100644 --- a/tests/docker-images/java-test-plugins/pom.xml +++ b/tests/docker-images/java-test-plugins/pom.xml @@ -31,7 +31,7 @@ jar - org.apache.pulsar + com.datastax.oss pulsar-broker ${project.version} provided diff --git a/tests/docker-images/latest-version-image/pom.xml b/tests/docker-images/latest-version-image/pom.xml index e99a4100d13d8..247d3bc45ba4a 100644 --- a/tests/docker-images/latest-version-image/pom.xml +++ b/tests/docker-images/latest-version-image/pom.xml @@ -50,7 +50,7 @@ ${project.parent.version} - org.apache.pulsar + com.datastax.oss pulsar-all-docker-image ${project.parent.version} pom diff --git a/tests/integration/pom.xml b/tests/integration/pom.xml index 44cca702ad2ed..a8f4071ddc3de 100644 --- a/tests/integration/pom.xml +++ b/tests/integration/pom.xml @@ -44,37 +44,37 @@ test - org.apache.pulsar + com.datastax.oss pulsar-functions-api-examples ${project.version} test - org.apache.pulsar + com.datastax.oss pulsar-broker ${project.version} test - org.apache.pulsar + com.datastax.oss pulsar-common ${project.version} test - org.apache.pulsar + com.datastax.oss pulsar-client-original ${project.version} test - org.apache.pulsar + com.datastax.oss pulsar-client-admin-original ${project.version} test - org.apache.pulsar + com.datastax.oss managed-ledger ${project.version} test @@ -96,7 +96,7 @@ test - org.apache.pulsar + com.datastax.oss pulsar-io-kafka ${project.version} test @@ -133,7 +133,7 @@ - org.apache.pulsar + com.datastax.oss pulsar-io-jdbc-postgres ${project.version} test diff --git a/tests/pulsar-client-admin-shade-test/pom.xml b/tests/pulsar-client-admin-shade-test/pom.xml index aba48a9abe335..0ca4b9c48342d 100644 --- a/tests/pulsar-client-admin-shade-test/pom.xml +++ b/tests/pulsar-client-admin-shade-test/pom.xml @@ -36,14 +36,14 @@ - org.apache.pulsar + com.datastax.oss pulsar-client-admin ${project.version} test - org.apache.pulsar + com.datastax.oss pulsar-client-admin-api ${project.version} test @@ -56,7 +56,7 @@ - org.apache.pulsar + com.datastax.oss pulsar-client-messagecrypto-bc ${project.version} test diff --git a/tests/pulsar-client-all-shade-test/pom.xml b/tests/pulsar-client-all-shade-test/pom.xml index 598af60141f7e..3ffc048da8688 100644 --- a/tests/pulsar-client-all-shade-test/pom.xml +++ b/tests/pulsar-client-all-shade-test/pom.xml @@ -36,7 +36,7 @@ - org.apache.pulsar + com.datastax.oss pulsar-client-all ${project.version} test @@ -49,14 +49,14 @@ - org.apache.pulsar + com.datastax.oss bouncy-castle-bc ${project.version} pkg - org.apache.pulsar + com.datastax.oss pulsar-client-messagecrypto-bc ${project.version} diff --git a/tests/pulsar-client-shade-test/pom.xml b/tests/pulsar-client-shade-test/pom.xml index 63d7befebfc3c..454732c59a36f 100644 --- a/tests/pulsar-client-shade-test/pom.xml +++ b/tests/pulsar-client-shade-test/pom.xml @@ -37,14 +37,14 @@ - org.apache.pulsar + com.datastax.oss pulsar-client ${project.version} test - org.apache.pulsar + com.datastax.oss pulsar-client-admin ${project.version} test From f837644c2f238bbc824b5b10b43f4f2e17bc9f95 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Tue, 19 Dec 2023 09:50:22 +0530 Subject: [PATCH 181/699] Updated groupID of artifact --- pulsar-client-all/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index 4e456e622806b..f58dec65e8115 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -150,7 +150,7 @@ com.datastax.oss:pulsar-client-original - com.datastax.com:pulsar-client-admin-original + com.datastax.oss:pulsar-client-admin-original org.apache.commons:commons-lang3 commons-codec:commons-codec commons-collections:commons-collections From 890f5edffc19f454371b8b3ee214e34177aec263 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Tue, 19 Dec 2023 17:26:54 +0530 Subject: [PATCH 182/699] Updated groupIDs in excludes --- distribution/server/src/assemble/bin.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/distribution/server/src/assemble/bin.xml b/distribution/server/src/assemble/bin.xml index 41ac24d0582da..aafb559d67fb2 100644 --- a/distribution/server/src/assemble/bin.xml +++ b/distribution/server/src/assemble/bin.xml @@ -133,12 +133,12 @@ ${artifact.groupId}-${artifact.artifactId}-${artifact.version}${dashClassifier?}.${artifact.extension} - org.apache.pulsar:pulsar-functions-runtime-all + com.datastax.oss:pulsar-functions-runtime-all org.projectlombok:lombok - org.apache.pulsar:pulsar-functions-api-examples + com.datastax.oss:pulsar-functions-api-examples *:tar.gz From 544e82faeae693c53d9c76c54c795813eda85365 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Tue, 19 Dec 2023 19:33:08 +0530 Subject: [PATCH 183/699] Fix for WrongTypeOfReturnValue due to limitations of concurrency testing when spy is stubbed --- .../org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 311e6a2369972..b990e434df330 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 @@ -4185,7 +4185,7 @@ public void testNoCleanupOffloadLedgerWhenMetadataExceptionHappens() throws Exce // mock the read handle to make the offload successful CompletableFuture readHandle = new CompletableFuture<>(); readHandle.complete(mock(ReadHandle.class)); - when(ml.getLedgerHandle(eq(ledgerInfo.getLedgerId()))).thenReturn(readHandle); + doReturn(readHandle).when(ml).getLedgerHandle(eq(ledgerInfo.getLedgerId())); when(ledgerOffloader.offload(any(), any(), anyMap())).thenReturn(CompletableFuture.completedFuture(null)); ml.ledgers.put(ledgerInfo.getLedgerId(), ledgerInfo); From 4f88a7498a48fafa482f57d5b3eb805431339ce4 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Wed, 20 Dec 2023 16:52:06 +0530 Subject: [PATCH 184/699] Add -fae option to mvn initialize while discovering sub-modules. mvn initialize fails to discover all the sub-modules if the release versions are not available in mvn repo. This would fail only incase of new releases where the artifacts are not yet published into the mvn repo. Error: Failed to execute goal on project pulsar-client-all-shade-test: Could not resolve dependencies for project org.apache.pulsar.tests:pulsar-client-all-shade-test:jar:3.1.2: Failed to collect dependencies at com.datastax.oss:pulsar-client-all:jar:3.1.2: Failed to read artifact descriptor for com.datastax.oss:pulsar-client-all:jar:3.1.2: Could not transfer artifact com.datastax.oss:pulsar-client-all:pom:3.1.2 from/to --- build/run_integration_group.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build/run_integration_group.sh b/build/run_integration_group.sh index 9372d1ecb9317..ffe20390cf7fa 100755 --- a/build/run_integration_group.sh +++ b/build/run_integration_group.sh @@ -33,7 +33,7 @@ TESTNG_VERSION="7.3.0" # returns a CSV value mvn_list_modules() { ( - mvn -B -ntp -Dscan=false "$@" initialize \ + mvn -fae -B -ntp -Dscan=false "$@" initialize \ | grep -- "-< .* >-" \ | sed -E 's/.*-< (.*) >-.*/\1/' \ | tr '\n' ',' | sed 's/,$/\n/' From f4d13a475663aeed14eec6bb6cfdc94e3949f89e Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Wed, 20 Dec 2023 17:42:17 +0530 Subject: [PATCH 185/699] Updated pulsar-experimental version --- docker/pulsar-experimental/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/pulsar-experimental/pom.xml b/docker/pulsar-experimental/pom.xml index 89344e610a37d..c19a278607a9b 100644 --- a/docker/pulsar-experimental/pom.xml +++ b/docker/pulsar-experimental/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.1 + 3.1.2 4.0.0 pulsar-experimental-docker-image From 6895a1498cbe7420a1c4b2037d1f3d87c15b8c15 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 8 Dec 2023 13:33:30 +0800 Subject: [PATCH 186/699] [fix] [broker] network package lost if enable haProxyProtocolEnabled (#21684) Fixes #21557 ### Motivation There is a network package loss issue after enabling `haProxyProtocolEnabled`, which leads the error `Checksum failed on the broker` and `Adjusted frame length exceeds`, you can reproduce the issue by the test `testSlowNetwork`. ### Modifications Fix the bug. (cherry picked from commit 6e18874d6f007be501827550b9894422ecd8eda8) --- .../service/EnableProxyProtocolTest.java | 128 +++++++++++++++--- .../api/InjectedClientCnxClientBuilder.java | 52 +++++++ .../OptionalProxyProtocolDecoder.java | 37 ++++- 3 files changed, 192 insertions(+), 25 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/InjectedClientCnxClientBuilder.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/EnableProxyProtocolTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/EnableProxyProtocolTest.java index 2f128fe6270a5..33e797fcb219f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/EnableProxyProtocolTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/EnableProxyProtocolTest.java @@ -19,9 +19,18 @@ package org.apache.pulsar.broker.service; import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandlerContext; +import java.util.concurrent.TimeUnit; import lombok.Cleanup; -import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.client.api.ClientBuilder; +import org.apache.pulsar.client.api.InjectedClientCnxClientBuilder; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.ClientBuilderImpl; import org.apache.pulsar.client.impl.ClientCnx; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.policies.data.SubscriptionStats; @@ -32,10 +41,6 @@ import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import java.net.InetSocketAddress; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; - @Test(groups = "broker") public class EnableProxyProtocolTest extends BrokerTestBase { @@ -46,6 +51,15 @@ protected void setup() throws Exception { super.baseSetup(); } + protected PulsarClient newPulsarClient(String url, int intervalInSecs) throws PulsarClientException { + ClientBuilder clientBuilder = + PulsarClient.builder() + .serviceUrl(url) + .statsInterval(intervalInSecs, TimeUnit.SECONDS); + customizeNewPulsarClientBuilder(clientBuilder); + return createNewPulsarClient(clientBuilder); + } + @AfterClass(alwaysRun = true) @Override protected void cleanup() throws Exception { @@ -53,7 +67,7 @@ protected void cleanup() throws Exception { } @Test - public void testSimpleProduceAndConsume() throws PulsarClientException { + public void testSimpleProduceAndConsume() throws Exception { final String namespace = "prop/ns-abc"; final String topicName = "persistent://" + namespace + "/testSimpleProduceAndConsume"; final String subName = "my-subscriber-name"; @@ -76,30 +90,104 @@ public void testSimpleProduceAndConsume() throws PulsarClientException { } Assert.assertEquals(received, messages); + + // cleanup. + org.apache.pulsar.broker.service.Consumer serverConsumer = pulsar.getBrokerService().getTopicReference(topicName) + .get().getSubscription(subName).getConsumers().get(0); + ((ServerCnx) serverConsumer.cnx()).close(); + consumer.close(); + producer.close(); + admin.topics().delete(topicName); } @Test - public void testProxyProtocol() throws PulsarClientException, ExecutionException, InterruptedException, PulsarAdminException { + public void testProxyProtocol() throws Exception { final String namespace = "prop/ns-abc"; final String topicName = "persistent://" + namespace + "/testProxyProtocol"; final String subName = "my-subscriber-name"; - PulsarClientImpl client = (PulsarClientImpl) pulsarClient; - CompletableFuture cnx = client.getCnxPool().getConnection(InetSocketAddress.createUnresolved("localhost", pulsar.getBrokerListenPort().get())); - // Simulate the proxy protcol message - cnx.get().ctx().channel().writeAndFlush(Unpooled.copiedBuffer("PROXY TCP4 198.51.100.22 203.0.113.7 35646 80\r\n".getBytes())); - pulsarClient.newConsumer().topic(topicName).subscriptionName(subName) - .subscribe(); - org.apache.pulsar.broker.service.Consumer c = pulsar.getBrokerService().getTopicReference(topicName).get().getSubscription(subName).getConsumers().get(0); - Awaitility.await().untilAsserted(() -> Assert.assertTrue(c.cnx().hasHAProxyMessage())); + + // Create a client that injected the protocol implementation. + ClientBuilderImpl clientBuilder = (ClientBuilderImpl) PulsarClient.builder().serviceUrl(lookupUrl.toString()); + PulsarClientImpl protocolClient = InjectedClientCnxClientBuilder.create(clientBuilder, + (conf, eventLoopGroup) -> new ClientCnx(conf, eventLoopGroup) { + public void channelActive(ChannelHandlerContext ctx) throws Exception { + byte[] bs = "PROXY TCP4 198.51.100.22 203.0.113.7 35646 80\r\n".getBytes(); + ctx.writeAndFlush(Unpooled.copiedBuffer(bs)); + super.channelActive(ctx); + } + }); + + // Verify the addr can be handled correctly. + testPubAndSub(topicName, subName, "198.51.100.22:35646", protocolClient); + + // cleanup. + admin.topics().delete(topicName); + } + + @Test(timeOut = 10000) + public void testPubSubWhenSlowNetwork() throws Exception { + final String namespace = "prop/ns-abc"; + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + namespace + "/tp"); + final String subName = "my-subscriber-name"; + + // Create a client that injected the protocol implementation. + ClientBuilderImpl clientBuilder = (ClientBuilderImpl) PulsarClient.builder().serviceUrl(lookupUrl.toString()); + PulsarClientImpl protocolClient = InjectedClientCnxClientBuilder.create(clientBuilder, + (conf, eventLoopGroup) -> new ClientCnx(conf, eventLoopGroup) { + public void channelActive(ChannelHandlerContext ctx) throws Exception { + Thread task = new Thread(() -> { + try { + byte[] bs1 = "PROXY".getBytes(); + byte[] bs2 = " TCP4 198.51.100.22 203.0.113.7 35646 80\r\n".getBytes(); + ctx.writeAndFlush(Unpooled.copiedBuffer(bs1)); + Thread.sleep(100); + ctx.writeAndFlush(Unpooled.copiedBuffer(bs2)); + super.channelActive(ctx); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + task.start(); + } + }); + + // Verify the addr can be handled correctly. + testPubAndSub(topicName, subName, "198.51.100.22:35646", protocolClient); + + // cleanup. + admin.topics().delete(topicName); + } + + private void testPubAndSub(String topicName, String subName, String expectedHostAndPort, + PulsarClientImpl pulsarClient) throws Exception { + // Verify: subscribe + org.apache.pulsar.client.api.Consumer clientConsumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName) + .subscriptionName(subName).subscribe(); + org.apache.pulsar.broker.service.Consumer serverConsumer = pulsar.getBrokerService() + .getTopicReference(topicName).get().getSubscription(subName).getConsumers().get(0); + Awaitility.await().untilAsserted(() -> Assert.assertTrue(serverConsumer.cnx().hasHAProxyMessage())); TopicStats topicStats = admin.topics().getStats(topicName); Assert.assertEquals(topicStats.getSubscriptions().size(), 1); SubscriptionStats subscriptionStats = topicStats.getSubscriptions().get(subName); Assert.assertEquals(subscriptionStats.getConsumers().size(), 1); - Assert.assertEquals(subscriptionStats.getConsumers().get(0).getAddress(), "198.51.100.22:35646"); + Assert.assertEquals(subscriptionStats.getConsumers().get(0).getAddress(), expectedHostAndPort); + + // Verify: producer register. + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topicName).create(); + TopicStats topicStats2 = admin.topics().getStats(topicName); + Assert.assertEquals(topicStats2.getPublishers().size(), 1); + Assert.assertEquals(topicStats2.getPublishers().get(0).getAddress(), expectedHostAndPort); + + // Verify: Pub & Sub + producer.send("1"); + Message msg = clientConsumer.receive(2, TimeUnit.SECONDS); + Assert.assertNotNull(msg); + Assert.assertEquals(msg.getValue(), "1"); + clientConsumer.acknowledge(msg); - pulsarClient.newProducer().topic(topicName).create(); - topicStats = admin.topics().getStats(topicName); - Assert.assertEquals(topicStats.getPublishers().size(), 1); - Assert.assertEquals(topicStats.getPublishers().get(0).getAddress(), "198.51.100.22:35646"); + // cleanup. + ((ServerCnx) serverConsumer.cnx()).close(); + producer.close(); + clientConsumer.close(); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InjectedClientCnxClientBuilder.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InjectedClientCnxClientBuilder.java new file mode 100644 index 0000000000000..d29dd4f7061b8 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/InjectedClientCnxClientBuilder.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.client.api; + +import io.netty.channel.EventLoopGroup; +import java.util.concurrent.ThreadFactory; +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.PulsarClientImpl; +import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.common.util.netty.EventLoopUtil; + +public class InjectedClientCnxClientBuilder { + + public static PulsarClientImpl create(final ClientBuilderImpl clientBuilder, + final ClientCnxFactory clientCnxFactory) throws Exception { + ClientConfigurationData conf = clientBuilder.getClientConfigurationData(); + ThreadFactory threadFactory = new ExecutorProvider + .ExtendedThreadFactory("pulsar-client-io", Thread.currentThread().isDaemon()); + EventLoopGroup eventLoopGroup = + EventLoopUtil.newEventLoopGroup(conf.getNumIoThreads(), conf.isEnableBusyWait(), threadFactory); + + // Inject into ClientCnx. + ConnectionPool pool = new ConnectionPool(conf, eventLoopGroup, + () -> clientCnxFactory.generate(conf, eventLoopGroup)); + + return new PulsarClientImpl(conf, eventLoopGroup, pool); + } + + public interface ClientCnxFactory { + + ClientCnx generate(ClientConfigurationData conf, EventLoopGroup eventLoopGroup); + } +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/OptionalProxyProtocolDecoder.java b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/OptionalProxyProtocolDecoder.java index 2f0a7884dde35..b4e15f8cd1d75 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/OptionalProxyProtocolDecoder.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/OptionalProxyProtocolDecoder.java @@ -19,36 +19,63 @@ package org.apache.pulsar.common.protocol; import io.netty.buffer.ByteBuf; +import io.netty.buffer.CompositeByteBuf; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; import io.netty.handler.codec.ProtocolDetectionResult; import io.netty.handler.codec.ProtocolDetectionState; import io.netty.handler.codec.haproxy.HAProxyMessageDecoder; import io.netty.handler.codec.haproxy.HAProxyProtocolVersion; +import lombok.extern.slf4j.Slf4j; /** * Decoder that added whether a new connection is prefixed with the ProxyProtocol. * More about the ProxyProtocol see: http://www.haproxy.org/download/1.8/doc/proxy-protocol.txt. */ +@Slf4j public class OptionalProxyProtocolDecoder extends ChannelInboundHandlerAdapter { public static final String NAME = "optional-proxy-protocol-decoder"; + public static final int MIN_BYTES_SIZE_TO_DETECT_PROTOCOL = 12; + + private CompositeByteBuf cumulatedByteBuf; + @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { if (msg instanceof ByteBuf) { - ProtocolDetectionResult result = - HAProxyMessageDecoder.detectProtocol((ByteBuf) msg); - // should accumulate data if need more data to detect the protocol + // Combine cumulated buffers. + ByteBuf buf = (ByteBuf) msg; + if (cumulatedByteBuf != null) { + buf = cumulatedByteBuf.addComponent(true, buf); + } + + ProtocolDetectionResult result = HAProxyMessageDecoder.detectProtocol(buf); if (result.state() == ProtocolDetectionState.NEEDS_MORE_DATA) { + // Accumulate data if need more data to detect the protocol. + if (cumulatedByteBuf == null) { + cumulatedByteBuf = new CompositeByteBuf(ctx.alloc(), false, MIN_BYTES_SIZE_TO_DETECT_PROTOCOL, buf); + } return; } + cumulatedByteBuf = null; if (result.state() == ProtocolDetectionState.DETECTED) { ctx.pipeline().addAfter(NAME, null, new HAProxyMessageDecoder()); - ctx.pipeline().remove(this); } + ctx.pipeline().remove(this); + super.channelRead(ctx, buf); + } else { + super.channelRead(ctx, msg); + } + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + super.channelInactive(ctx); + if (cumulatedByteBuf != null) { + log.info("Release cumulated byte buffer when channel inactive."); + cumulatedByteBuf = null; } - super.channelRead(ctx, msg); } } From 7fe8c82941e0db722ba4ba77b85b871653ee6e58 Mon Sep 17 00:00:00 2001 From: Masahiro Sakamoto Date: Sat, 16 Dec 2023 01:24:04 +0900 Subject: [PATCH 187/699] [fix][sec] Upgrade org.bouncycastle:bc-fips to 1.0.2.4 (#21730) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index fbcc86f61d9e6..9eb54682d8cd3 100644 --- a/pom.xml +++ b/pom.xml @@ -154,7 +154,7 @@ flexible messaging model and an intuitive client API. 2.18.0 1.75 1.0.6 - 1.0.2.3 + 1.0.2.4 2.14.2 0.10.2 1.6.2 From 8e25c5af235a1d102ea4c6dd639effbecdf64bf0 Mon Sep 17 00:00:00 2001 From: Christophe Bornet Date: Tue, 28 Feb 2023 16:51:26 -0400 Subject: [PATCH 188/699] Add support for custom proxy lookup handler (#163) (cherry picked from commit 62f86e9fd62e54cc16708086f4ddeafcb2de41c4) Conflicts: pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyConnection.java --- .../server/DefaultLookupProxyHandler.java | 472 ++++++++++++++++++ .../proxy/server/LookupProxyHandler.java | 419 +--------------- .../proxy/server/ProxyConfiguration.java | 6 + .../pulsar/proxy/server/ProxyConnection.java | 14 +- .../pulsar/proxy/server/ProxyService.java | 4 +- .../server/URLRegexLookupProxyHandler.java | 131 +++++ .../server/ProxyStuckConnectionTest.java | 4 +- .../URLRegexLookupProxyHandlerTest.java | 168 +++++++ 8 files changed, 808 insertions(+), 410 deletions(-) create mode 100644 pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DefaultLookupProxyHandler.java create mode 100644 pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/URLRegexLookupProxyHandler.java create mode 100644 pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/URLRegexLookupProxyHandlerTest.java diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DefaultLookupProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DefaultLookupProxyHandler.java new file mode 100644 index 0000000000000..b62d988904f63 --- /dev/null +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DefaultLookupProxyHandler.java @@ -0,0 +1,472 @@ +/* + * 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.proxy.server; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Semaphore; +import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.impl.BinaryProtoLookupService; +import org.apache.pulsar.common.api.proto.CommandGetSchema; +import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; +import org.apache.pulsar.common.api.proto.CommandLookupTopic; +import org.apache.pulsar.common.api.proto.CommandLookupTopicResponse.LookupType; +import org.apache.pulsar.common.api.proto.CommandPartitionedTopicMetadata; +import org.apache.pulsar.common.api.proto.ServerError; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.protocol.schema.BytesSchemaVersion; +import org.apache.pulsar.common.protocol.schema.SchemaVersion; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.netty.NettyChannelUtil; +import org.apache.pulsar.policies.data.loadbalancer.ServiceLookupData; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DefaultLookupProxyHandler implements LookupProxyHandler { + protected final String throttlingErrorMessage = "Too many concurrent lookup and partitionsMetadata requests"; + protected ProxyConnection proxyConnection; + protected BrokerDiscoveryProvider discoveryProvider; + protected boolean connectWithTLS; + + protected SocketAddress clientAddress; + protected String brokerServiceURL; + + protected Semaphore lookupRequestSemaphore; + + @Override + public void initialize(ProxyService proxy, ProxyConnection proxyConnection) { + this.discoveryProvider = proxy.getDiscoveryProvider(); + this.lookupRequestSemaphore = proxy.getLookupRequestSemaphore(); + this.proxyConnection = proxyConnection; + this.clientAddress = proxyConnection.clientAddress(); + this.connectWithTLS = proxy.getConfiguration().isTlsEnabledWithBroker(); + this.brokerServiceURL = this.connectWithTLS ? proxy.getConfiguration().getBrokerServiceURLTLS() + : proxy.getConfiguration().getBrokerServiceURL(); + } + + public void handleLookup(CommandLookupTopic lookup) { + if (log.isDebugEnabled()) { + log.debug("Received Lookup from {}", clientAddress); + } + long clientRequestId = lookup.getRequestId(); + if (lookupRequestSemaphore.tryAcquire()) { + try { + LOOKUP_REQUESTS.inc(); + String serviceUrl = getBrokerServiceUrl(clientRequestId); + if (serviceUrl != null) { + performLookup(clientRequestId, lookup.getTopic(), serviceUrl, false, 10) + .whenComplete( + (brokerUrl, ex) -> { + if (ex != null) { + ServerError serverError = ex instanceof LookupException + ? ((LookupException) ex).getServerError() + : getServerError(ex); + proxyConnection.ctx().writeAndFlush( + Commands.newLookupErrorResponse(serverError, ex.getMessage(), + clientRequestId)); + } else { + proxyConnection.ctx().writeAndFlush( + Commands.newLookupResponse(brokerUrl, brokerUrl, true, + LookupType.Connect, clientRequestId, + true /* this is coming from proxy */)); + } + }); + } + } finally { + lookupRequestSemaphore.release(); + } + } else { + REJECTED_LOOKUP_REQUESTS.inc(); + if (log.isDebugEnabled()) { + log.debug("Lookup Request ID {} from {} rejected - {}.", clientRequestId, clientAddress, + throttlingErrorMessage); + } + writeAndFlush(Commands.newLookupErrorResponse(ServerError.ServiceNotReady, + throttlingErrorMessage, clientRequestId)); + } + } + + protected static class LookupException extends RuntimeException { + private final ServerError serverError; + + public LookupException(ServerError serverError, String message) { + super(message); + this.serverError = serverError; + } + + public ServerError getServerError() { + return serverError; + } + } + + protected CompletableFuture performLookup(long clientRequestId, String topic, String brokerServiceUrl, + boolean authoritative, int numberOfRetries) { + if (numberOfRetries == 0) { + writeAndFlush(Commands.newLookupErrorResponse(ServerError.ServiceNotReady, + "Reached max number of redirections", clientRequestId)); + return FutureUtil.failedFuture( + new LookupException(ServerError.ServiceNotReady, "Reached max number of redirections")); + } + + URI brokerURI; + try { + brokerURI = new URI(brokerServiceUrl); + } catch (URISyntaxException e) { + writeAndFlush( + Commands.newLookupErrorResponse(ServerError.MetadataError, e.getMessage(), clientRequestId)); + return FutureUtil.failedFuture(new LookupException(ServerError.MetadataError, e.getMessage())); + } + + InetSocketAddress addr = InetSocketAddress.createUnresolved(brokerURI.getHost(), brokerURI.getPort()); + if (log.isDebugEnabled()) { + log.debug("Getting connections to '{}' for Looking up topic '{}' with clientReq Id '{}'", addr, topic, + clientRequestId); + } + CompletableFuture brokerUrlFuture = new CompletableFuture<>(); + proxyConnection.getConnectionPool().getConnection(addr).thenAccept(clientCnx -> { + // Connected to backend broker + long requestId = proxyConnection.newRequestId(); + ByteBuf command; + command = Commands.newLookup(topic, authoritative, requestId); + + clientCnx.newLookup(command, requestId).whenComplete((r, t) -> { + if (t != null) { + log.warn("[{}] Failed to lookup topic {}: {}", clientAddress, topic, t.getMessage()); + writeAndFlush( + Commands.newLookupErrorResponse(getServerError(t), t.getMessage(), clientRequestId)); + brokerUrlFuture.completeExceptionally(t); + } else { + String brokerUrl = resolveBrokerUrlFromLookupDataResult(r); + if (r.redirect) { + // Need to try the lookup again on a different broker + performLookup(clientRequestId, topic, brokerUrl, r.authoritative, numberOfRetries - 1) + .whenComplete((result, ex) -> { + if (ex != null) { + brokerUrlFuture.completeExceptionally(ex); + } else { + brokerUrlFuture.complete(result); + } + }); + } else { + // Reply the same address for both TLS non-TLS. The reason + // is that whether we use TLS + // and broker is independent of whether the client itself + // uses TLS, but we need to force the + // client + // to use the appropriate target broker (and port) when it + // will connect back. + if (log.isDebugEnabled()) { + log.debug("Successfully perform lookup '{}' for topic '{}'" + + " with clientReq Id '{}' and lookup-broker {}", + addr, topic, clientRequestId, brokerUrl); + } + brokerUrlFuture.complete(brokerUrl); + } + } + proxyConnection.getConnectionPool().releaseConnection(clientCnx); + }); + }).exceptionally(ex -> { + // Failed to connect to backend broker + writeAndFlush( + Commands.newLookupErrorResponse(getServerError(ex), ex.getMessage(), clientRequestId)); + brokerUrlFuture.completeExceptionally(ex); + return null; + }); + return brokerUrlFuture; + } + + protected String resolveBrokerUrlFromLookupDataResult(BinaryProtoLookupService.LookupDataResult r) { + return connectWithTLS ? r.brokerUrlTls : r.brokerUrl; + } + + public void handlePartitionMetadataResponse(CommandPartitionedTopicMetadata partitionMetadata) { + PARTITIONS_METADATA_REQUESTS.inc(); + if (log.isDebugEnabled()) { + log.debug("[{}] Received PartitionMetadataLookup", clientAddress); + } + final long clientRequestId = partitionMetadata.getRequestId(); + if (lookupRequestSemaphore.tryAcquire()) { + try { + handlePartitionMetadataResponse(partitionMetadata, clientRequestId); + } finally { + lookupRequestSemaphore.release(); + } + } else { + REJECTED_PARTITIONS_METADATA_REQUESTS.inc(); + if (log.isDebugEnabled()) { + log.debug("PartitionMetaData Request ID {} from {} rejected - {}.", clientRequestId, clientAddress, + throttlingErrorMessage); + } + writeAndFlush(Commands.newPartitionMetadataResponse(ServerError.ServiceNotReady, + throttlingErrorMessage, clientRequestId)); + } + } + + /** + * Always get partition metadata from broker service. + **/ + private void handlePartitionMetadataResponse(CommandPartitionedTopicMetadata partitionMetadata, + long clientRequestId) { + TopicName topicName = TopicName.get(partitionMetadata.getTopic()); + + String serviceUrl = getBrokerServiceUrl(clientRequestId); + if (serviceUrl == null) { + log.warn("No available broker for {} to lookup partition metadata", topicName); + return; + } + InetSocketAddress addr = getAddr(serviceUrl, clientRequestId); + if (addr == null) { + return; + } + + if (log.isDebugEnabled()) { + log.debug("Getting connections to '{}' for Looking up topic '{}' with clientReq Id '{}'", addr, + topicName.getPartitionedTopicName(), clientRequestId); + } + proxyConnection.getConnectionPool().getConnection(addr).thenAccept(clientCnx -> { + // Connected to backend broker + long requestId = proxyConnection.newRequestId(); + ByteBuf command; + command = Commands.newPartitionMetadataRequest(topicName.toString(), requestId); + clientCnx.newLookup(command, requestId).whenComplete((r, t) -> { + if (t != null) { + log.warn("[{}] failed to get Partitioned metadata : {}", topicName.toString(), + t.getMessage(), t); + writeAndFlush(Commands.newLookupErrorResponse(getServerError(t), + t.getMessage(), clientRequestId)); + } else { + writeAndFlush( + Commands.newPartitionMetadataResponse(r.partitions, clientRequestId)); + } + proxyConnection.getConnectionPool().releaseConnection(clientCnx); + }); + }).exceptionally(ex -> { + // Failed to connect to backend broker + writeAndFlush(Commands.newPartitionMetadataResponse(getServerError(ex), + ex.getMessage(), clientRequestId)); + return null; + }); + } + + public void handleGetTopicsOfNamespace(CommandGetTopicsOfNamespace commandGetTopicsOfNamespace) { + GET_TOPICS_OF_NAMESPACE_REQUESTS.inc(); + if (log.isDebugEnabled()) { + log.debug("[{}] Received GetTopicsOfNamespace", clientAddress); + } + + final long requestId = commandGetTopicsOfNamespace.getRequestId(); + + if (lookupRequestSemaphore.tryAcquire()) { + try { + handleGetTopicsOfNamespace(commandGetTopicsOfNamespace, requestId); + } finally { + lookupRequestSemaphore.release(); + } + } else { + REJECTED_GET_TOPICS_OF_NAMESPACE_REQUESTS.inc(); + if (log.isDebugEnabled()) { + log.debug("GetTopicsOfNamespace Request ID {} from {} rejected - {}.", requestId, clientAddress, + throttlingErrorMessage); + } + writeAndFlush(Commands.newError( + requestId, ServerError.ServiceNotReady, throttlingErrorMessage + )); + } + } + + private void handleGetTopicsOfNamespace(CommandGetTopicsOfNamespace commandGetTopicsOfNamespace, + long clientRequestId) { + String serviceUrl = getBrokerServiceUrl(clientRequestId); + + if (!StringUtils.isNotBlank(serviceUrl)) { + return; + } + String topicsPattern = commandGetTopicsOfNamespace.hasTopicsPattern() + ? commandGetTopicsOfNamespace.getTopicsPattern() : null; + String topicsHash = commandGetTopicsOfNamespace.hasTopicsHash() + ? commandGetTopicsOfNamespace.getTopicsHash() : null; + performGetTopicsOfNamespace(clientRequestId, commandGetTopicsOfNamespace.getNamespace(), serviceUrl, + 10, topicsPattern, topicsHash, commandGetTopicsOfNamespace.getMode()); + } + + private void performGetTopicsOfNamespace(long clientRequestId, + String namespaceName, + String brokerServiceUrl, + int numberOfRetries, + String topicsPattern, + String topicsHash, + CommandGetTopicsOfNamespace.Mode mode) { + if (numberOfRetries == 0) { + writeAndFlush(Commands.newError(clientRequestId, ServerError.ServiceNotReady, + "Reached max number of redirections")); + return; + } + + InetSocketAddress addr = getAddr(brokerServiceUrl, clientRequestId); + + if (addr == null) { + return; + } + + if (log.isDebugEnabled()) { + log.debug("Getting connections to '{}' for getting TopicsOfNamespace '{}' with clientReq Id '{}'", + addr, namespaceName, clientRequestId); + } + proxyConnection.getConnectionPool().getConnection(addr).thenAccept(clientCnx -> { + // Connected to backend broker + long requestId = proxyConnection.newRequestId(); + ByteBuf command; + command = Commands.newGetTopicsOfNamespaceRequest(namespaceName, requestId, mode, + topicsPattern, topicsHash); + clientCnx.newGetTopicsOfNamespace(command, requestId).whenComplete((r, t) -> { + if (t != null) { + log.warn("[{}] Failed to get TopicsOfNamespace {}: {}", + clientAddress, namespaceName, t.getMessage()); + writeAndFlush( + Commands.newError(clientRequestId, getServerError(t), t.getMessage())); + } else { + writeAndFlush( + Commands.newGetTopicsOfNamespaceResponse(r.getTopics(), r.getTopicsHash(), r.isFiltered(), + r.isChanged(), clientRequestId)); + } + }); + + proxyConnection.getConnectionPool().releaseConnection(clientCnx); + }).exceptionally(ex -> { + // Failed to connect to backend broker + writeAndFlush( + Commands.newError(clientRequestId, getServerError(ex), ex.getMessage())); + return null; + }); + } + + public void handleGetSchema(CommandGetSchema commandGetSchema) { + GET_SCHEMA_REQUESTS.inc(); + if (log.isDebugEnabled()) { + log.debug("[{}] Received GetSchema {}", clientAddress, commandGetSchema); + } + + final long clientRequestId = commandGetSchema.getRequestId(); + String serviceUrl = getBrokerServiceUrl(clientRequestId); + String topic = commandGetSchema.getTopic(); + Optional schemaVersion; + if (commandGetSchema.hasSchemaVersion()) { + schemaVersion = Optional.of(commandGetSchema.getSchemaVersion()).map(BytesSchemaVersion::of); + } else { + schemaVersion = Optional.empty(); + } + + if (!StringUtils.isNotBlank(serviceUrl)) { + return; + } + InetSocketAddress addr = getAddr(serviceUrl, clientRequestId); + + if (addr == null) { + return; + } + if (log.isDebugEnabled()) { + log.debug("Getting connections to '{}' for getting schema of topic '{}' with clientReq Id '{}'", + addr, topic, clientRequestId); + } + + proxyConnection.getConnectionPool().getConnection(addr).thenAccept(clientCnx -> { + // Connected to backend broker + long requestId = proxyConnection.newRequestId(); + ByteBuf command; + command = Commands.newGetSchema(requestId, topic, schemaVersion); + clientCnx.sendGetRawSchema(command, requestId).whenComplete((r, t) -> { + if (t != null) { + log.warn("[{}] Failed to get schema {}: {}", clientAddress, topic, t); + writeAndFlush( + Commands.newError(clientRequestId, getServerError(t), t.getMessage())); + } else { + writeAndFlush( + Commands.newGetSchemaResponse(clientRequestId, r)); + } + + proxyConnection.getConnectionPool().releaseConnection(clientCnx); + }); + }).exceptionally(ex -> { + // Failed to connect to backend broker + writeAndFlush( + Commands.newError(clientRequestId, getServerError(ex), ex.getMessage())); + return null; + }); + + } + + /** + * Get default broker service url or discovery an available broker. + **/ + protected String getBrokerServiceUrl(long clientRequestId) { + if (StringUtils.isNotBlank(brokerServiceURL)) { + return brokerServiceURL; + } + ServiceLookupData availableBroker; + try { + availableBroker = discoveryProvider.nextBroker(); + } catch (Exception e) { + log.warn("[{}] Failed to get next active broker {}", clientAddress, e.getMessage(), e); + writeAndFlush(Commands.newError( + clientRequestId, ServerError.ServiceNotReady, e.getMessage() + )); + return null; + } + return this.connectWithTLS ? availableBroker.getPulsarServiceUrlTls() : availableBroker.getPulsarServiceUrl(); + } + + private InetSocketAddress getAddr(String brokerServiceUrl, long clientRequestId) { + URI brokerURI; + try { + brokerURI = new URI(brokerServiceUrl); + } catch (URISyntaxException e) { + writeAndFlush( + Commands.newError(clientRequestId, ServerError.MetadataError, e.getMessage())); + return null; + } + return InetSocketAddress.createUnresolved(brokerURI.getHost(), brokerURI.getPort()); + } + + protected ServerError getServerError(Throwable error) { + ServerError responseError; + if (error instanceof PulsarClientException.AuthorizationException) { + responseError = ServerError.AuthorizationError; + } else if (error instanceof PulsarClientException.AuthenticationException) { + responseError = ServerError.AuthenticationError; + } else { + responseError = ServerError.ServiceNotReady; + } + return responseError; + } + + private void writeAndFlush(ByteBuf cmd) { + final ChannelHandlerContext ctx = proxyConnection.ctx(); + NettyChannelUtil.writeAndFlushWithVoidPromise(ctx, cmd); + } + + private static final Logger log = LoggerFactory.getLogger(LookupProxyHandler.class); +} diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java index b62b3bacf0114..23259d0cccf68 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/LookupProxyHandler.java @@ -18,445 +18,52 @@ */ package org.apache.pulsar.proxy.server; -import io.netty.buffer.ByteBuf; -import io.netty.channel.ChannelHandlerContext; import io.prometheus.client.Counter; -import java.net.InetSocketAddress; -import java.net.SocketAddress; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.Optional; -import java.util.concurrent.Semaphore; -import org.apache.commons.lang3.StringUtils; -import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.impl.BinaryProtoLookupService; import org.apache.pulsar.common.api.proto.CommandGetSchema; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; import org.apache.pulsar.common.api.proto.CommandLookupTopic; -import org.apache.pulsar.common.api.proto.CommandLookupTopicResponse.LookupType; import org.apache.pulsar.common.api.proto.CommandPartitionedTopicMetadata; -import org.apache.pulsar.common.api.proto.ServerError; -import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.protocol.Commands; -import org.apache.pulsar.common.protocol.schema.BytesSchemaVersion; -import org.apache.pulsar.common.protocol.schema.SchemaVersion; -import org.apache.pulsar.common.util.netty.NettyChannelUtil; -import org.apache.pulsar.policies.data.loadbalancer.ServiceLookupData; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -public class LookupProxyHandler { - private final String throttlingErrorMessage = "Too many concurrent lookup and partitionsMetadata requests"; - private final ProxyConnection proxyConnection; - private final BrokerDiscoveryProvider discoveryProvider; - private final boolean connectWithTLS; +public interface LookupProxyHandler { - private final SocketAddress clientAddress; - private final String brokerServiceURL; - - private static final Counter LOOKUP_REQUESTS = Counter + Counter LOOKUP_REQUESTS = Counter .build("pulsar_proxy_lookup_requests", "Counter of topic lookup requests").create().register(); - private static final Counter PARTITIONS_METADATA_REQUESTS = Counter + Counter PARTITIONS_METADATA_REQUESTS = Counter .build("pulsar_proxy_partitions_metadata_requests", "Counter of partitions metadata requests").create() .register(); - private static final Counter GET_TOPICS_OF_NAMESPACE_REQUESTS = Counter + Counter GET_TOPICS_OF_NAMESPACE_REQUESTS = Counter .build("pulsar_proxy_get_topics_of_namespace_requests", "Counter of getTopicsOfNamespace requests") .create() .register(); - private static final Counter GET_SCHEMA_REQUESTS = Counter + Counter GET_SCHEMA_REQUESTS = Counter .build("pulsar_proxy_get_schema_requests", "Counter of schema requests") .create() .register(); - static final Counter REJECTED_LOOKUP_REQUESTS = Counter.build("pulsar_proxy_rejected_lookup_requests", + Counter REJECTED_LOOKUP_REQUESTS = Counter.build("pulsar_proxy_rejected_lookup_requests", "Counter of topic lookup requests rejected due to throttling").create().register(); - static final Counter REJECTED_PARTITIONS_METADATA_REQUESTS = Counter + Counter REJECTED_PARTITIONS_METADATA_REQUESTS = Counter .build("pulsar_proxy_rejected_partitions_metadata_requests", "Counter of partitions metadata requests rejected due to throttling") .create().register(); - static final Counter REJECTED_GET_TOPICS_OF_NAMESPACE_REQUESTS = Counter + Counter REJECTED_GET_TOPICS_OF_NAMESPACE_REQUESTS = Counter .build("pulsar_proxy_rejected_get_topics_of_namespace_requests", "Counter of getTopicsOfNamespace requests rejected due to throttling") .create().register(); - private final Semaphore lookupRequestSemaphore; - - public LookupProxyHandler(ProxyService proxy, ProxyConnection proxyConnection) { - this.discoveryProvider = proxy.getDiscoveryProvider(); - this.lookupRequestSemaphore = proxy.getLookupRequestSemaphore(); - this.proxyConnection = proxyConnection; - this.clientAddress = proxyConnection.clientAddress(); - this.connectWithTLS = proxy.getConfiguration().isTlsEnabledWithBroker(); - this.brokerServiceURL = this.connectWithTLS ? proxy.getConfiguration().getBrokerServiceURLTLS() - : proxy.getConfiguration().getBrokerServiceURL(); - } - - public void handleLookup(CommandLookupTopic lookup) { - if (log.isDebugEnabled()) { - log.debug("Received Lookup from {}", clientAddress); - } - long clientRequestId = lookup.getRequestId(); - if (lookupRequestSemaphore.tryAcquire()) { - try { - LOOKUP_REQUESTS.inc(); - String serviceUrl = getBrokerServiceUrl(clientRequestId); - if (serviceUrl != null) { - performLookup(clientRequestId, lookup.getTopic(), serviceUrl, false, 10); - } - } finally { - lookupRequestSemaphore.release(); - } - } else { - REJECTED_LOOKUP_REQUESTS.inc(); - if (log.isDebugEnabled()) { - log.debug("Lookup Request ID {} from {} rejected - {}.", clientRequestId, clientAddress, - throttlingErrorMessage); - } - writeAndFlush(Commands.newLookupErrorResponse(ServerError.ServiceNotReady, - throttlingErrorMessage, clientRequestId)); - } - - } - - private void performLookup(long clientRequestId, String topic, String brokerServiceUrl, boolean authoritative, - int numberOfRetries) { - if (numberOfRetries == 0) { - writeAndFlush(Commands.newLookupErrorResponse(ServerError.ServiceNotReady, - "Reached max number of redirections", clientRequestId)); - return; - } - - URI brokerURI; - try { - brokerURI = new URI(brokerServiceUrl); - } catch (URISyntaxException e) { - writeAndFlush( - Commands.newLookupErrorResponse(ServerError.MetadataError, e.getMessage(), clientRequestId)); - return; - } - - InetSocketAddress addr = InetSocketAddress.createUnresolved(brokerURI.getHost(), brokerURI.getPort()); - if (log.isDebugEnabled()) { - log.debug("Getting connections to '{}' for Looking up topic '{}' with clientReq Id '{}'", addr, topic, - clientRequestId); - } - proxyConnection.getConnectionPool().getConnection(addr).thenAccept(clientCnx -> { - // Connected to backend broker - long requestId = proxyConnection.newRequestId(); - ByteBuf command; - command = Commands.newLookup(topic, authoritative, requestId); - - clientCnx.newLookup(command, requestId).whenComplete((r, t) -> { - if (t != null) { - log.warn("[{}] Failed to lookup topic {}: {}", clientAddress, topic, t.getMessage()); - writeAndFlush( - Commands.newLookupErrorResponse(getServerError(t), t.getMessage(), clientRequestId)); - } else { - String brokerUrl = resolveBrokerUrlFromLookupDataResult(r); - if (r.redirect) { - // Need to try the lookup again on a different broker - performLookup(clientRequestId, topic, brokerUrl, r.authoritative, numberOfRetries - 1); - } else { - // Reply the same address for both TLS non-TLS. The reason - // is that whether we use TLS - // and broker is independent of whether the client itself - // uses TLS, but we need to force the - // client - // to use the appropriate target broker (and port) when it - // will connect back. - if (log.isDebugEnabled()) { - log.debug("Successfully perform lookup '{}' for topic '{}'" - + " with clientReq Id '{}' and lookup-broker {}", - addr, topic, clientRequestId, brokerUrl); - } - writeAndFlush(Commands.newLookupResponse(brokerUrl, brokerUrl, true, - LookupType.Connect, clientRequestId, true /* this is coming from proxy */)); - } - } - proxyConnection.getConnectionPool().releaseConnection(clientCnx); - }); - }).exceptionally(ex -> { - // Failed to connect to backend broker - writeAndFlush( - Commands.newLookupErrorResponse(getServerError(ex), ex.getMessage(), clientRequestId)); - return null; - }); - } - - protected String resolveBrokerUrlFromLookupDataResult(BinaryProtoLookupService.LookupDataResult r) { - return connectWithTLS ? r.brokerUrlTls : r.brokerUrl; - } - - public void handlePartitionMetadataResponse(CommandPartitionedTopicMetadata partitionMetadata) { - PARTITIONS_METADATA_REQUESTS.inc(); - if (log.isDebugEnabled()) { - log.debug("[{}] Received PartitionMetadataLookup", clientAddress); - } - final long clientRequestId = partitionMetadata.getRequestId(); - if (lookupRequestSemaphore.tryAcquire()) { - try { - handlePartitionMetadataResponse(partitionMetadata, clientRequestId); - } finally { - lookupRequestSemaphore.release(); - } - } else { - REJECTED_PARTITIONS_METADATA_REQUESTS.inc(); - if (log.isDebugEnabled()) { - log.debug("PartitionMetaData Request ID {} from {} rejected - {}.", clientRequestId, clientAddress, - throttlingErrorMessage); - } - writeAndFlush(Commands.newPartitionMetadataResponse(ServerError.ServiceNotReady, - throttlingErrorMessage, clientRequestId)); - } - } - - /** - * Always get partition metadata from broker service. - * - * - **/ - private void handlePartitionMetadataResponse(CommandPartitionedTopicMetadata partitionMetadata, - long clientRequestId) { - TopicName topicName = TopicName.get(partitionMetadata.getTopic()); - - String serviceUrl = getBrokerServiceUrl(clientRequestId); - if (serviceUrl == null) { - log.warn("No available broker for {} to lookup partition metadata", topicName); - return; - } - InetSocketAddress addr = getAddr(serviceUrl, clientRequestId); - if (addr == null) { - return; - } - - if (log.isDebugEnabled()) { - log.debug("Getting connections to '{}' for Looking up topic '{}' with clientReq Id '{}'", addr, - topicName.getPartitionedTopicName(), clientRequestId); - } - proxyConnection.getConnectionPool().getConnection(addr).thenAccept(clientCnx -> { - // Connected to backend broker - long requestId = proxyConnection.newRequestId(); - ByteBuf command; - command = Commands.newPartitionMetadataRequest(topicName.toString(), requestId); - clientCnx.newLookup(command, requestId).whenComplete((r, t) -> { - if (t != null) { - log.warn("[{}] failed to get Partitioned metadata : {}", topicName.toString(), - t.getMessage(), t); - writeAndFlush(Commands.newLookupErrorResponse(getServerError(t), - t.getMessage(), clientRequestId)); - } else { - writeAndFlush( - Commands.newPartitionMetadataResponse(r.partitions, clientRequestId)); - } - proxyConnection.getConnectionPool().releaseConnection(clientCnx); - }); - }).exceptionally(ex -> { - // Failed to connect to backend broker - writeAndFlush(Commands.newPartitionMetadataResponse(getServerError(ex), - ex.getMessage(), clientRequestId)); - return null; - }); - } - - public void handleGetTopicsOfNamespace(CommandGetTopicsOfNamespace commandGetTopicsOfNamespace) { - GET_TOPICS_OF_NAMESPACE_REQUESTS.inc(); - if (log.isDebugEnabled()) { - log.debug("[{}] Received GetTopicsOfNamespace", clientAddress); - } - - final long requestId = commandGetTopicsOfNamespace.getRequestId(); - - if (lookupRequestSemaphore.tryAcquire()) { - try { - handleGetTopicsOfNamespace(commandGetTopicsOfNamespace, requestId); - } finally { - lookupRequestSemaphore.release(); - } - } else { - REJECTED_GET_TOPICS_OF_NAMESPACE_REQUESTS.inc(); - if (log.isDebugEnabled()) { - log.debug("GetTopicsOfNamespace Request ID {} from {} rejected - {}.", requestId, clientAddress, - throttlingErrorMessage); - } - writeAndFlush(Commands.newError( - requestId, ServerError.ServiceNotReady, throttlingErrorMessage - )); - } - } - - private void handleGetTopicsOfNamespace(CommandGetTopicsOfNamespace commandGetTopicsOfNamespace, - long clientRequestId) { - String serviceUrl = getBrokerServiceUrl(clientRequestId); - - if (!StringUtils.isNotBlank(serviceUrl)) { - return; - } - String topicsPattern = commandGetTopicsOfNamespace.hasTopicsPattern() - ? commandGetTopicsOfNamespace.getTopicsPattern() : null; - String topicsHash = commandGetTopicsOfNamespace.hasTopicsHash() - ? commandGetTopicsOfNamespace.getTopicsHash() : null; - performGetTopicsOfNamespace(clientRequestId, commandGetTopicsOfNamespace.getNamespace(), serviceUrl, - 10, topicsPattern, topicsHash, commandGetTopicsOfNamespace.getMode()); - } - - private void performGetTopicsOfNamespace(long clientRequestId, - String namespaceName, - String brokerServiceUrl, - int numberOfRetries, - String topicsPattern, - String topicsHash, - CommandGetTopicsOfNamespace.Mode mode) { - if (numberOfRetries == 0) { - writeAndFlush(Commands.newError(clientRequestId, ServerError.ServiceNotReady, - "Reached max number of redirections")); - return; - } - - InetSocketAddress addr = getAddr(brokerServiceUrl, clientRequestId); - - if (addr == null) { - return; - } - - if (log.isDebugEnabled()) { - log.debug("Getting connections to '{}' for getting TopicsOfNamespace '{}' with clientReq Id '{}'", - addr, namespaceName, clientRequestId); - } - proxyConnection.getConnectionPool().getConnection(addr).thenAccept(clientCnx -> { - // Connected to backend broker - long requestId = proxyConnection.newRequestId(); - ByteBuf command; - command = Commands.newGetTopicsOfNamespaceRequest(namespaceName, requestId, mode, - topicsPattern, topicsHash); - clientCnx.newGetTopicsOfNamespace(command, requestId).whenComplete((r, t) -> { - if (t != null) { - log.warn("[{}] Failed to get TopicsOfNamespace {}: {}", - clientAddress, namespaceName, t.getMessage()); - writeAndFlush( - Commands.newError(clientRequestId, getServerError(t), t.getMessage())); - } else { - writeAndFlush( - Commands.newGetTopicsOfNamespaceResponse(r.getTopics(), r.getTopicsHash(), r.isFiltered(), - r.isChanged(), clientRequestId)); - } - }); - - proxyConnection.getConnectionPool().releaseConnection(clientCnx); - }).exceptionally(ex -> { - // Failed to connect to backend broker - writeAndFlush( - Commands.newError(clientRequestId, getServerError(ex), ex.getMessage())); - return null; - }); - } - - public void handleGetSchema(CommandGetSchema commandGetSchema) { - GET_SCHEMA_REQUESTS.inc(); - if (log.isDebugEnabled()) { - log.debug("[{}] Received GetSchema {}", clientAddress, commandGetSchema); - } - - final long clientRequestId = commandGetSchema.getRequestId(); - String serviceUrl = getBrokerServiceUrl(clientRequestId); - String topic = commandGetSchema.getTopic(); - Optional schemaVersion; - if (commandGetSchema.hasSchemaVersion()) { - schemaVersion = Optional.of(commandGetSchema.getSchemaVersion()).map(BytesSchemaVersion::of); - } else { - schemaVersion = Optional.empty(); - } - - if (!StringUtils.isNotBlank(serviceUrl)) { - return; - } - InetSocketAddress addr = getAddr(serviceUrl, clientRequestId); - - if (addr == null) { - return; - } - if (log.isDebugEnabled()) { - log.debug("Getting connections to '{}' for getting schema of topic '{}' with clientReq Id '{}'", - addr, topic, clientRequestId); - } - - proxyConnection.getConnectionPool().getConnection(addr).thenAccept(clientCnx -> { - // Connected to backend broker - long requestId = proxyConnection.newRequestId(); - ByteBuf command; - command = Commands.newGetSchema(requestId, topic, schemaVersion); - clientCnx.sendGetRawSchema(command, requestId).whenComplete((r, t) -> { - if (t != null) { - log.warn("[{}] Failed to get schema {}: {}", clientAddress, topic, t); - writeAndFlush( - Commands.newError(clientRequestId, getServerError(t), t.getMessage())); - } else { - writeAndFlush( - Commands.newGetSchemaResponse(clientRequestId, r)); - } - - proxyConnection.getConnectionPool().releaseConnection(clientCnx); - }); - }).exceptionally(ex -> { - // Failed to connect to backend broker - writeAndFlush( - Commands.newError(clientRequestId, getServerError(ex), ex.getMessage())); - return null; - }); - } + void initialize(ProxyService proxy, ProxyConnection proxyConnection); - /** - * Get default broker service url or discovery an available broker. - **/ - private String getBrokerServiceUrl(long clientRequestId) { - if (StringUtils.isNotBlank(brokerServiceURL)) { - return brokerServiceURL; - } - ServiceLookupData availableBroker; - try { - availableBroker = discoveryProvider.nextBroker(); - } catch (Exception e) { - log.warn("[{}] Failed to get next active broker {}", clientAddress, e.getMessage(), e); - writeAndFlush(Commands.newError( - clientRequestId, ServerError.ServiceNotReady, e.getMessage() - )); - return null; - } - return this.connectWithTLS ? availableBroker.getPulsarServiceUrlTls() : availableBroker.getPulsarServiceUrl(); - } + void handleLookup(CommandLookupTopic lookup); - private InetSocketAddress getAddr(String brokerServiceUrl, long clientRequestId) { - URI brokerURI; - try { - brokerURI = new URI(brokerServiceUrl); - } catch (URISyntaxException e) { - writeAndFlush( - Commands.newError(clientRequestId, ServerError.MetadataError, e.getMessage())); - return null; - } - return InetSocketAddress.createUnresolved(brokerURI.getHost(), brokerURI.getPort()); - } + void handlePartitionMetadataResponse(CommandPartitionedTopicMetadata partitionMetadata); - private ServerError getServerError(Throwable error) { - ServerError responseError; - if (error instanceof PulsarClientException.AuthorizationException) { - responseError = ServerError.AuthorizationError; - } else if (error instanceof PulsarClientException.AuthenticationException) { - responseError = ServerError.AuthenticationError; - } else { - responseError = ServerError.ServiceNotReady; - } - return responseError; - } + void handleGetTopicsOfNamespace(CommandGetTopicsOfNamespace commandGetTopicsOfNamespace); - private void writeAndFlush(ByteBuf cmd) { - final ChannelHandlerContext ctx = proxyConnection.ctx(); - NettyChannelUtil.writeAndFlushWithVoidPromise(ctx, cmd); - } + void handleGetSchema(CommandGetSchema commandGetSchema); - private static final Logger log = LoggerFactory.getLogger(LookupProxyHandler.class); } 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 a4cb7926bebf1..b9564eba8239b 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 @@ -193,6 +193,12 @@ public class ProxyConfiguration implements PulsarConfiguration { ) private String brokerWebServiceURLTLS; + @FieldContext( + category = CATEGORY_BROKER_DISCOVERY, + doc = "The lookup handler class name" + ) + private String lookupHandler = "org.apache.pulsar.proxy.server.DefaultLookupProxyHandler"; + @FieldContext( category = CATEGORY_BROKER_DISCOVERY, doc = "The web service url points to the function worker cluster." 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 ba9247a085dff..d7e91f4530f57 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 @@ -73,6 +73,7 @@ import org.apache.pulsar.common.api.proto.ServerError; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.PulsarHandler; +import org.apache.pulsar.common.util.Reflections; import org.apache.pulsar.common.util.Runnables; import org.apache.pulsar.common.util.netty.NettyChannelUtil; import org.apache.pulsar.policies.data.loadbalancer.ServiceLookupData; @@ -396,7 +397,18 @@ private synchronized void completeConnect() throws PulsarClientException { } state = State.ProxyLookupRequests; - lookupProxyHandler = service.newLookupProxyHandler(this); + try { + lookupProxyHandler = + Reflections.createInstance(service.getConfiguration().getLookupHandler(), LookupProxyHandler.class, + Thread.currentThread() + .getContextClassLoader()); + lookupProxyHandler.initialize(service, this); + } catch (Exception e) { + LOG.error("Failed to initialize lookup proxy handler", e); + ctx().writeAndFlush(Commands.newError(-1, ServerError.ServiceNotReady, + "Failed to initialize lookup proxy handler")).addListener(ChannelFutureListener.CLOSE); + return; + } if (service.getConfiguration().isAuthenticationEnabled() && service.getConfiguration().getAuthenticationRefreshCheckSeconds() > 0) { authRefreshTask = ctx.executor().scheduleAtFixedRate( 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 a934b8b078426..b360820ad007c 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 @@ -530,6 +530,8 @@ public synchronized void addPrometheusRawMetricsProvider(PrometheusRawMetricsPro private static final Logger LOG = LoggerFactory.getLogger(ProxyService.class); protected LookupProxyHandler newLookupProxyHandler(ProxyConnection proxyConnection) { - return new LookupProxyHandler(this, proxyConnection); + LookupProxyHandler newLookupProxyHandler = new DefaultLookupProxyHandler(); + newLookupProxyHandler.initialize(this, proxyConnection); + return newLookupProxyHandler; } } diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/URLRegexLookupProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/URLRegexLookupProxyHandler.java new file mode 100644 index 0000000000000..d66986985176f --- /dev/null +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/URLRegexLookupProxyHandler.java @@ -0,0 +1,131 @@ +/* + * 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.proxy.server; + +import com.google.common.base.Strings; +import java.util.Properties; +import java.util.regex.Pattern; +import org.apache.pulsar.common.api.proto.CommandLookupTopic; +import org.apache.pulsar.common.api.proto.CommandLookupTopicResponse; +import org.apache.pulsar.common.api.proto.ServerError; +import org.apache.pulsar.common.protocol.Commands; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class URLRegexLookupProxyHandler extends DefaultLookupProxyHandler { + + private static final Logger log = LoggerFactory.getLogger(URLRegexLookupProxyHandler.class); + + private Pattern pattern; + + private String replacement; + + @Override + public void initialize(ProxyService proxy, ProxyConnection proxyConnection) { + super.initialize(proxy, proxyConnection); + Properties properties = proxy.getConfiguration().getProperties(); + String regex = properties.getProperty("urlRegexLookupProxyHandlerRegex"); + if (Strings.isNullOrEmpty(regex)) { + throw new IllegalArgumentException("urlRegexLookupProxyHandlerRegex is not set"); + } + this.pattern = Pattern.compile(regex); + this.replacement = properties.getProperty("urlRegexLookupProxyHandlerReplacement"); + if (Strings.isNullOrEmpty(this.replacement)) { + throw new IllegalArgumentException("urlRegexLookupProxyHandlerReplacement is not set"); + } + } + + @Override + public void handleLookup(CommandLookupTopic lookup) { + if (log.isDebugEnabled()) { + log.debug("Received Lookup from {}", clientAddress); + } + long clientRequestId = lookup.getRequestId(); + if (lookupRequestSemaphore.tryAcquire()) { + try { + LOOKUP_REQUESTS.inc(); + String serviceUrl = getBrokerServiceUrl(clientRequestId); + if (serviceUrl != null) { + if (lookup.isAuthoritative()) { + performLookup(clientRequestId, lookup.getTopic(), serviceUrl, false, 10) + .whenComplete( + (brokerUrl, ex) -> { + if (ex != null) { + ServerError serverError = ex instanceof LookupException + ? ((LookupException) ex).getServerError() + : getServerError(ex); + proxyConnection.ctx().writeAndFlush( + Commands.newLookupErrorResponse(serverError, ex.getMessage(), + clientRequestId)); + } else { + proxyConnection.ctx().writeAndFlush( + Commands.newLookupResponse(brokerUrl, brokerUrl, true, + CommandLookupTopicResponse.LookupType.Connect, clientRequestId, + true /* this is coming from proxy */)); + } + }); + } else { + performLookup(clientRequestId, lookup.getTopic(), serviceUrl, false, 10) + .whenComplete( + (brokerUrl, ex) -> { + try { + if (pattern.matcher(brokerUrl).matches()) { + if (log.isDebugEnabled()) { + log.debug("Broker URL {} matches regex {}", brokerUrl, pattern); + } + String proxyUrl = pattern.matcher(brokerUrl).replaceAll(replacement); + if (log.isDebugEnabled()) { + log.debug("Redirect to proxy URL {}", proxyUrl); + } + proxyConnection.ctx().writeAndFlush( + Commands.newLookupResponse(proxyUrl, proxyUrl, true, + CommandLookupTopicResponse.LookupType.Redirect, clientRequestId, + false)); + } else { + if (log.isDebugEnabled()) { + log.debug("Broker URL {} doesn't match regex {}", brokerUrl, pattern); + } + proxyConnection.ctx().writeAndFlush( + Commands.newLookupErrorResponse(ServerError.ServiceNotReady, + "Broker URL does not match the lookup handler regex", + clientRequestId)); + } + } catch (IllegalArgumentException iae) { + proxyConnection.ctx().writeAndFlush( + Commands.newLookupErrorResponse(ServerError.ServiceNotReady, + iae.getMessage(), clientRequestId)); + } + }); + + } + } + } finally { + lookupRequestSemaphore.release(); + } + } else { + REJECTED_LOOKUP_REQUESTS.inc(); + if (log.isDebugEnabled()) { + log.debug("Lookup Request ID {} from {} rejected - {}.", clientRequestId, clientAddress, + throttlingErrorMessage); + } + proxyConnection.ctx().writeAndFlush(Commands.newLookupErrorResponse(ServerError.ServiceNotReady, + throttlingErrorMessage, clientRequestId)); + } + } +} diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStuckConnectionTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStuckConnectionTest.java index 97279659af626..373926d11c10d 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStuckConnectionTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStuckConnectionTest.java @@ -110,9 +110,9 @@ protected void cleanup() throws Exception { } } - private final class TestLookupProxyHandler extends LookupProxyHandler { + private final class TestLookupProxyHandler extends DefaultLookupProxyHandler { public TestLookupProxyHandler(ProxyService proxy, ProxyConnection proxyConnection) { - super(proxy, proxyConnection); + super.initialize(proxy, proxyConnection); } @Override diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/URLRegexLookupProxyHandlerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/URLRegexLookupProxyHandlerTest.java new file mode 100644 index 0000000000000..96c63f5cae9d0 --- /dev/null +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/URLRegexLookupProxyHandlerTest.java @@ -0,0 +1,168 @@ +/* + * 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.proxy.server; + +import static org.mockito.Mockito.doReturn; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertThrows; +import static org.testng.Assert.assertTrue; + +import java.util.Optional; +import java.util.concurrent.TimeUnit; + +import lombok.Cleanup; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +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.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; +import org.apache.pulsar.metadata.impl.ZKMetadataStore; +import org.mockito.Mockito; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +public class URLRegexLookupProxyHandlerTest extends MockedPulsarServiceBaseTest { + + protected ProxyService proxyService; + protected ProxyConfiguration proxyConfig = new ProxyConfiguration(); + + @Override + @BeforeClass + protected void setup() throws Exception { + internalSetup(); + + proxyConfig.setServicePort(Optional.of(0)); + proxyConfig.setBrokerProxyAllowedTargetPorts("*"); + proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); + proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( + PulsarConfigurationLoader.convertFrom(proxyConfig)))); + doReturn(new ZKMetadataStore(mockZooKeeper)).when(proxyService).createLocalMetadataStore(); + doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(proxyService).createConfigurationMetadataStore(); + + proxyService.start(); + } + + @Override + protected void cleanup() throws Exception { + internalCleanup(); + + proxyService.close(); + } + + @Test + void testMatchingRegex() throws Exception { + ProxyConfiguration redirectProxyConfig = new ProxyConfiguration(); + redirectProxyConfig.setServicePort(Optional.of(0)); + redirectProxyConfig.setBrokerProxyAllowedTargetPorts("*"); + redirectProxyConfig.setMetadataStoreUrl(DUMMY_VALUE); + redirectProxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + redirectProxyConfig.setLookupHandler("org.apache.pulsar.proxy.server.URLRegexLookupProxyHandler"); + redirectProxyConfig.getProperties().setProperty("urlRegexLookupProxyHandlerRegex", "pulsar:\\/\\/(\\w+):\\d+"); + redirectProxyConfig.getProperties() + .setProperty("urlRegexLookupProxyHandlerReplacement", proxyService.getServiceUrl()); + + @Cleanup + ProxyService redirectProxyService = Mockito.spy(new ProxyService(redirectProxyConfig, new AuthenticationService( + PulsarConfigurationLoader.convertFrom(redirectProxyConfig)))); + doReturn(new ZKMetadataStore(mockZooKeeper)).when(redirectProxyService).createLocalMetadataStore(); + doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(redirectProxyService).createConfigurationMetadataStore(); + + redirectProxyService.start(); + + // Check that the target proxy is not connected to any broker at the moment + assertEquals(proxyService.getClientCnxs().size(), 0); + + @Cleanup + PulsarClient client = PulsarClient.builder().serviceUrl(redirectProxyService.getServiceUrl()) + .lookupTimeout(5, TimeUnit.SECONDS) + .build(); + + @Cleanup + Producer producer = client.newProducer(Schema.BYTES) + .topic("persistent://sample/test/local/producer-consumer-topic") + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create(); + + // Create a consumer directly attached to broker + @Cleanup + Consumer consumer = pulsarClient.newConsumer() + .topic("persistent://sample/test/local/producer-consumer-topic").subscriptionName("my-sub").subscribe(); + + for (int i = 0; i < 10; i++) { + producer.send("test".getBytes()); + } + + for (int i = 0; i < 10; i++) { + Message msg = consumer.receive(1, TimeUnit.SECONDS); + assertNotNull(msg); + consumer.acknowledge(msg); + } + + Message msg = consumer.receive(0, TimeUnit.SECONDS); + assertNull(msg); + + // Check that the target proxy now has connections to the broker + assertTrue(proxyService.getClientCnxs().size() > 0); + } + + @Test + void testNotMatchingRegex() throws Exception { + ProxyConfiguration redirectProxyConfig = new ProxyConfiguration(); + redirectProxyConfig.setServicePort(Optional.of(0)); + redirectProxyConfig.setBrokerProxyAllowedTargetPorts("*"); + redirectProxyConfig.setMetadataStoreUrl(DUMMY_VALUE); + redirectProxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + redirectProxyConfig.setLookupHandler("org.apache.pulsar.proxy.server.URLRegexLookupProxyHandler"); + redirectProxyConfig.getProperties().setProperty("urlRegexLookupProxyHandlerRegex", "invalid"); + redirectProxyConfig.getProperties().setProperty("urlRegexLookupProxyHandlerReplacement", proxyService.getServiceUrl()); + + @Cleanup + ProxyService redirectProxyService = Mockito.spy(new ProxyService(redirectProxyConfig, new AuthenticationService( + PulsarConfigurationLoader.convertFrom(redirectProxyConfig)))); + doReturn(new ZKMetadataStore(mockZooKeeper)).when(redirectProxyService).createLocalMetadataStore(); + doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(redirectProxyService).createConfigurationMetadataStore(); + + redirectProxyService.start(); + + // Check that the target proxy is not connected to any broker at the moment + assertEquals(proxyService.getClientCnxs().size(), 0); + + @Cleanup + PulsarClient client = PulsarClient.builder().serviceUrl(redirectProxyService.getServiceUrl()) + .lookupTimeout(5, TimeUnit.SECONDS) + .build(); + + assertThrows(PulsarClientException.LookupException.class, () -> client.newProducer(Schema.BYTES) + .topic("persistent://sample/test/local/producer-consumer-topic") + .enableBatching(false) + .messageRoutingMode(MessageRoutingMode.SinglePartition) + .create()); + } + +} From 7e14c7a1606040d815f2982c404cb01e757d9f93 Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Tue, 12 Dec 2023 12:46:39 +0530 Subject: [PATCH 189/699] Fix checkstyle for ProxyConnection.java --- .../org/apache/pulsar/proxy/server/ProxyConnection.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) 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 d7e91f4530f57..64c04c3f217ed 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 @@ -398,10 +398,8 @@ private synchronized void completeConnect() throws PulsarClientException { state = State.ProxyLookupRequests; try { - lookupProxyHandler = - Reflections.createInstance(service.getConfiguration().getLookupHandler(), LookupProxyHandler.class, - Thread.currentThread() - .getContextClassLoader()); + lookupProxyHandler = Reflections.createInstance(service.getConfiguration().getLookupHandler(), + LookupProxyHandler.class, Thread.currentThread().getContextClassLoader()); lookupProxyHandler.initialize(service, this); } catch (Exception e) { LOG.error("Failed to initialize lookup proxy handler", e); From 8a84061ea32b4ed4830e19489cfa896da381a111 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Wed, 27 Dec 2023 11:05:49 +0530 Subject: [PATCH 190/699] Added unaccounted datastax jars in server/LICENSE. Enabled binary license check in ci-pulsar workflow. --- .github/workflows/pulsar-ci.yaml | 8 +++---- .../server/src/assemble/LICENSE.bin.txt | 24 +++++++++++++++++++ 2 files changed, 28 insertions(+), 4 deletions(-) diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index f4d258a5b4317..7af0f0991343c 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -142,8 +142,8 @@ jobs: run: | mvn -B -T 1C -ntp -Pcore-modules,-main clean install -DskipTests -Dlicense.skip=true -Drat.skip=true -Dcheckstyle.skip=true -# - name: Check binary licenses -# run: src/check-binary-license.sh ./distribution/server/target/apache-pulsar-*-bin.tar.gz + - name: Check binary licenses + run: src/check-binary-license.sh ./distribution/server/target/apache-pulsar-*-bin.tar.gz - name: Install gh-actions-artifact-client.js uses: apache/pulsar-test-infra/gh-actions-artifact-client/dist@master @@ -793,8 +793,8 @@ jobs: -Dspotbugs.skip=true -Dlicense.skip=true -Dcheckstyle.skip=true -Drat.skip=true # check full build artifacts licenses -# - name: Check binary licenses -# 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: Check binary licenses + 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: Clean up disk space run: | diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 287fe8e680338..3943ee8794a45 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -580,6 +580,30 @@ Bouncy Castle License - org.bouncycastle-bcprov-jdk18on-1.75.jar - org.bouncycastle-bcutil-jdk18on-1.75.jar +Datastax + - com.datastax.oss-managed-ledger-3.1.2.jar + - com.datastax.oss-pulsar-broker-3.1.2.jar + - com.datastax.oss-pulsar-broker-auth-oidc-3.1.2.jar + - com.datastax.oss-pulsar-broker-auth-sasl-3.1.2.jar + - com.datastax.oss-pulsar-broker-common-3.1.2.jar + - com.datastax.oss-pulsar-config-validation-3.1.2.jar + - com.datastax.oss-pulsar-docs-tools-3.1.2.jar + - com.datastax.oss-pulsar-functions-api-3.1.2.jar + - com.datastax.oss-pulsar-functions-instance-3.1.2.jar + - com.datastax.oss-pulsar-functions-local-runner-original-3.1.2.jar + - com.datastax.oss-pulsar-functions-proto-3.1.2.jar + - com.datastax.oss-pulsar-functions-runtime-3.1.2.jar + - com.datastax.oss-pulsar-functions-secrets-3.1.2.jar + - com.datastax.oss-pulsar-functions-utils-3.1.2.jar + - com.datastax.oss-pulsar-functions-worker-3.1.2.jar + - com.datastax.oss-pulsar-io-core-3.1.2.jar + - com.datastax.oss-pulsar-metadata-3.1.2.jar + - com.datastax.oss-pulsar-proxy-3.1.2.jar + - com.datastax.oss-pulsar-testclient-3.1.2.jar + - com.datastax.oss-pulsar-transaction-common-3.1.2.jar + - com.datastax.oss-pulsar-transaction-coordinator-3.1.2.jar + - com.datastax.oss-testmocks-3.1.2.jar + ------------------------ Additionaly, Netty includes code with the following licenses: From c2d3d3788e4967a2f40cb1fe3fd640b50534b0a5 Mon Sep 17 00:00:00 2001 From: lifepuzzlefun Date: Thu, 21 Dec 2023 18:24:08 +0800 Subject: [PATCH 191/699] [improve][broker] Avoid record inactiveproducers when deduplication is disable. (#21193) Co-authored-by: Jiwe Guo (cherry picked from commit 3b51b6c34f54707ea218da9fd222813b450ae8ed) --- .../persistent/MessageDeduplication.java | 21 ++++++++ .../persistent/MessageDuplicationTest.java | 49 ++++++++++++++++++- 2 files changed, 69 insertions(+), 1 deletion(-) 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 e75e22ff58cfe..c4319b0f80185 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 @@ -463,6 +463,10 @@ private boolean isDeduplicationEnabled() { * Topic will call this method whenever a producer connects. */ public void producerAdded(String producerName) { + if (!isEnabled()) { + return; + } + // Producer is no-longer inactive inactiveProducers.remove(producerName); } @@ -471,6 +475,10 @@ public void producerAdded(String producerName) { * Topic will call this method whenever a producer disconnects. */ public void producerRemoved(String producerName) { + if (!isEnabled()) { + return; + } + // Producer is no-longer active inactiveProducers.put(producerName, System.currentTimeMillis()); } @@ -482,6 +490,14 @@ public synchronized void purgeInactiveProducers() { long minimumActiveTimestamp = System.currentTimeMillis() - TimeUnit.MINUTES .toMillis(pulsar.getConfiguration().getBrokerDeduplicationProducerInactivityTimeoutMinutes()); + // if not enabled just clear all inactive producer record. + if (!isEnabled()) { + if (!inactiveProducers.isEmpty()) { + inactiveProducers.clear(); + } + return; + } + Iterator> mapIterator = inactiveProducers.entrySet().iterator(); boolean hasInactive = false; while (mapIterator.hasNext()) { @@ -534,5 +550,10 @@ ManagedCursor getManagedCursor() { return managedCursor; } + @VisibleForTesting + Map getInactiveProducers() { + return inactiveProducers; + } + private static final Logger log = LoggerFactory.getLogger(MessageDeduplication.class); } 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 a665681528114..1dcd477ca53f3 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 @@ -32,6 +32,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 io.netty.buffer.ByteBuf; import io.netty.channel.EventLoopGroup; import java.lang.reflect.Field; @@ -47,16 +48,23 @@ import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.broker.service.BacklogQuotaManager; 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.client.api.Producer; +import org.apache.pulsar.client.api.Schema; 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.apache.pulsar.compaction.CompactionServiceFactory; +import org.awaitility.Awaitility; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @Slf4j @Test(groups = "broker") -public class MessageDuplicationTest { +public class MessageDuplicationTest extends BrokerTestBase { private static final int BROKER_DEDUPLICATION_ENTRIES_INTERVAL = 10; private static final int BROKER_DEDUPLICATION_MAX_NUMBER_PRODUCERS = 10; @@ -440,4 +448,43 @@ public void completed(Exception e, long ledgerId, long entryId) { } }); } + + @BeforeMethod(alwaysRun = true) + @Override + protected void setup() throws Exception { + this.conf.setBrokerDeduplicationEnabled(true); + super.baseSetup(); + } + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test + public void testMessageDeduplication() throws Exception { + String topicName = "persistent://prop/ns-abc/testMessageDeduplication"; + String producerName = "test-producer"; + Producer producer = pulsarClient + .newProducer(Schema.STRING) + .producerName(producerName) + .topic(topicName) + .create(); + final PersistentTopic persistentTopic = (PersistentTopic) pulsar.getBrokerService() + .getTopicIfExists(topicName).get().orElse(null); + assertNotNull(persistentTopic); + final MessageDeduplication messageDeduplication = persistentTopic.getMessageDeduplication(); + assertFalse(messageDeduplication.getInactiveProducers().containsKey(producerName)); + producer.close(); + Awaitility.await().untilAsserted(() -> assertTrue(messageDeduplication.getInactiveProducers().containsKey(producerName))); + admin.topicPolicies().setDeduplicationStatus(topicName, false); + Awaitility.await().untilAsserted(() -> { + final Boolean deduplicationStatus = admin.topicPolicies().getDeduplicationStatus(topicName); + Assert.assertNotNull(deduplicationStatus); + Assert.assertFalse(deduplicationStatus); + }); + messageDeduplication.purgeInactiveProducers(); + assertTrue(messageDeduplication.getInactiveProducers().isEmpty()); + } } From 4b7f34b9b0cfac932db3afc0f6209aec32ef3246 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Thu, 14 Dec 2023 11:54:03 +0800 Subject: [PATCH 192/699] [fix][broker] Fixed ServiceUnitStateChannel monitor to tombstone only inactive bundle states (#21721) (cherry picked from commit 8d165803e948c770ff87d806fd54b94b7b6c8632) (cherry picked from commit 30fe5648862d78efe0a5620e2e9bba5d0f0761bd) --- .../extensions/channel/ServiceUnitStateChannelImpl.java | 2 +- .../extensions/channel/ServiceUnitStateChannelTest.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) 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 3cf16709cde1b..bd7e032a24a0c 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 @@ -1412,7 +1412,7 @@ protected void monitorOwnerships(List brokers) { continue; } - if (now - stateData.timestamp() > semiTerminalStateWaitingTimeInMillis) { + if (!isActiveState(state) && now - stateData.timestamp() > semiTerminalStateWaitingTimeInMillis) { log.info("Found semi-terminal states to tombstone" + " serviceUnit:{}, stateData:{}", serviceUnit, stateData); tombstoneAsync(serviceUnit).whenComplete((__, e) -> { 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 1da7059bf49ce..acf87ec750007 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 @@ -636,7 +636,7 @@ public void splitAndRetryTest() throws Exception { var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; validateMonitorCounters(leader, 0, - 3, + 1, 0, 0, 0, @@ -1406,7 +1406,7 @@ public void splitAndRetryFailureTest() throws Exception { validateMonitorCounters(leader, 0, - 3, + 1, 1, 0, 0, From fc612fdb0ed57f04af54bb4c58215c90989cf312 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Fri, 15 Dec 2023 12:34:59 -0800 Subject: [PATCH 193/699] [fix][broker] Skip topic auto-creation for ExtensibleLoadManager internal topics (#21729) (cherry picked from commit 88df040ed34e6863f2c255ace1b050030a3d54e7) (cherry picked from commit b403f3c28798657a9a60477ac3faded7747f9192) --- .../extensions/ExtensibleLoadManagerImpl.java | 2 +- .../pulsar/broker/service/BrokerService.java | 8 +-- .../BrokerServiceAutoTopicCreationTest.java | 63 +++++++++++++++++++ 3 files changed, 68 insertions(+), 5 deletions(-) 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 d3119365ddfea..37ca29da2601a 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 @@ -732,7 +732,7 @@ public void close() throws PulsarServerException { } } - private boolean isInternalTopic(String topic) { + public static boolean isInternalTopic(String topic) { return topic.startsWith(ServiceUnitStateChannelImpl.TOPIC) || topic.startsWith(BROKER_LOAD_DATA_STORE_TOPIC) || topic.startsWith(TOP_BUNDLES_LOAD_DATA_STORE_TOPIC); 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 382bca68dffc9..07e0f8856d700 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 @@ -103,7 +103,7 @@ import org.apache.pulsar.broker.intercept.BrokerInterceptor; import org.apache.pulsar.broker.intercept.ManagedLedgerInterceptorImpl; import org.apache.pulsar.broker.loadbalance.LoadManager; -import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; +import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.resources.DynamicConfigurationResources; import org.apache.pulsar.broker.resources.LocalPoliciesResources; @@ -3360,10 +3360,10 @@ private CompletableFuture isAllowAutoTopicCreationAsync(final TopicName return CompletableFuture.completedFuture(false); } - // ServiceUnitStateChannelImpl.TOPIC expects to be a non-partitioned-topic now. + // ExtensibleLoadManagerImpl.internal topics expects to be non-partitioned-topics now. // We don't allow the auto-creation here. - // ServiceUnitStateChannelImpl.start() is responsible to create the topic. - if (ServiceUnitStateChannelImpl.TOPIC.equals(topicName.toString())) { + // ExtensibleLoadManagerImpl.start() is responsible to create the internal system topics. + if (ExtensibleLoadManagerImpl.isInternalTopic(topicName.toString())) { return CompletableFuture.completedFuture(false); } 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 a28b60bbae354..0a6cffc7685d4 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 @@ -25,18 +25,27 @@ import java.util.List; +import java.util.Optional; +import java.util.Set; import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import lombok.Cleanup; +import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; import org.apache.pulsar.client.admin.ListNamespaceTopicsOptions; import org.apache.pulsar.client.admin.PulsarAdminException; 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.common.naming.NamespaceName; import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.policies.data.TopicType; +import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -526,4 +535,58 @@ public void testDynamicConfigurationTopicAutoCreationPartitionedWhenDefaultMoreT } } + @Test + public void testExtensibleLoadManagerImplInternalTopicAutoCreations() + throws PulsarAdminException, PulsarClientException { + pulsar.getConfiguration().setAllowAutoTopicCreation(true); + pulsar.getConfiguration().setAllowAutoTopicCreationType(TopicType.PARTITIONED); + pulsar.getConfiguration().setDefaultNumPartitions(3); + pulsar.getConfiguration().setMaxNumPartitionsPerPartitionedTopic(5); + final String namespaceName = NamespaceName.SYSTEM_NAMESPACE.toString(); + TenantInfoImpl tenantInfo = new TenantInfoImpl(); + tenantInfo.setAllowedClusters(Set.of(configClusterName)); + admin.tenants().createTenant("pulsar", tenantInfo); + admin.namespaces().createNamespace(namespaceName); + admin.topics().createNonPartitionedTopic(ServiceUnitStateChannelImpl.TOPIC); + admin.topics().createNonPartitionedTopic(ExtensibleLoadManagerImpl.BROKER_LOAD_DATA_STORE_TOPIC); + admin.topics().createNonPartitionedTopic(ExtensibleLoadManagerImpl.TOP_BUNDLES_LOAD_DATA_STORE_TOPIC); + + // clear the topics to test the auto creation of non-persistent topics. + ConcurrentOpenHashMap>> topics = + pulsar.getBrokerService().getTopics(); + ConcurrentOpenHashMap>> oldTopics = new ConcurrentOpenHashMap<>(); + topics.forEach((key, val) -> oldTopics.put(key, val)); + topics.clear(); + + // The created persistent topic correctly can be found by + // pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); + Producer producer = pulsarClient.newProducer().topic(ServiceUnitStateChannelImpl.TOPIC).create(); + + // The created non-persistent topics cannot be found, as we did topics.clear() + try { + pulsarClient.newProducer().topic(ExtensibleLoadManagerImpl.BROKER_LOAD_DATA_STORE_TOPIC).create(); + Assert.fail("Create should have failed."); + } catch (PulsarClientException.TopicDoesNotExistException e) { + // expected + } + try { + pulsarClient.newProducer().topic(ExtensibleLoadManagerImpl.TOP_BUNDLES_LOAD_DATA_STORE_TOPIC).create(); + Assert.fail("Create should have failed."); + } catch (PulsarClientException.TopicDoesNotExistException e) { + // expected + } + + oldTopics.forEach((key, val) -> topics.put(key, val)); + + Awaitility.await().atMost(5, TimeUnit.SECONDS).untilAsserted(() -> { + List partitionedTopicList = admin.topics().getPartitionedTopicList(namespaceName); + assertEquals(partitionedTopicList.size(), 0); + }); + + producer.close(); + admin.namespaces().deleteNamespace(namespaceName); + admin.tenants().deleteTenant("pulsar"); + + } + } From ed56e74b0ac97b7d72f4304cfb698a654071ec6d Mon Sep 17 00:00:00 2001 From: crossoverJie Date: Thu, 14 Dec 2023 21:45:37 +0800 Subject: [PATCH 194/699] [fix][broker] Fix the issue of topics possibly being deleted. (#21704) Co-authored-by: Jiwe Guo (cherry picked from commit 84ea1ca05decbcb5d3a3bd1812e53ad10773b259) (cherry picked from commit 49490b382fa12295655804abd101dc07134b7449) --- .../service/persistent/PersistentTopic.java | 9 +- .../PersistentTopicInitializeDelayTest.java | 142 ++++++++++++++++++ 2 files changed, 149 insertions(+), 2 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicInitializeDelayTest.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 d0de36c624211..4409114e013dc 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 @@ -296,8 +296,6 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS .build(); this.backloggedCursorThresholdEntries = brokerService.pulsar().getConfiguration().getManagedLedgerCursorBackloggedThreshold(); - registerTopicPolicyListener(); - this.messageDeduplication = new MessageDeduplication(brokerService.pulsar(), this, ledger); if (ledger.getProperties().containsKey(TOPIC_EPOCH_PROPERTY_NAME)) { topicEpoch = Optional.of(Long.parseLong(ledger.getProperties().get(TOPIC_EPOCH_PROPERTY_NAME))); @@ -1594,6 +1592,11 @@ public CompletableFuture checkReplication() { } List configuredClusters = topicPolicies.getReplicationClusters().get(); + if (CollectionUtils.isEmpty(configuredClusters)) { + log.warn("[{}] No replication clusters configured", name); + return CompletableFuture.completedFuture(null); + } + int newMessageTTLInSeconds = topicPolicies.getMessageTTLInSeconds().get(); String localCluster = brokerService.pulsar().getConfiguration().getClusterName(); @@ -3534,6 +3537,8 @@ private CompletableFuture updateSubscriptionsDispatcherRateLimiter() { protected CompletableFuture initTopicPolicy() { if (brokerService.pulsar().getConfig().isSystemTopicEnabled() && brokerService.pulsar().getConfig().isTopicLevelPoliciesEnabled()) { + brokerService.getPulsar().getTopicPoliciesService() + .registerListener(TopicName.getPartitionedTopicName(topic), this); return CompletableFuture.completedFuture(null).thenRunAsync(() -> onUpdate( brokerService.getPulsar().getTopicPoliciesService() .getTopicPoliciesIfExists(TopicName.getPartitionedTopicName(topic))), diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicInitializeDelayTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicInitializeDelayTest.java new file mode 100644 index 0000000000000..ab8d4dbe5cc01 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PersistentTopicInitializeDelayTest.java @@ -0,0 +1,142 @@ +/* + * 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 lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.pulsar.broker.service.nonpersistent.NonPersistentTopic; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.Policies; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.awaitility.Awaitility; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; +import java.io.IOException; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + +@Test(groups = "broker") +@Slf4j +public class PersistentTopicInitializeDelayTest extends BrokerTestBase { + + @BeforeMethod + @Override + protected void setup() throws Exception { + conf.setTopicFactoryClassName(MyTopicFactory.class.getName()); + conf.setAllowAutoTopicCreation(true); + conf.setManagedLedgerMaxEntriesPerLedger(1); + conf.setBrokerDeleteInactiveTopicsEnabled(false); + conf.setTransactionCoordinatorEnabled(false); + conf.setTopicLoadTimeoutSeconds(30); + super.baseSetup(); + } + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test(timeOut = 30 * 1000) + public void testTopicInitializeDelay() throws Exception { + admin.tenants().createTenant("public", TenantInfo.builder().allowedClusters(Set.of(configClusterName)).build()); + String namespace = "public/initialize-delay"; + admin.namespaces().createNamespace(namespace); + final String topicName = "persistent://" + namespace + "/testTopicInitializeDelay"; + admin.topics().createNonPartitionedTopic(topicName); + + admin.topicPolicies().setMaxConsumers(topicName, 10); + Awaitility.await().untilAsserted(() -> assertEquals(admin.topicPolicies().getMaxConsumers(topicName), 10)); + admin.topics().unload(topicName); + CompletableFuture> optionalFuture = pulsar.getBrokerService().getTopic(topicName, true); + + Optional topic = optionalFuture.get(15, TimeUnit.SECONDS); + assertTrue(topic.isPresent()); + } + + 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 { + + private static AtomicInteger checkReplicationInvocationCount = new AtomicInteger(0); + + public MyPersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerService) { + super(topic, ledger, brokerService); + SystemTopicBasedTopicPoliciesService topicPoliciesService = + (SystemTopicBasedTopicPoliciesService) brokerService.getPulsar().getTopicPoliciesService(); + if (topicPoliciesService.getListeners().containsKey(TopicName.get(topic)) ) { + this.onUpdate(brokerService.getPulsar().getTopicPoliciesService().getTopicPoliciesIfExists(TopicName.get(topic))); + } + } + + protected void updateTopicPolicyByNamespacePolicy(Policies namespacePolicies) { + try { + Thread.sleep(10 * 1000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + super.updateTopicPolicyByNamespacePolicy(namespacePolicies); + } + + public CompletableFuture checkReplication() { + if (TopicName.get(topic).getLocalName().equalsIgnoreCase("testTopicInitializeDelay")) { + checkReplicationInvocationCount.incrementAndGet(); + log.info("checkReplication, count = {}", checkReplicationInvocationCount.get()); + List configuredClusters = topicPolicies.getReplicationClusters().get(); + if (!(configuredClusters.size() == 1 && configuredClusters.contains(brokerService.pulsar().getConfiguration().getClusterName()))) { + try { + // this will cause the get topic timeout. + Thread.sleep(8 * 1000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + throw new RuntimeException("checkReplication error"); + } + } + return super.checkReplication(); + } + } +} From e78533ae089fb56aec48181292e1036a93472419 Mon Sep 17 00:00:00 2001 From: wenbingshen Date: Tue, 26 Dec 2023 19:10:59 +0800 Subject: [PATCH 195/699] [fix][client] Fix producer thread block forever on memory limit controller (#21790) (cherry picked from commit 99d06b94fa715b3f1062c4a3f616d5cc725e47a4) (cherry picked from commit 04ed338ee1ced32442bd3d7a238abb235eae4b8f) --- .../client/impl/ProducerMemoryLimitTest.java | 42 +++++++++++++++++-- .../impl/BatchMessageContainerImpl.java | 6 ++- 2 files changed, 43 insertions(+), 5 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerMemoryLimitTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerMemoryLimitTest.java index 3ec784e248cba..d776fdb0ed915 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerMemoryLimitTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ProducerMemoryLimitTest.java @@ -23,7 +23,12 @@ import static org.mockito.Mockito.mock; import io.netty.buffer.ByteBufAllocator; import java.lang.reflect.Field; +import java.nio.charset.StandardCharsets; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import lombok.Cleanup; +import org.apache.pulsar.client.api.CompressionType; +import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; @@ -35,9 +40,6 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import java.nio.charset.StandardCharsets; -import java.util.concurrent.TimeUnit; - @Test(groups = "broker-impl") public class ProducerMemoryLimitTest extends ProducerConsumerBase { @@ -191,6 +193,40 @@ public void testProducerCloseMemoryRelease() throws Exception { Assert.assertEquals(memoryLimitController.currentUsage(), 0); } + @Test(timeOut = 10_000) + public void testProducerBlockReserveMemory() throws Exception { + replacePulsarClient(PulsarClient.builder(). + serviceUrl(lookupUrl.toString()) + .memoryLimit(1, SizeUnit.KILO_BYTES)); + @Cleanup + ProducerImpl producer = (ProducerImpl) pulsarClient.newProducer() + .topic("testProducerMemoryLimit") + .sendTimeout(5, TimeUnit.SECONDS) + .compressionType(CompressionType.SNAPPY) + .messageRoutingMode(MessageRoutingMode.RoundRobinPartition) + .maxPendingMessages(0) + .blockIfQueueFull(true) + .enableBatching(true) + .batchingMaxMessages(100) + .batchingMaxBytes(65536) + .batchingMaxPublishDelay(100, TimeUnit.MILLISECONDS) + .create(); + int msgCount = 5; + CountDownLatch cdl = new CountDownLatch(msgCount); + for (int i = 0; i < msgCount; i++) { + producer.sendAsync("memory-test".getBytes(StandardCharsets.UTF_8)).whenComplete(((messageId, throwable) -> { + cdl.countDown(); + })); + } + + cdl.await(); + + producer.close(); + PulsarClientImpl clientImpl = (PulsarClientImpl) this.pulsarClient; + final MemoryLimitController memoryLimitController = clientImpl.getMemoryLimitController(); + Assert.assertEquals(memoryLimitController.currentUsage(), 0); + } + private void initClientWithMemoryLimit() throws PulsarClientException { replacePulsarClient(PulsarClient.builder(). serviceUrl(lookupUrl.toString()) 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 9be7210a38742..dfcbc42bcc6b8 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 @@ -324,9 +324,11 @@ public OpSendMsg createOpSendMsg() throws IOException { protected void updateAndReserveBatchAllocatedSize(int updatedSizeBytes) { int delta = updatedSizeBytes - batchAllocatedSizeBytes; batchAllocatedSizeBytes = updatedSizeBytes; - if (delta != 0) { - if (producer != null) { + if (producer != null) { + if (delta > 0) { producer.client.getMemoryLimitController().forceReserveMemory(delta); + } else if (delta < 0) { + producer.client.getMemoryLimitController().releaseMemory(-delta); } } } From c7aa1396ddffc6aab33f3f6dcbc57ce50ac8e213 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Wed, 27 Dec 2023 07:47:36 -0800 Subject: [PATCH 196/699] [fix][broker] Fixed the ExtensibleLoadManagerImpl internal system getTopic failure when the leadership changes #21764 (#21801) (cherry picked from commit 39b69a3cb08265542d631bc54a9b66c3ec579df4) --- .../extensions/ExtensibleLoadManagerImpl.java | 116 +++++++++--------- .../extensions/store/LoadDataStore.java | 17 +++ .../store/TableViewLoadDataStoreImpl.java | 30 ++++- .../ExtensibleLoadManagerImplTest.java | 16 +-- .../filter/BrokerFilterTestBase.java | 15 +++ .../scheduler/TransferShedderTest.java | 30 +++++ .../extensions/store/LoadDataStoreTest.java | 3 + .../LeastResourceUsageWithWeightTest.java | 15 +++ .../ExtensibleLoadManagerTest.java | 47 ++++--- 9 files changed, 206 insertions(+), 83 deletions(-) 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 37ca29da2601a..f717286fe5d24 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 @@ -279,13 +279,18 @@ public static void createSystemTopic(PulsarService pulsar, String topic) throws log.info("Created topic {}.", topic); } catch (PulsarAdminException.ConflictException ex) { if (debug(pulsar.getConfiguration(), log)) { - log.info("Topic {} already exists.", topic, ex); + log.info("Topic {} already exists.", topic); } } catch (PulsarAdminException e) { throw new PulsarServerException(e); } } + private static void createSystemTopics(PulsarService pulsar) throws PulsarServerException { + createSystemTopic(pulsar, BROKER_LOAD_DATA_STORE_TOPIC); + createSystemTopic(pulsar, TOP_BUNDLES_LOAD_DATA_STORE_TOPIC); + } + @Override public void start() throws PulsarServerException { if (this.started) { @@ -321,13 +326,9 @@ public void start() throws PulsarServerException { this.isolationPoliciesHelper = new IsolationPoliciesHelper(policies); this.brokerFilterPipeline.add(new BrokerIsolationPoliciesFilter(isolationPoliciesHelper)); - createSystemTopic(pulsar, BROKER_LOAD_DATA_STORE_TOPIC); - createSystemTopic(pulsar, TOP_BUNDLES_LOAD_DATA_STORE_TOPIC); - try { this.brokerLoadDataStore = LoadDataStoreFactory .create(pulsar.getClient(), BROKER_LOAD_DATA_STORE_TOPIC, BrokerLoadData.class); - this.brokerLoadDataStore.startTableView(); this.topBundlesLoadDataStore = LoadDataStoreFactory .create(pulsar.getClient(), TOP_BUNDLES_LOAD_DATA_STORE_TOPIC, TopBundlesLoadData.class); } catch (LoadDataStoreException e) { @@ -382,7 +383,6 @@ public void start() throws PulsarServerException { this.unloadScheduler = new UnloadScheduler( pulsar, pulsar.getLoadManagerExecutor(), unloadManager, context, serviceUnitStateChannel, unloadCounter, unloadMetrics); - this.unloadScheduler.start(); this.splitScheduler = new SplitScheduler( pulsar, serviceUnitStateChannel, splitManager, splitCounter, splitMetrics, context); this.splitScheduler.start(); @@ -740,74 +740,74 @@ public static boolean isInternalTopic(String topic) { @VisibleForTesting void playLeader() { - if (role != Leader) { - log.info("This broker:{} is changing the role from {} to {}", - pulsar.getLookupServiceAddress(), role, Leader); - int retry = 0; - while (true) { + log.info("This broker:{} is setting the role from {} to {}", + pulsar.getLookupServiceAddress(), role, Leader); + int retry = 0; + while (!Thread.currentThread().isInterrupted()) { + try { + initWaiter.await(); + // 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). + // Otherwise, the system topic existence check will fail on the leader broker. + createSystemTopics(pulsar); + brokerLoadDataStore.init(); + topBundlesLoadDataStore.init(); + unloadScheduler.start(); + serviceUnitStateChannel.scheduleOwnershipMonitor(); + break; + } catch (Throwable e) { + log.error("The broker:{} failed to set the role. Retrying {} th ...", + pulsar.getLookupServiceAddress(), ++retry, e); try { - initWaiter.await(); - serviceUnitStateChannel.scheduleOwnershipMonitor(); - topBundlesLoadDataStore.startTableView(); - unloadScheduler.start(); - break; - } catch (Throwable e) { - log.error("The broker:{} failed to change the role. Retrying {} th ...", - pulsar.getLookupServiceAddress(), ++retry, e); - try { - Thread.sleep(Math.min(retry * 10, MAX_ROLE_CHANGE_RETRY_DELAY_IN_MILLIS)); - } catch (InterruptedException ex) { - log.warn("Interrupted while sleeping."); - } + Thread.sleep(Math.min(retry * 10, MAX_ROLE_CHANGE_RETRY_DELAY_IN_MILLIS)); + } catch (InterruptedException ex) { + log.warn("Interrupted while sleeping."); + // preserve thread's interrupt status + Thread.currentThread().interrupt(); } } - role = Leader; - log.info("This broker:{} plays the leader now.", pulsar.getLookupServiceAddress()); } + role = Leader; + log.info("This broker:{} plays the leader now.", pulsar.getLookupServiceAddress()); // flush the load data when the leader is elected. - if (brokerLoadDataReporter != null) { - brokerLoadDataReporter.reportAsync(true); - } - if (topBundleLoadDataReporter != null) { - topBundleLoadDataReporter.reportAsync(true); - } + brokerLoadDataReporter.reportAsync(true); + topBundleLoadDataReporter.reportAsync(true); } @VisibleForTesting void playFollower() { - if (role != Follower) { - log.info("This broker:{} is changing the role from {} to {}", - pulsar.getLookupServiceAddress(), role, Follower); - int retry = 0; - while (true) { + log.info("This broker:{} is setting the role from {} to {}", + pulsar.getLookupServiceAddress(), role, Follower); + int retry = 0; + while (!Thread.currentThread().isInterrupted()) { + try { + initWaiter.await(); + unloadScheduler.close(); + serviceUnitStateChannel.cancelOwnershipMonitor(); + brokerLoadDataStore.init(); + topBundlesLoadDataStore.close(); + topBundlesLoadDataStore.startProducer(); + break; + } catch (Throwable e) { + log.error("The broker:{} failed to set the role. Retrying {} th ...", + pulsar.getLookupServiceAddress(), ++retry, e); try { - initWaiter.await(); - serviceUnitStateChannel.cancelOwnershipMonitor(); - topBundlesLoadDataStore.closeTableView(); - unloadScheduler.close(); - break; - } catch (Throwable e) { - log.error("The broker:{} failed to change the role. Retrying {} th ...", - pulsar.getLookupServiceAddress(), ++retry, e); - try { - Thread.sleep(Math.min(retry * 10, MAX_ROLE_CHANGE_RETRY_DELAY_IN_MILLIS)); - } catch (InterruptedException ex) { - log.warn("Interrupted while sleeping."); - } + Thread.sleep(Math.min(retry * 10, MAX_ROLE_CHANGE_RETRY_DELAY_IN_MILLIS)); + } catch (InterruptedException ex) { + log.warn("Interrupted while sleeping."); + // preserve thread's interrupt status + Thread.currentThread().interrupt(); } } - role = Follower; - log.info("This broker:{} plays a follower now.", pulsar.getLookupServiceAddress()); } + role = Follower; + log.info("This broker:{} plays a follower now.", pulsar.getLookupServiceAddress()); // flush the load data when the leader is elected. - if (brokerLoadDataReporter != null) { - brokerLoadDataReporter.reportAsync(true); - } - if (topBundleLoadDataReporter != null) { - topBundleLoadDataReporter.reportAsync(true); - } + brokerLoadDataReporter.reportAsync(true); + topBundleLoadDataReporter.reportAsync(true); } public List getMetrics() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java index 680a36523a214..a7deeeaad8a5c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStore.java @@ -81,9 +81,26 @@ public interface LoadDataStore extends Closeable { */ void closeTableView() throws IOException; + + /** + * Starts the data store (both producer and table view). + */ + void start() throws LoadDataStoreException; + + /** + * Inits the data store (close and start the data store). + */ + void init() throws IOException; + /** * Starts the table view. */ void startTableView() throws LoadDataStoreException; + + /** + * Starts the producer. + */ + void startProducer() throws LoadDataStoreException; + } 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 a400163ebf122..ead0a7081fd37 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 @@ -37,9 +37,9 @@ */ public class TableViewLoadDataStoreImpl implements LoadDataStore { - private TableView tableView; + private volatile TableView tableView; - private final Producer producer; + private volatile Producer producer; private final PulsarClient client; @@ -50,7 +50,6 @@ public class TableViewLoadDataStoreImpl implements LoadDataStore { public TableViewLoadDataStoreImpl(PulsarClient client, String topic, Class clazz) throws LoadDataStoreException { try { this.client = client; - this.producer = client.newProducer(Schema.JSON(clazz)).topic(topic).create(); this.topic = topic; this.clazz = clazz; } catch (Exception e) { @@ -99,6 +98,12 @@ public void closeTableView() throws IOException { } } + @Override + public void start() throws LoadDataStoreException { + startProducer(); + startTableView(); + } + @Override public void startTableView() throws LoadDataStoreException { if (tableView == null) { @@ -111,14 +116,33 @@ public void startTableView() throws LoadDataStoreException { } } + @Override + public void startProducer() throws LoadDataStoreException { + if (producer == null) { + try { + producer = client.newProducer(Schema.JSON(clazz)).topic(topic).create(); + } catch (PulsarClientException e) { + producer = null; + throw new LoadDataStoreException(e); + } + } + } + @Override public void close() throws IOException { if (producer != null) { producer.close(); + producer = null; } closeTableView(); } + @Override + public void init() throws IOException { + close(); + start(); + } + private void validateTableViewStart() { if (tableView == null) { throw new IllegalStateException("table view has not been started"); 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 20ba9500cb1fd..545601ea59c6b 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 @@ -821,12 +821,12 @@ public void testRoleChange() throws Exception { FieldUtils.writeDeclaredField(secondaryLoadManager, "topBundlesLoadDataStore", topBundlesLoadDataStoreSecondarySpy, true); if (channel1.isChannelOwnerAsync().get(5, TimeUnit.SECONDS)) { - primaryLoadManager.playFollower(); - primaryLoadManager.playFollower(); + primaryLoadManager.playFollower(); // close 3 times + primaryLoadManager.playFollower(); // close 1 time secondaryLoadManager.playLeader(); secondaryLoadManager.playLeader(); - primaryLoadManager.playLeader(); - primaryLoadManager.playLeader(); + primaryLoadManager.playLeader(); // close 3 times and open 3 times + primaryLoadManager.playLeader(); // close 1 time and open 1 time, secondaryLoadManager.playFollower(); secondaryLoadManager.playFollower(); } else { @@ -841,10 +841,10 @@ public void testRoleChange() throws Exception { } - verify(topBundlesLoadDataStorePrimarySpy, times(3)).startTableView(); - verify(topBundlesLoadDataStorePrimarySpy, times(3)).closeTableView(); - verify(topBundlesLoadDataStoreSecondarySpy, times(3)).startTableView(); - verify(topBundlesLoadDataStoreSecondarySpy, times(3)).closeTableView(); + verify(topBundlesLoadDataStorePrimarySpy, times(4)).startTableView(); + verify(topBundlesLoadDataStorePrimarySpy, times(8)).closeTableView(); + verify(topBundlesLoadDataStoreSecondarySpy, times(4)).startTableView(); + verify(topBundlesLoadDataStoreSecondarySpy, times(8)).closeTableView(); FieldUtils.writeDeclaredField(primaryLoadManager, "topBundlesLoadDataStore", topBundlesLoadDataStorePrimary, true); FieldUtils.writeDeclaredField(secondaryLoadManager, "topBundlesLoadDataStore", topBundlesLoadDataStoreSecondary, true); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerFilterTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerFilterTestBase.java index 68bd7b29094cd..a120ef473e9a5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerFilterTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/filter/BrokerFilterTestBase.java @@ -90,10 +90,25 @@ public void closeTableView() throws IOException { } + @Override + public void start() throws LoadDataStoreException { + + } + + @Override + public void init() throws IOException { + + } + @Override public void startTableView() throws LoadDataStoreException { } + + @Override + public void startProducer() throws LoadDataStoreException { + + } }; configuration.setPreferLaterVersions(true); doReturn(configuration).when(mockContext).brokerConfiguration(); 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 26d95a0158d52..4eec612477758 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 @@ -383,10 +383,25 @@ public void closeTableView() throws IOException { } + @Override + public void start() throws LoadDataStoreException { + + } + + @Override + public void init() throws IOException { + + } + @Override public void startTableView() throws LoadDataStoreException { } + + @Override + public void startProducer() throws LoadDataStoreException { + + } }; var topBundleLoadDataStore = new LoadDataStore() { @@ -436,10 +451,25 @@ public void closeTableView() throws IOException { } + @Override + public void start() throws LoadDataStoreException { + + } + + @Override + public void init() throws IOException { + + } + @Override public void startTableView() throws LoadDataStoreException { } + + @Override + public void startProducer() throws LoadDataStoreException { + + } }; BrokerRegistry brokerRegistry = mock(BrokerRegistry.class); 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 184c337a47c80..7431b9815f93f 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 @@ -75,6 +75,7 @@ public void testPushGetAndRemove() throws Exception { @Cleanup LoadDataStore loadDataStore = LoadDataStoreFactory.create(pulsar.getClient(), topic, MyClass.class); + loadDataStore.startProducer(); loadDataStore.startTableView(); MyClass myClass1 = new MyClass("1", 1); loadDataStore.pushAsync("key1", myClass1).get(); @@ -108,6 +109,7 @@ public void testForEach() throws Exception { @Cleanup LoadDataStore loadDataStore = LoadDataStoreFactory.create(pulsar.getClient(), topic, Integer.class); + loadDataStore.startProducer(); loadDataStore.startTableView(); Map map = new HashMap<>(); @@ -132,6 +134,7 @@ public void testTableViewRestart() throws Exception { String topic = TopicDomain.persistent + "://" + NamespaceName.SYSTEM_NAMESPACE + "/" + UUID.randomUUID(); LoadDataStore loadDataStore = LoadDataStoreFactory.create(pulsar.getClient(), topic, Integer.class); + loadDataStore.startProducer(); loadDataStore.startTableView(); loadDataStore.pushAsync("1", 1).get(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeightTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeightTest.java index 0eea1d87513bf..b1e09bf2f3afb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeightTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/strategy/LeastResourceUsageWithWeightTest.java @@ -252,10 +252,25 @@ public void closeTableView() throws IOException { } + @Override + public void start() throws LoadDataStoreException { + + } + + @Override + public void init() throws IOException { + + } + @Override public void startTableView() throws LoadDataStoreException { } + + @Override + public void startProducer() throws LoadDataStoreException { + + } }; doReturn(conf).when(ctx).brokerConfiguration(); diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java index 49e5ae378342d..23abf50bdb063 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java @@ -38,6 +38,7 @@ import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; import java.util.regex.Matcher; import java.util.regex.Pattern; import lombok.extern.slf4j.Slf4j; @@ -53,6 +54,7 @@ import org.apache.pulsar.tests.integration.containers.BrokerContainer; import org.apache.pulsar.tests.integration.topologies.PulsarCluster; import org.apache.pulsar.tests.integration.topologies.PulsarClusterSpec; +import org.awaitility.Awaitility; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; @@ -309,23 +311,40 @@ public void testIsolationPolicy() throws PulsarAdminException { parameters1.put("min_limit", "1"); parameters1.put("usage_threshold", "100"); - List activeBrokers = admin.brokers().getActiveBrokers(); + Awaitility.await().atMost(10, TimeUnit.SECONDS).untilAsserted( + () -> { + List activeBrokers = admin.brokers().getActiveBrokers(); + assertEquals(activeBrokers.size(), NUM_BROKERS); + } + ); + try { + admin.namespaces().createNamespace(isolationEnabledNameSpace); + } catch (PulsarAdminException.ConflictException e) { + //expected when retried + } - assertEquals(activeBrokers.size(), NUM_BROKERS); + try { + admin.clusters() + .createNamespaceIsolationPolicy(clusterName, namespaceIsolationPolicyName, NamespaceIsolationData + .builder() + .namespaces(List.of(isolationEnabledNameSpace)) + .autoFailoverPolicy(AutoFailoverPolicyData.builder() + .policyType(AutoFailoverPolicyType.min_available) + .parameters(parameters1) + .build()) + .primary(List.of(getHostName(0))) + .secondary(List.of(getHostName(1))) + .build()); + } catch (PulsarAdminException.ConflictException e) { + //expected when retried + } - admin.namespaces().createNamespace(isolationEnabledNameSpace); - admin.clusters().createNamespaceIsolationPolicy(clusterName, namespaceIsolationPolicyName, NamespaceIsolationData - .builder() - .namespaces(List.of(isolationEnabledNameSpace)) - .autoFailoverPolicy(AutoFailoverPolicyData.builder() - .policyType(AutoFailoverPolicyType.min_available) - .parameters(parameters1) - .build()) - .primary(List.of(getHostName(0))) - .secondary(List.of(getHostName(1))) - .build()); final String topic = "persistent://" + isolationEnabledNameSpace + "/topic"; - admin.topics().createNonPartitionedTopic(topic); + try { + admin.topics().createNonPartitionedTopic(topic); + } catch (PulsarAdminException.ConflictException e) { + //expected when retried + } String broker = admin.lookups().lookupTopic(topic); From f16f13345a6d263abe83bd035b93f4787b8803ff Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 2 Jan 2024 19:17:16 +0200 Subject: [PATCH 197/699] Upgrade OWASP dependency check maven plugin version (cherry picked from commit b0e8b5abb5777c175812f3ad709d4481a3a21f40) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 9eb54682d8cd3..494584c2ee193 100644 --- a/pom.xml +++ b/pom.xml @@ -296,7 +296,7 @@ flexible messaging model and an intuitive client API. 0.1.4 1.3 0.4 - 8.2.1 + 9.0.7 0.9.44 1.6.1 6.4.0 From 8e5f56a02cf032c2ebf63bbb15c670b37909a811 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 20 Dec 2023 04:08:18 +0200 Subject: [PATCH 198/699] [improve][proxy] Fix comment about enableProxyStatsEndpoints (#21757) (cherry picked from commit fea34d9d41b722cb72fec90513426ba64cc43164) --- conf/proxy.conf | 2 +- .../java/org/apache/pulsar/proxy/server/ProxyConfiguration.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/conf/proxy.conf b/conf/proxy.conf index c41c54670eea4..4194bf7621985 100644 --- a/conf/proxy.conf +++ b/conf/proxy.conf @@ -370,7 +370,7 @@ zooKeeperCacheExpirySeconds=-1 ### --- Metrics --- ### -# Whether to enable the proxy's /metrics, /proxy-stats, and /status.html http endpoints +# Whether to enable the proxy's /metrics and /proxy-stats http endpoints enableProxyStatsEndpoints=true # Whether the '/metrics' endpoint requires authentication. Defaults to true authenticateMetricsEndpoint=true 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 b9564eba8239b..148eb579ed196 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 @@ -379,7 +379,7 @@ public class ProxyConfiguration implements PulsarConfiguration { @FieldContext( category = CATEGORY_HTTP, - doc = "Whether to enable the proxy's /metrics, /proxy-stats, and /status.html http endpoints" + doc = "Whether to enable the proxy's /metrics and /proxy-stats http endpoints" ) private boolean enableProxyStatsEndpoints = true; From 8d3bea25877ab26b04ad6dd5fc613837ad4a00a8 Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Mon, 25 Dec 2023 21:22:03 +0800 Subject: [PATCH 199/699] [fix][broker] Fix TableViewLoadDataStoreImpl NPE (#21777) (cherry picked from commit 174e76b05355f146d2e5e63b23b60d548024c533) --- .../store/TableViewLoadDataStoreImpl.java | 34 +++++++++++-------- .../extensions/store/LoadDataStoreTest.java | 30 ++++++++++++++++ 2 files changed, 50 insertions(+), 14 deletions(-) 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 ead0a7081fd37..56afbef04565c 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 @@ -29,6 +29,7 @@ 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.util.FutureUtil; /** * The load data store, base on {@link TableView }. @@ -58,40 +59,46 @@ public TableViewLoadDataStoreImpl(PulsarClient client, String topic, Class cl } @Override - public CompletableFuture pushAsync(String key, T loadData) { + public synchronized CompletableFuture pushAsync(String key, T loadData) { + if (producer == null) { + return FutureUtil.failedFuture(new IllegalStateException("producer has not been started")); + } return producer.newMessage().key(key).value(loadData).sendAsync().thenAccept(__ -> {}); } @Override - public CompletableFuture removeAsync(String key) { + public synchronized CompletableFuture removeAsync(String key) { + if (producer == null) { + return FutureUtil.failedFuture(new IllegalStateException("producer has not been started")); + } return producer.newMessage().key(key).value(null).sendAsync().thenAccept(__ -> {}); } @Override - public Optional get(String key) { + public synchronized Optional get(String key) { validateTableViewStart(); return Optional.ofNullable(tableView.get(key)); } @Override - public void forEach(BiConsumer action) { + public synchronized void forEach(BiConsumer action) { validateTableViewStart(); tableView.forEach(action); } - public Set> entrySet() { + public synchronized Set> entrySet() { validateTableViewStart(); return tableView.entrySet(); } @Override - public int size() { + public synchronized int size() { validateTableViewStart(); return tableView.size(); } @Override - public void closeTableView() throws IOException { + public synchronized void closeTableView() throws IOException { if (tableView != null) { tableView.close(); tableView = null; @@ -99,13 +106,13 @@ public void closeTableView() throws IOException { } @Override - public void start() throws LoadDataStoreException { + public synchronized void start() throws LoadDataStoreException { startProducer(); startTableView(); } @Override - public void startTableView() throws LoadDataStoreException { + public synchronized void startTableView() throws LoadDataStoreException { if (tableView == null) { try { tableView = client.newTableViewBuilder(Schema.JSON(clazz)).topic(topic).create(); @@ -117,7 +124,7 @@ public void startTableView() throws LoadDataStoreException { } @Override - public void startProducer() throws LoadDataStoreException { + public synchronized void startProducer() throws LoadDataStoreException { if (producer == null) { try { producer = client.newProducer(Schema.JSON(clazz)).topic(topic).create(); @@ -129,7 +136,7 @@ public void startProducer() throws LoadDataStoreException { } @Override - public void close() throws IOException { + public synchronized void close() throws IOException { if (producer != null) { producer.close(); producer = null; @@ -138,15 +145,14 @@ public void close() throws IOException { } @Override - public void init() throws IOException { + public synchronized void init() throws IOException { close(); start(); } - private void validateTableViewStart() { + private synchronized void validateTableViewStart() { if (tableView == null) { throw new IllegalStateException("table view has not been started"); } } - } 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 7431b9815f93f..f486370400c92 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 @@ -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.fail; import static org.testng.AssertJUnit.assertTrue; import com.google.common.collect.Sets; @@ -39,6 +40,7 @@ import java.util.HashMap; import java.util.Map; import java.util.UUID; +import java.util.concurrent.ExecutionException; @Test(groups = "broker") public class LoadDataStoreTest extends MockedPulsarServiceBaseTest { @@ -154,4 +156,32 @@ public void testTableViewRestart() throws Exception { Awaitility.await().untilAsserted(() -> assertEquals(loadDataStore.get("1").get(), 2)); } + @Test + public void testProducerStop() throws Exception { + String topic = TopicDomain.persistent + "://" + NamespaceName.SYSTEM_NAMESPACE + "/" + UUID.randomUUID(); + LoadDataStore loadDataStore = + LoadDataStoreFactory.create(pulsar.getClient(), topic, Integer.class); + loadDataStore.startProducer(); + loadDataStore.pushAsync("1", 1).get(); + loadDataStore.removeAsync("1").get(); + + loadDataStore.close(); + + try { + loadDataStore.pushAsync("2", 2).get(); + fail(); + } catch (ExecutionException ex) { + assertTrue(ex.getCause() instanceof IllegalStateException); + } + try { + loadDataStore.removeAsync("2").get(); + fail(); + } catch (ExecutionException ex) { + assertTrue(ex.getCause() instanceof IllegalStateException); + } + loadDataStore.startProducer(); + loadDataStore.pushAsync("3", 3).get(); + loadDataStore.removeAsync("3").get(); + } + } From 2776e550d0aea0bee2a271cb7ada5f59ba21f97c Mon Sep 17 00:00:00 2001 From: ken <1647023764@qq.com> Date: Wed, 13 Dec 2023 22:19:02 +0800 Subject: [PATCH 200/699] [improve][broker] cleanup the empty subscriptionAuthenticationMap in zk when revoke subscription permission (#21696) Co-authored-by: fanjianye Co-authored-by: Jiwe Guo (cherry picked from commit b7d3a9caa7a8325c532be56a18b5d94cfa1aa0c7) --- .../PulsarAuthorizationProvider.java | 3 ++ .../AuthenticatedProducerConsumerTest.java | 45 +++++++++++++++++++ 2 files changed, 48 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 ece22fe223b97..acb6fce9b92e4 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 @@ -357,6 +357,9 @@ private CompletableFuture updateSubscriptionPermissionAsync(NamespaceName policies.auth_policies.getSubscriptionAuthentication().get(subscriptionName); if (subscriptionAuth != null) { subscriptionAuth.removeAll(roles); + if (subscriptionAuth.isEmpty()) { + policies.auth_policies.getSubscriptionAuthentication().remove(subscriptionName); + } } else { log.info("[{}] Couldn't find role {} while revoking for sub = {}", namespace, roles, subscriptionName); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java index 8189f8e86b5b3..3bd8b920a30fe 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/AuthenticatedProducerConsumerTest.java @@ -49,6 +49,7 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.zookeeper.KeeperException.Code; import org.awaitility.Awaitility; @@ -497,4 +498,48 @@ public void testCleanupEmptyTopicAuthenticationMap() throws Exception { .get().auth_policies.getTopicAuthentication().containsKey(topic)); }); } + + @Test + public void testCleanupEmptySubscriptionAuthenticationMap() throws Exception { + Map authParams = new HashMap<>(); + authParams.put("tlsCertFile", getTlsFileForClient("admin.cert")); + authParams.put("tlsKeyFile", getTlsFileForClient("admin.key-pk8")); + Authentication authTls = new AuthenticationTls(); + authTls.configure(authParams); + internalSetup(authTls); + + admin.clusters().createCluster("test", ClusterData.builder().build()); + admin.tenants().createTenant("p1", + new TenantInfoImpl(Collections.emptySet(), new HashSet<>(admin.clusters().getClusters()))); + String namespace = "p1/ns1"; + admin.namespaces().createNamespace("p1/ns1"); + + // grant permission1 and permission2 + String subscription = "test-sub-1"; + String role1 = "test-user-1"; + String role2 = "test-user-2"; + Set roles = new HashSet<>(); + roles.add(role1); + roles.add(role2); + admin.namespaces().grantPermissionOnSubscription(namespace, subscription, roles); + Optional policies = pulsar.getPulsarResources().getNamespaceResources().getPolicies(NamespaceName.get(namespace)); + assertTrue(policies.isPresent()); + assertTrue(policies.get().auth_policies.getSubscriptionAuthentication().containsKey(subscription)); + assertTrue(policies.get().auth_policies.getSubscriptionAuthentication().get(subscription).contains(role1)); + assertTrue(policies.get().auth_policies.getSubscriptionAuthentication().get(subscription).contains(role2)); + + // revoke permission1 + admin.namespaces().revokePermissionOnSubscription(namespace, subscription, role1); + policies = pulsar.getPulsarResources().getNamespaceResources().getPolicies(NamespaceName.get(namespace)); + assertTrue(policies.isPresent()); + assertTrue(policies.get().auth_policies.getSubscriptionAuthentication().containsKey(subscription)); + assertFalse(policies.get().auth_policies.getSubscriptionAuthentication().get(subscription).contains(role1)); + assertTrue(policies.get().auth_policies.getSubscriptionAuthentication().get(subscription).contains(role2)); + + // revoke permission2 + admin.namespaces().revokePermissionOnSubscription(namespace, subscription, role2); + policies = pulsar.getPulsarResources().getNamespaceResources().getPolicies(NamespaceName.get(namespace)); + assertTrue(policies.isPresent()); + assertFalse(policies.get().auth_policies.getSubscriptionAuthentication().containsKey(subscription)); + } } From a4d6d2a7363fd750da3c9f426cfcc703977458b4 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Tue, 12 Dec 2023 10:47:37 +0800 Subject: [PATCH 201/699] [fix][test] Fix PerformanceProducer send count error (#21706) (cherry picked from commit e7e7d3a454f3be12290761da1c41b74e0fc7e8ce) --- .../apache/pulsar/testclient/PerformanceProducer.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java index 63e3e2ec6fd23..8b5775f67818f 100644 --- a/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java +++ b/pulsar-testclient/src/main/java/org/apache/pulsar/testclient/PerformanceProducer.java @@ -566,7 +566,7 @@ private static void runProducer(int producerId, } } // Send messages on all topics/producers - long totalSent = 0; + AtomicLong totalSent = new AtomicLong(0); AtomicLong numMessageSend = new AtomicLong(0); Semaphore numMsgPerTxnLimit = new Semaphore(arguments.numMessagesPerTransaction); while (true) { @@ -586,7 +586,7 @@ private static void runProducer(int producerId, } if (numMessages > 0) { - if (totalSent++ >= numMessages) { + if (totalSent.get() >= numMessages) { log.info("------------- DONE (reached the maximum number: {} of production) --------------" , numMessages); doneLatch.countDown(); @@ -604,7 +604,7 @@ private static void runProducer(int producerId, if (arguments.payloadFilename != null) { if (messageFormatter != null) { - payloadData = messageFormatter.formatMessage(arguments.producerName, totalSent, + payloadData = messageFormatter.formatMessage(arguments.producerName, totalSent.get(), payloadByteList.get(ThreadLocalRandom.current().nextInt(payloadByteList.size()))); } else { payloadData = payloadByteList.get( @@ -642,13 +642,13 @@ private static void runProducer(int producerId, if (msgKeyMode == MessageKeyGenerationMode.random) { messageBuilder.key(String.valueOf(ThreadLocalRandom.current().nextInt())); } else if (msgKeyMode == MessageKeyGenerationMode.autoIncrement) { - messageBuilder.key(String.valueOf(totalSent)); + messageBuilder.key(String.valueOf(totalSent.get())); } PulsarClient pulsarClient = client; messageBuilder.sendAsync().thenRun(() -> { bytesSent.add(payloadData.length); messagesSent.increment(); - + totalSent.incrementAndGet(); totalMessagesSent.increment(); totalBytesSent.add(payloadData.length); From f6255e5e10b8a9dbd42c4bda9d407c21d9da68b4 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Tue, 12 Dec 2023 17:09:17 +0800 Subject: [PATCH 202/699] [fix][fn] Fix Deadlock in Functions Worker LeaderService (#21711) Fixes #21501 ### Motivation No need to `synchronized` the method `isLeader` in LeaderService See the deadlock stack : ``` "pulsar-external-listener-44525-1": at org.apache.pulsar.functions.worker.FunctionMetaDataManager.giveupLeadership(FunctionMetaDataManager.java) - waiting to lock <0x0000100013535c90> (a org.apache.pulsar.functions.worker.FunctionMetaDataManager) at org.apache.pulsar.functions.worker.LeaderService.becameInactive(LeaderService.java:167) - locked <0x000010001344c6d8> (a org.apache.pulsar.functions.worker.LeaderService) at org.apache.pulsar.client.impl.ConsumerImpl.lambda$activeConsumerChanged$27(ConsumerImpl.java:1136) at org.apache.pulsar.client.impl.ConsumerImpl$$Lambda$2606/0x00007f854ce9cb10.run(Unknown Source) at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@17.0.8.1/ThreadPoolExecutor.java:1136) at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@17.0.8.1/ThreadPoolExecutor.java:635) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.lang.Thread.run(java.base@17.0.8.1/Thread.java:833) "pulsar-web-44514-6": at org.apache.pulsar.functions.worker.LeaderService.isLeader(LeaderService.java) - waiting to lock <0x000010001344c6d8> (a org.apache.pulsar.functions.worker.LeaderService) at org.apache.pulsar.functions.worker.SchedulerManager.scheduleInternal(SchedulerManager.java:200) at org.apache.pulsar.functions.worker.SchedulerManager.schedule(SchedulerManager.java:229) at org.apache.pulsar.functions.worker.FunctionMetaDataManager.updateFunctionOnLeader(FunctionMetaDataManager.java:251) - locked <0x0000100013535c90> (a org.apache.pulsar.functions.worker.FunctionMetaDataManager) at org.apache.pulsar.functions.worker.rest.api.ComponentImpl.internalProcessFunctionRequest(ComponentImpl.java:1775) at org.apache.pulsar.functions.worker.rest.api.ComponentImpl.updateRequest(ComponentImpl.java:996) at org.apache.pulsar.functions.worker.rest.api.FunctionsImpl.registerFunction(FunctionsImpl.java:222) at org.apache.pulsar.broker.admin.impl.FunctionsBase.registerFunction(FunctionsBase.java:196) ``` (cherry picked from commit ac116558ccde3aaa7700299ee4b9ff0b1d64abc8) --- .../functions/worker/PulsarFunctionTlsTest.java | 16 ++++++++++++++++ .../pulsar/functions/worker/LeaderService.java | 4 ++-- 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java index 1e8b26beee38a..9882b15450e40 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java @@ -20,6 +20,8 @@ import static org.apache.pulsar.common.util.PortManager.nextLockedFreePort; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.assertNotNull; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Sets; @@ -33,6 +35,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; @@ -41,6 +44,7 @@ import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.common.functions.FunctionConfig; +import org.apache.pulsar.common.functions.WorkerInfo; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.util.ClassLoaderUtils; import org.apache.pulsar.common.util.ObjectMapperFactory; @@ -242,6 +246,18 @@ public void testFunctionsCreation() throws Exception { log.info(" -------- Start test function : {}", functionName); + int finalI = i; + Awaitility.await().atMost(1, TimeUnit.MINUTES).pollInterval(1, TimeUnit.SECONDS).untilAsserted(() -> { + final PulsarWorkerService workerService = ((PulsarWorkerService) fnWorkerServices[finalI]); + final LeaderService leaderService = workerService.getLeaderService(); + assertNotNull(leaderService); + if (leaderService.isLeader()) { + assertTrue(true); + } else { + final WorkerInfo workerInfo = workerService.getMembershipManager().getLeader(); + assertTrue(workerInfo != null && !workerInfo.getWorkerId().equals(workerService.getWorkerConfig().getWorkerId())); + } + }); pulsarAdmins[i].functions().createFunctionWithUrl( functionConfig, jarFilePathUrl ); diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/LeaderService.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/LeaderService.java index 7f035b5562f24..e7816f06aacc8 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/LeaderService.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/LeaderService.java @@ -41,7 +41,7 @@ public class LeaderService implements AutoCloseable, ConsumerEventListener { private ConsumerImpl consumer; private final WorkerConfig workerConfig; private final PulsarClient pulsarClient; - private boolean isLeader = false; + private volatile boolean isLeader = false; static final String COORDINATION_TOPIC_SUBSCRIPTION = "participants"; @@ -172,7 +172,7 @@ public synchronized void becameInactive(Consumer consumer, int partitionId) { } } - public synchronized boolean isLeader() { + public boolean isLeader() { return isLeader; } From 7381dfa4e2e65e8bfb14bd8b469111a2a6edd025 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Wed, 20 Dec 2023 09:39:00 +0800 Subject: [PATCH 203/699] [fix][broker] Avoid compaction task stuck when the last message to compact is a marker (#21718) (cherry picked from commit 1f995688ac8c3a87dd61864f013f896df1984ec2) Conflicts: pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java Fixed duplicate namespace created in transactionTest --- .../service/AbstractBaseDispatcher.java | 36 +++++++---- .../pulsar/compaction/TwoPhaseCompactor.java | 11 +++- .../service/ReplicatorSubscriptionTest.java | 61 ++++++++++++++++++ .../broker/transaction/TransactionTest.java | 62 ++++++++++++++++++- 4 files changed, 154 insertions(+), 16 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java index b36389ab2dada..0497a72acc64d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java @@ -49,6 +49,7 @@ import org.apache.pulsar.common.api.proto.ReplicatedSubscriptionsSnapshot; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.Markers; +import org.apache.pulsar.compaction.Compactor; import org.checkerframework.checker.nullness.qual.Nullable; @Slf4j @@ -174,13 +175,15 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i if (msgMetadata != null && msgMetadata.hasTxnidMostBits() && msgMetadata.hasTxnidLeastBits()) { if (Markers.isTxnMarker(msgMetadata)) { - // because consumer can receive message is smaller than maxReadPosition, - // so this marker is useless for this subscription - individualAcknowledgeMessageIfNeeded(Collections.singletonList(entry.getPosition()), - Collections.emptyMap()); - entries.set(i, null); - entry.release(); - continue; + if (cursor == null || !cursor.getName().equals(Compactor.COMPACTION_SUBSCRIPTION)) { + // because consumer can receive message is smaller than maxReadPosition, + // so this marker is useless for this subscription + individualAcknowledgeMessageIfNeeded(Collections.singletonList(entry.getPosition()), + Collections.emptyMap()); + entries.set(i, null); + entry.release(); + continue; + } } else if (((PersistentTopic) subscription.getTopic()) .isTxnAborted(new TxnID(msgMetadata.getTxnidMostBits(), msgMetadata.getTxnidLeastBits()), (PositionImpl) entry.getPosition())) { @@ -192,19 +195,26 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i } } - if (msgMetadata == null || Markers.isServerOnlyMarker(msgMetadata)) { + if (msgMetadata == null || (Markers.isServerOnlyMarker(msgMetadata))) { PositionImpl pos = (PositionImpl) entry.getPosition(); // Message metadata was corrupted or the messages was a server-only marker if (Markers.isReplicatedSubscriptionSnapshotMarker(msgMetadata)) { + final int readerIndex = metadataAndPayload.readerIndex(); processReplicatedSubscriptionSnapshot(pos, metadataAndPayload); + metadataAndPayload.readerIndex(readerIndex); } - entries.set(i, null); - entry.release(); - individualAcknowledgeMessageIfNeeded(Collections.singletonList(pos), - Collections.emptyMap()); - continue; + // Deliver marker to __compaction cursor to avoid compaction task stuck, + // and filter out them when doing topic compaction. + if (msgMetadata == null || cursor == null + || !cursor.getName().equals(Compactor.COMPACTION_SUBSCRIPTION)) { + entries.set(i, null); + entry.release(); + individualAcknowledgeMessageIfNeeded(Collections.singletonList(pos), + Collections.emptyMap()); + continue; + } } else if (trackDelayedDelivery(entry.getLedgerId(), entry.getEntryId(), msgMetadata)) { // The message is marked for delayed delivery. Ignore for now. entries.set(i, null); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java index a78323a9cfe6d..647c34a94ad81 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/TwoPhaseCompactor.java @@ -45,6 +45,7 @@ import org.apache.pulsar.client.impl.RawBatchConverter; import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.protocol.Commands; +import org.apache.pulsar.common.protocol.Markers; import org.apache.pulsar.common.util.FutureUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -130,7 +131,10 @@ private void phaseOneLoop(RawReader reader, boolean replaceMessage = false; mxBean.addCompactionReadOp(reader.getTopic(), m.getHeadersAndPayload().readableBytes()); MessageMetadata metadata = Commands.parseMessageMetadata(m.getHeadersAndPayload()); - if (RawBatchConverter.isReadableBatch(metadata)) { + if (Markers.isServerOnlyMarker(metadata)) { + mxBean.addCompactionRemovedEvent(reader.getTopic()); + deletedMessage = true; + } else if (RawBatchConverter.isReadableBatch(metadata)) { try { int numMessagesInBatch = metadata.getNumMessagesInBatch(); int deleteCnt = 0; @@ -262,7 +266,10 @@ private void phaseTwoLoop(RawReader reader, MessageId to, Map MessageId id = m.getMessageId(); Optional messageToAdd = Optional.empty(); mxBean.addCompactionReadOp(reader.getTopic(), m.getHeadersAndPayload().readableBytes()); - if (RawBatchConverter.isReadableBatch(m)) { + MessageMetadata metadata = Commands.parseMessageMetadata(m.getHeadersAndPayload()); + if (Markers.isServerOnlyMarker(metadata)) { + messageToAdd = Optional.empty(); + } else if (RawBatchConverter.isReadableBatch(metadata)) { try { messageToAdd = rebatchMessage(reader.getTopic(), m, (key, subid) -> subid.equals(latestForKey.get(key)), topicCompactionRetainNullKey); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java index 529fb923f5918..fe519827be74a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java @@ -42,6 +42,7 @@ import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.persistent.ReplicatedSubscriptionsController; +import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -822,6 +823,66 @@ public void testWriteMarkerTaskOfReplicateSubscriptions(boolean isTopicPolicyEna pulsar1.getConfiguration().setForceDeleteNamespaceAllowed(false); } + @Test + public void testReplicatedSubscriptionWithCompaction() throws Exception { + final String namespace = BrokerTestUtil.newUniqueName("pulsar/replicatedsubscription"); + final String topicName = "persistent://" + namespace + "/testReplicatedSubscriptionWithCompaction"; + final String subName = "sub"; + + admin1.namespaces().createNamespace(namespace); + admin1.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("r1", "r2")); + admin1.topics().createNonPartitionedTopic(topicName); + admin1.topicPolicies().setCompactionThreshold(topicName, 100 * 1024 * 1024L); + + @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(url1.toString()) + .statsInterval(0, TimeUnit.SECONDS).build(); + + Producer producer = client.newProducer(Schema.STRING).topic(topicName).create(); + producer.newMessage().key("K1").value("V1").send(); + producer.newMessage().key("K1").value("V2").send(); + producer.close(); + + createReplicatedSubscription(client, topicName, subName, true); + Awaitility.await().untilAsserted(() -> { + Map status = admin1.topics().getReplicatedSubscriptionStatus(topicName, subName); + assertTrue(status.get(topicName)); + }); + + Awaitility.await().untilAsserted(() -> { + PersistentTopic t1 = (PersistentTopic) pulsar1.getBrokerService() + .getTopic(topicName, false).get().get(); + ReplicatedSubscriptionsController rsc1 = t1.getReplicatedSubscriptionController().get(); + Assert.assertTrue(rsc1.getLastCompletedSnapshotId().isPresent()); + assertEquals(t1.getPendingWriteOps().get(), 0L); + }); + + admin1.topics().triggerCompaction(topicName); + + Awaitility.await().untilAsserted(() -> { + assertEquals(admin1.topics().compactionStatus(topicName).status, + LongRunningProcessStatus.Status.SUCCESS); + }); + + @Cleanup + Consumer consumer = client.newConsumer(Schema.STRING) + .topic(topicName) + .subscriptionName("sub2") + .subscriptionType(SubscriptionType.Exclusive) + .readCompacted(true) + .subscribe(); + List result = new ArrayList<>(); + while (true) { + Message receive = consumer.receive(2, TimeUnit.SECONDS); + if (receive == null) { + break; + } + + result.add(receive.getValue()); + } + + Assert.assertEquals(result, List.of("V2")); + } + /** * Disable replication subscription. * Test scheduled task case. 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 8630b1970b12f..b387e193c6231 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 @@ -81,6 +81,7 @@ import org.apache.commons.lang3.RandomStringUtils; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.commons.lang3.reflect.MethodUtils; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.intercept.CounterBrokerInterceptor; @@ -108,6 +109,7 @@ import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore; import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStoreProvider; import org.apache.pulsar.broker.transaction.pendingack.impl.PendingAckHandleImpl; +import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -1789,7 +1791,7 @@ private void getTopic(String topicName) { @Test public void testReadCommittedWithReadCompacted() throws Exception{ - final String namespace = "tnx/ns-prechecks"; + final String namespace = BrokerTestUtil.newUniqueName("tnx/ns-prechecks"); final String topic = "persistent://" + namespace + "/test_transaction_topic"; admin.namespaces().createNamespace(namespace); admin.topics().createNonPartitionedTopic(topic); @@ -1842,6 +1844,64 @@ public void testReadCommittedWithReadCompacted() throws Exception{ Assert.assertEquals(messages, List.of("V2", "V3")); } + + @Test + public void testReadCommittedWithCompaction() throws Exception{ + final String namespace = BrokerTestUtil.newUniqueName("tnx/ns-prechecks"); + final String topic = "persistent://" + namespace + "/test_transaction_topic" + UUID.randomUUID(); + admin.namespaces().createNamespace(namespace); + admin.topics().createNonPartitionedTopic(topic); + + admin.topicPolicies().setCompactionThreshold(topic, 100 * 1024 * 1024); + + @Cleanup + Producer producer = this.pulsarClient.newProducer(Schema.STRING) + .topic(topic) + .create(); + + producer.newMessage().key("K1").value("V1").send(); + + Transaction txn = pulsarClient.newTransaction() + .withTransactionTimeout(1, TimeUnit.MINUTES).build().get(); + producer.newMessage(txn).key("K2").value("V2").send(); + producer.newMessage(txn).key("K3").value("V3").send(); + txn.commit().get(); + + producer.newMessage().key("K1").value("V4").send(); + + Transaction txn2 = pulsarClient.newTransaction() + .withTransactionTimeout(1, TimeUnit.MINUTES).build().get(); + producer.newMessage(txn2).key("K2").value("V5").send(); + producer.newMessage(txn2).key("K3").value("V6").send(); + txn2.commit().get(); + + admin.topics().triggerCompaction(topic); + + Awaitility.await().untilAsserted(() -> { + assertEquals(admin.topics().compactionStatus(topic).status, + LongRunningProcessStatus.Status.SUCCESS); + }); + + @Cleanup + Consumer consumer = this.pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("sub") + .subscriptionType(SubscriptionType.Exclusive) + .readCompacted(true) + .subscribe(); + List result = new ArrayList<>(); + while (true) { + Message receive = consumer.receive(2, TimeUnit.SECONDS); + if (receive == null) { + break; + } + + result.add(receive.getValue()); + } + + Assert.assertEquals(result, List.of("V4", "V5", "V6")); + } + @DataProvider(name = "BlockTransactionsIfReplicationEnabledValues") public static Object[][] packageNamesProvider() { return new Object[][]{ From 63dbaef0f9f32dbd0238382de1d7c441d9a5bf96 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Fri, 29 Dec 2023 14:39:45 +0800 Subject: [PATCH 204/699] [fix][broker] Delete compacted ledger when topic is deleted (#21745) (cherry picked from commit 1c8aeba60032965b98c354a1deacebe0114a3346) --- .../service/persistent/PersistentTopic.java | 61 ++++++-- .../PulsarCompactorSubscription.java | 17 +++ .../pulsar/client/impl/RawReaderImpl.java | 3 +- .../pulsar/compaction/CompactedTopicImpl.java | 5 + .../pulsar/compaction/CompactionTest.java | 130 ++++++++++++++++++ 5 files changed, 203 insertions(+), 13 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 4409114e013dc..f3d8ec7970136 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 @@ -213,7 +213,8 @@ public class PersistentTopic extends AbstractTopic implements Topic, AddEntryCal protected final MessageDeduplication messageDeduplication; private static final Long COMPACTION_NEVER_RUN = -0xfebecffeL; - private CompletableFuture currentCompaction = CompletableFuture.completedFuture(COMPACTION_NEVER_RUN); + private volatile CompletableFuture currentCompaction = CompletableFuture.completedFuture( + COMPACTION_NEVER_RUN); private TopicCompactionService topicCompactionService; // TODO: Create compaction strategy from topic policy when exposing strategic compaction to users. @@ -1166,13 +1167,14 @@ private void asyncDeleteCursorWithClearDelayedMessage(String subscriptionName, CompletableFuture unsubscribeFuture) { PersistentSubscription persistentSubscription = subscriptions.get(subscriptionName); if (persistentSubscription == null) { - log.warn("[{}][{}] Can't find subscription, skip clear delayed message", topic, subscriptionName); + log.warn("[{}][{}] Can't find subscription, skip delete cursor", topic, subscriptionName); unsubscribeFuture.complete(null); return; } + if (!isDelayedDeliveryEnabled() || !(brokerService.getDelayedDeliveryTrackerFactory() instanceof BucketDelayedDeliveryTrackerFactory)) { - asyncDeleteCursor(subscriptionName, unsubscribeFuture); + asyncDeleteCursorWithCleanCompactionLedger(persistentSubscription, unsubscribeFuture); return; } @@ -1187,7 +1189,7 @@ private void asyncDeleteCursorWithClearDelayedMessage(String subscriptionName, if (ex != null) { unsubscribeFuture.completeExceptionally(ex); } else { - asyncDeleteCursor(subscriptionName, unsubscribeFuture); + asyncDeleteCursorWithCleanCompactionLedger(persistentSubscription, unsubscribeFuture); } }); } @@ -1197,6 +1199,29 @@ private void asyncDeleteCursorWithClearDelayedMessage(String subscriptionName, dispatcher.clearDelayedMessages().whenComplete((__, ex) -> { if (ex != null) { unsubscribeFuture.completeExceptionally(ex); + } else { + asyncDeleteCursorWithCleanCompactionLedger(persistentSubscription, unsubscribeFuture); + } + }); + } + + private void asyncDeleteCursorWithCleanCompactionLedger(PersistentSubscription subscription, + CompletableFuture unsubscribeFuture) { + final String subscriptionName = subscription.getName(); + if ((!isCompactionSubscription(subscriptionName)) || !(subscription instanceof PulsarCompactorSubscription)) { + asyncDeleteCursor(subscriptionName, unsubscribeFuture); + return; + } + + currentCompaction.handle((__, e) -> { + if (e != null) { + log.warn("[{}][{}] Last compaction task failed", topic, subscriptionName); + } + return ((PulsarCompactorSubscription) subscription).cleanCompactedLedger(); + }).whenComplete((__, ex) -> { + if (ex != null) { + log.error("[{}][{}] Error cleaning compacted ledger", topic, subscriptionName, ex); + unsubscribeFuture.completeExceptionally(ex); } else { asyncDeleteCursor(subscriptionName, unsubscribeFuture); } @@ -3187,17 +3212,29 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { public synchronized void triggerCompaction() throws PulsarServerException, AlreadyRunningException { if (currentCompaction.isDone()) { + if (!lock.readLock().tryLock()) { + log.info("[{}] Conflict topic-close, topic-delete, skip triggering compaction", topic); + return; + } + try { + if (isClosingOrDeleting) { + log.info("[{}] Topic is closing or deleting, skip triggering compaction", topic); + return; + } - if (strategicCompactionMap.containsKey(topic)) { - currentCompaction = brokerService.pulsar().getStrategicCompactor() - .compact(topic, strategicCompactionMap.get(topic)); - } else { - currentCompaction = topicCompactionService.compact().thenApply(x -> null); + if (strategicCompactionMap.containsKey(topic)) { + currentCompaction = brokerService.pulsar().getStrategicCompactor() + .compact(topic, strategicCompactionMap.get(topic)); + } else { + currentCompaction = topicCompactionService.compact().thenApply(x -> null); + } + } finally { + lock.readLock().unlock(); } currentCompaction.whenComplete((ignore, ex) -> { - if (ex != null){ - log.warn("[{}] Compaction failure.", topic, ex); - } + if (ex != null) { + log.warn("[{}] Compaction failure.", topic, ex); + } }); } else { throw new AlreadyRunningException("Compaction already in progress"); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PulsarCompactorSubscription.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PulsarCompactorSubscription.java index dbb09f6ac39fd..fe13aeb572e2e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PulsarCompactorSubscription.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PulsarCompactorSubscription.java @@ -22,12 +22,15 @@ import static org.apache.pulsar.broker.service.AbstractBaseDispatcher.checkAndApplyReachedEndOfTopicOrTopicMigration; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.mledger.AsyncCallbacks.MarkDeleteCallback; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.compaction.CompactedTopic; +import org.apache.pulsar.compaction.CompactedTopicContext; +import org.apache.pulsar.compaction.CompactedTopicImpl; import org.apache.pulsar.compaction.Compactor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -106,5 +109,19 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { } } + CompletableFuture cleanCompactedLedger() { + final CompletableFuture compactedTopicContextFuture = + ((CompactedTopicImpl) compactedTopic).getCompactedTopicContextFuture(); + if (compactedTopicContextFuture != null) { + return compactedTopicContextFuture.thenCompose(context -> { + long compactedLedgerId = context.getLedger().getId(); + ((CompactedTopicImpl) compactedTopic).reset(); + return compactedTopic.deleteCompactedLedger(compactedLedgerId); + }); + } else { + return CompletableFuture.completedFuture(null); + } + } + private static final Logger log = LoggerFactory.getLogger(PulsarCompactorSubscription.class); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawReaderImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawReaderImpl.java index 70bda888bf7ea..f65232413991f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawReaderImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawReaderImpl.java @@ -59,6 +59,7 @@ public RawReaderImpl(PulsarClientImpl client, String topic, String subscription, consumerConfiguration.setReceiverQueueSize(DEFAULT_RECEIVER_QUEUE_SIZE); consumerConfiguration.setReadCompacted(true); consumerConfiguration.setSubscriptionInitialPosition(SubscriptionInitialPosition.Earliest); + consumerConfiguration.setAckReceiptEnabled(true); consumer = new RawConsumerImpl(client, consumerConfiguration, consumerFuture); @@ -122,7 +123,7 @@ static class RawConsumerImpl extends ConsumerImpl { MessageId.earliest, 0 /* startMessageRollbackDurationInSec */, Schema.BYTES, null, - true + false ); incomingRawMessages = new GrowableArrayBlockingQueue<>(); pendingRawReceives = new ConcurrentLinkedQueue<>(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java index 8794e2736d4d4..a8e124c84a250 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java @@ -330,6 +330,11 @@ public Optional getCompactionHorizon() { return Optional.ofNullable(this.compactionHorizon); } + public void reset() { + this.compactionHorizon = null; + this.compactedTopicContext = null; + } + @Nullable public CompletableFuture getCompactedTopicContextFuture() { return compactedTopicContext; 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 69baca3abc31c..25fd18228ddf7 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 @@ -25,6 +25,7 @@ import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; import com.google.common.collect.Sets; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -46,21 +47,26 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; import lombok.Cleanup; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.api.OpenBuilder; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerInfo; import org.apache.bookkeeper.mledger.Position; +import org.apache.commons.lang3.mutable.MutableLong; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.persistent.SystemTopic; import org.apache.pulsar.client.admin.LongRunningProcessStatus; @@ -81,6 +87,7 @@ import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.ConsumerImpl; +import org.apache.pulsar.common.api.proto.CommandAck; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; @@ -1989,4 +1996,127 @@ public void testCompactionDuplicate() throws Exception { } } } + + @Test + public void testDeleteCompactedLedger() throws Exception { + String topicName = "persistent://my-property/use/my-ns/testDeleteCompactedLedger"; + + final String subName = "my-sub"; + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .enableBatching(false).topic(topicName).create(); + + pulsarClient.newConsumer().topic(topicName).subscriptionName(subName).readCompacted(true).subscribe().close(); + + for (int i = 0; i < 10; i++) { + producer.newMessage().key(String.valueOf(i % 2)).value(String.valueOf(i)).sendAsync(); + } + producer.flush(); + + compact(topicName); + + MutableLong compactedLedgerId = new MutableLong(-1); + Awaitility.await().untilAsserted(() -> { + PersistentTopicInternalStats stats = admin.topics().getInternalStats(topicName); + Assert.assertNotEquals(stats.compactedLedger.ledgerId, -1L); + compactedLedgerId.setValue(stats.compactedLedger.ledgerId); + Assert.assertEquals(stats.compactedLedger.entries, 2L); + }); + + // delete compacted ledger + admin.topics().deleteSubscription(topicName, "__compaction"); + + Awaitility.await().untilAsserted(() -> { + PersistentTopicInternalStats stats = admin.topics().getInternalStats(topicName); + Assert.assertEquals(stats.compactedLedger.ledgerId, -1L); + Assert.assertEquals(stats.compactedLedger.entries, -1L); + assertThrows(BKException.BKNoSuchLedgerExistsException.class, () -> pulsarTestContext.getBookKeeperClient() + .openLedger(compactedLedgerId.getValue(), BookKeeper.DigestType.CRC32C, new byte[]{})); + }); + + compact(topicName); + + MutableLong compactedLedgerId2 = new MutableLong(-1); + Awaitility.await().untilAsserted(() -> { + PersistentTopicInternalStats stats = admin.topics().getInternalStats(topicName); + Assert.assertNotEquals(stats.compactedLedger.ledgerId, -1L); + compactedLedgerId2.setValue(stats.compactedLedger.ledgerId); + Assert.assertEquals(stats.compactedLedger.entries, 2L); + }); + + producer.close(); + admin.topics().delete(topicName); + + Awaitility.await().untilAsserted(() -> assertThrows(BKException.BKNoSuchLedgerExistsException.class, + () -> pulsarTestContext.getBookKeeperClient().openLedger( + compactedLedgerId2.getValue(), BookKeeper.DigestType.CRC32, new byte[]{}))); + } + + @Test + public void testDeleteCompactedLedgerWithSlowAck() throws Exception { + // Disable topic level policies, since block ack thread may also block thread of delete topic policies. + conf.setTopicLevelPoliciesEnabled(false); + restartBroker(); + + String topicName = "persistent://my-property/use/my-ns/testDeleteCompactedLedgerWithSlowAck"; + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .enableBatching(false).topic(topicName).create(); + + pulsarClient.newConsumer().topic(topicName).subscriptionType(SubscriptionType.Exclusive) + .subscriptionName(Compactor.COMPACTION_SUBSCRIPTION) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest).readCompacted(true).subscribe() + .close(); + + for (int i = 0; i < 10; i++) { + producer.newMessage().key(String.valueOf(i % 2)).value(String.valueOf(i)).sendAsync(); + } + producer.flush(); + + PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get(); + PersistentSubscription subscription = spy(topic.getSubscription(Compactor.COMPACTION_SUBSCRIPTION)); + topic.getSubscriptions().put(Compactor.COMPACTION_SUBSCRIPTION, subscription); + + AtomicLong compactedLedgerId = new AtomicLong(-1); + AtomicBoolean pauseAck = new AtomicBoolean(); + Mockito.doAnswer(invocationOnMock -> { + Map properties = (Map) invocationOnMock.getArguments()[2]; + log.info("acknowledgeMessage properties: {}", properties); + compactedLedgerId.set(properties.get(Compactor.COMPACTED_TOPIC_LEDGER_PROPERTY)); + pauseAck.set(true); + while (pauseAck.get()) { + Thread.sleep(200); + } + return invocationOnMock.callRealMethod(); + }).when(subscription).acknowledgeMessage(Mockito.any(), Mockito.eq( + CommandAck.AckType.Cumulative), Mockito.any()); + + admin.topics().triggerCompaction(topicName); + + while (!pauseAck.get()) { + Thread.sleep(100); + } + + CompletableFuture currentCompaction = + (CompletableFuture) FieldUtils.readDeclaredField(topic, "currentCompaction", true); + CompletableFuture spyCurrentCompaction = spy(currentCompaction); + FieldUtils.writeDeclaredField(topic, "currentCompaction", spyCurrentCompaction, true); + currentCompaction.whenComplete((obj, throwable) -> { + if (throwable != null) { + spyCurrentCompaction.completeExceptionally(throwable); + } else { + spyCurrentCompaction.complete(obj); + } + }); + Mockito.doAnswer(invocationOnMock -> { + pauseAck.set(false); + return invocationOnMock.callRealMethod(); + }).when(spyCurrentCompaction).handle(Mockito.any()); + + admin.topics().delete(topicName, true); + + Awaitility.await().untilAsserted(() -> assertThrows(BKException.BKNoSuchLedgerExistsException.class, + () -> pulsarTestContext.getBookKeeperClient().openLedger( + compactedLedgerId.get(), BookKeeper.DigestType.CRC32, new byte[]{}))); + } } From 2338bc54b79a4ee3c00e4377b2f3d89d7863f755 Mon Sep 17 00:00:00 2001 From: jiangpengcheng Date: Mon, 11 Dec 2023 09:13:43 +0800 Subject: [PATCH 205/699] [improve][io] Make connectors load sensitive fields from secrets (#21675) (cherry picked from commit c55df07d3c43df09130ddd2e1e8411b39467e46a) --- pulsar-io/canal/pom.xml | 5 ++ .../pulsar/io/canal/CanalAbstractSource.java | 2 +- .../pulsar/io/canal/CanalSourceConfig.java | 7 ++- .../pulsar/io/common/IOConfigUtils.java | 7 ++- .../pulsar/io/common/IOConfigUtilsTest.java | 11 ++++ pulsar-io/dynamodb/pom.xml | 6 ++ .../pulsar/io/dynamodb/DynamoDBSource.java | 2 +- .../io/dynamodb/DynamoDBSourceConfig.java | 8 ++- .../dynamodb/DynamoDBSourceConfigTests.java | 52 ++++++++++++++-- pulsar-io/influxdb/pom.xml | 5 ++ .../influxdb/InfluxDBGenericRecordSink.java | 4 +- .../io/influxdb/v1/InfluxDBAbstractSink.java | 2 +- .../io/influxdb/v1/InfluxDBSinkConfig.java | 11 ++-- .../pulsar/io/influxdb/v2/InfluxDBSink.java | 2 +- .../io/influxdb/v2/InfluxDBSinkConfig.java | 14 ++--- .../influxdb/v1/InfluxDBSinkConfigTest.java | 56 ++++++++++++++--- .../influxdb/v2/InfluxDBSinkConfigTest.java | 29 +++++++-- pulsar-io/jdbc/core/pom.xml | 6 ++ .../pulsar/io/jdbc/JdbcAbstractSink.java | 2 +- .../apache/pulsar/io/jdbc/JdbcSinkConfig.java | 7 ++- pulsar-io/kafka/pom.xml | 5 ++ .../pulsar/io/kafka/KafkaAbstractSink.java | 6 +- .../pulsar/io/kafka/KafkaAbstractSource.java | 2 +- .../pulsar/io/kafka/KafkaSinkConfig.java | 11 ++-- .../pulsar/io/kafka/KafkaSourceConfig.java | 9 ++- .../io/kafka/sink/KafkaAbstractSinkTest.java | 8 +-- .../kafka/source/KafkaAbstractSourceTest.java | 26 +++++++- pulsar-io/mongo/pom.xml | 5 ++ .../io/mongodb/MongoAbstractConfig.java | 3 +- .../apache/pulsar/io/mongodb/MongoSink.java | 2 +- .../pulsar/io/mongodb/MongoSinkConfig.java | 9 ++- .../apache/pulsar/io/mongodb/MongoSource.java | 2 +- .../pulsar/io/mongodb/MongoSourceConfig.java | 10 ++-- .../io/mongodb/MongoSinkConfigTest.java | 41 ++++++++++--- .../io/mongodb/MongoSourceConfigTest.java | 38 ++++++++++-- pulsar-io/rabbitmq/pom.xml | 5 ++ .../pulsar/io/rabbitmq/RabbitMQSink.java | 2 +- .../io/rabbitmq/RabbitMQSinkConfig.java | 9 ++- .../pulsar/io/rabbitmq/RabbitMQSource.java | 2 +- .../io/rabbitmq/RabbitMQSourceConfig.java | 7 ++- .../rabbitmq/sink/RabbitMQSinkConfigTest.java | 52 ++++++++++++++-- .../source/RabbitMQSourceConfigTest.java | 60 +++++++++++++++++-- pulsar-io/redis/pom.xml | 5 ++ .../pulsar/io/redis/RedisAbstractConfig.java | 5 +- .../pulsar/io/redis/sink/RedisSink.java | 2 +- .../pulsar/io/redis/sink/RedisSinkConfig.java | 11 ++-- .../io/redis/sink/RedisSinkConfigTest.java | 47 ++++++++++++--- .../pulsar/io/redis/sink/RedisSinkTest.java | 5 +- pulsar-io/solr/pom.xml | 5 ++ .../pulsar/io/solr/SolrAbstractSink.java | 2 +- .../apache/pulsar/io/solr/SolrSinkConfig.java | 7 ++- .../pulsar/io/solr/SolrSinkConfigTest.java | 47 ++++++++++++--- 52 files changed, 540 insertions(+), 148 deletions(-) diff --git a/pulsar-io/canal/pom.xml b/pulsar-io/canal/pom.xml index 34b96d1502b9c..5e65d582dccc6 100644 --- a/pulsar-io/canal/pom.xml +++ b/pulsar-io/canal/pom.xml @@ -37,6 +37,11 @@
+ + ${project.groupId} + pulsar-io-common + ${project.version} + ${project.groupId} pulsar-io-core diff --git a/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/CanalAbstractSource.java b/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/CanalAbstractSource.java index 06c8788d5aea1..7d0cd0305a49e 100644 --- a/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/CanalAbstractSource.java +++ b/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/CanalAbstractSource.java @@ -57,7 +57,7 @@ public abstract class CanalAbstractSource extends PushSource { @Override public void open(Map config, SourceContext sourceContext) throws Exception { - canalSourceConfig = CanalSourceConfig.load(config); + canalSourceConfig = CanalSourceConfig.load(config, sourceContext); if (canalSourceConfig.getCluster()) { connector = CanalConnectors.newClusterConnector(canalSourceConfig.getZkServers(), canalSourceConfig.getDestination(), canalSourceConfig.getUsername(), diff --git a/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/CanalSourceConfig.java b/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/CanalSourceConfig.java index a0408e60e5f76..5a754988ffdc1 100644 --- a/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/CanalSourceConfig.java +++ b/pulsar-io/canal/src/main/java/org/apache/pulsar/io/canal/CanalSourceConfig.java @@ -26,6 +26,8 @@ import java.util.Map; import lombok.Data; import lombok.experimental.Accessors; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SourceContext; import org.apache.pulsar.io.core.annotations.FieldDoc; @@ -86,8 +88,7 @@ public static CanalSourceConfig load(String yamlFile) throws IOException { } - public static CanalSourceConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), CanalSourceConfig.class); + public static CanalSourceConfig load(Map map, SourceContext sourceContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, CanalSourceConfig.class, sourceContext); } } diff --git a/pulsar-io/common/src/main/java/org/apache/pulsar/io/common/IOConfigUtils.java b/pulsar-io/common/src/main/java/org/apache/pulsar/io/common/IOConfigUtils.java index d15986a897caa..69d981bf68728 100644 --- a/pulsar-io/common/src/main/java/org/apache/pulsar/io/common/IOConfigUtils.java +++ b/pulsar-io/common/src/main/java/org/apache/pulsar/io/common/IOConfigUtils.java @@ -77,13 +77,14 @@ private static T loadWithSecrets(Map map, Class clazz, } } configs.computeIfAbsent(field.getName(), key -> { - if (fieldDoc.required()) { - throw new IllegalArgumentException(field.getName() + " cannot be null"); - } + // Use default value if it is not null before checking required String value = fieldDoc.defaultValue(); if (value != null && !value.isEmpty()) { return value; } + if (fieldDoc.required()) { + throw new IllegalArgumentException(field.getName() + " cannot be null"); + } return null; }); } diff --git a/pulsar-io/common/src/test/java/org/apache/pulsar/io/common/IOConfigUtilsTest.java b/pulsar-io/common/src/test/java/org/apache/pulsar/io/common/IOConfigUtilsTest.java index 52afac1a5ac0c..fdcd2ea7fe88a 100644 --- a/pulsar-io/common/src/test/java/org/apache/pulsar/io/common/IOConfigUtilsTest.java +++ b/pulsar-io/common/src/test/java/org/apache/pulsar/io/common/IOConfigUtilsTest.java @@ -54,6 +54,14 @@ static class TestDefaultConfig { ) protected String testRequired; + @FieldDoc( + required = true, + defaultValue = "defaultRequired", + sensitive = true, + help = "testRequired" + ) + protected String testDefaultRequired; + @FieldDoc( required = false, defaultValue = "defaultStr", @@ -299,6 +307,9 @@ public void testDefaultValue() { configMap.put("testRequired", "test"); TestDefaultConfig testDefaultConfig = IOConfigUtils.loadWithSecrets(configMap, TestDefaultConfig.class, new TestSinkContext()); + // if there is default value for a required field and no value provided when load config, + // it should not throw exception but use the default value. + Assert.assertEquals(testDefaultConfig.getTestDefaultRequired(), "defaultRequired"); Assert.assertEquals(testDefaultConfig.getDefaultStr(), "defaultStr"); Assert.assertEquals(testDefaultConfig.isDefaultBool(), true); Assert.assertEquals(testDefaultConfig.getDefaultInt(), 100); diff --git a/pulsar-io/dynamodb/pom.xml b/pulsar-io/dynamodb/pom.xml index 7d6dc2ca6c125..372847e89afaf 100644 --- a/pulsar-io/dynamodb/pom.xml +++ b/pulsar-io/dynamodb/pom.xml @@ -32,6 +32,12 @@ + + ${project.groupId} + pulsar-io-common + ${project.version} + + ${project.groupId} pulsar-io-core diff --git a/pulsar-io/dynamodb/src/main/java/org/apache/pulsar/io/dynamodb/DynamoDBSource.java b/pulsar-io/dynamodb/src/main/java/org/apache/pulsar/io/dynamodb/DynamoDBSource.java index d67c4e21154ee..2193cf39c17a5 100644 --- a/pulsar-io/dynamodb/src/main/java/org/apache/pulsar/io/dynamodb/DynamoDBSource.java +++ b/pulsar-io/dynamodb/src/main/java/org/apache/pulsar/io/dynamodb/DynamoDBSource.java @@ -65,7 +65,7 @@ public void close() throws Exception { @Override public void open(Map config, SourceContext sourceContext) throws Exception { - this.dynamodbSourceConfig = DynamoDBSourceConfig.load(config); + this.dynamodbSourceConfig = DynamoDBSourceConfig.load(config, sourceContext); checkArgument(isNotBlank(dynamodbSourceConfig.getAwsDynamodbStreamArn()), "empty dynamo-stream arn"); // Even if the endpoint is set, it seems to require a region to go with it diff --git a/pulsar-io/dynamodb/src/main/java/org/apache/pulsar/io/dynamodb/DynamoDBSourceConfig.java b/pulsar-io/dynamodb/src/main/java/org/apache/pulsar/io/dynamodb/DynamoDBSourceConfig.java index b734dd5741155..0547ff8f863e0 100644 --- a/pulsar-io/dynamodb/src/main/java/org/apache/pulsar/io/dynamodb/DynamoDBSourceConfig.java +++ b/pulsar-io/dynamodb/src/main/java/org/apache/pulsar/io/dynamodb/DynamoDBSourceConfig.java @@ -35,6 +35,8 @@ import java.util.Map; import lombok.Data; import org.apache.pulsar.io.aws.AwsCredentialProviderPlugin; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SourceContext; import org.apache.pulsar.io.core.annotations.FieldDoc; import software.amazon.awssdk.regions.Region; @@ -77,6 +79,7 @@ public class DynamoDBSourceConfig implements Serializable { @FieldDoc( required = false, defaultValue = "", + sensitive = true, help = "json-parameters to initialize `AwsCredentialsProviderPlugin`") private String awsCredentialPluginParam = ""; @@ -170,9 +173,8 @@ public static DynamoDBSourceConfig load(String yamlFile) throws IOException { return mapper.readValue(new File(yamlFile), DynamoDBSourceConfig.class); } - public static DynamoDBSourceConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), DynamoDBSourceConfig.class); + public static DynamoDBSourceConfig load(Map map, SourceContext sourceContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, DynamoDBSourceConfig.class, sourceContext); } protected Region regionAsV2Region() { diff --git a/pulsar-io/dynamodb/src/test/java/org/apache/pulsar/io/dynamodb/DynamoDBSourceConfigTests.java b/pulsar-io/dynamodb/src/test/java/org/apache/pulsar/io/dynamodb/DynamoDBSourceConfigTests.java index f84cb785896e6..bdccaa2e5846e 100644 --- a/pulsar-io/dynamodb/src/test/java/org/apache/pulsar/io/dynamodb/DynamoDBSourceConfigTests.java +++ b/pulsar-io/dynamodb/src/test/java/org/apache/pulsar/io/dynamodb/DynamoDBSourceConfigTests.java @@ -31,6 +31,8 @@ import java.util.Map; import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; +import org.apache.pulsar.io.core.SourceContext; +import org.mockito.Mockito; import org.testng.annotations.Test; @@ -90,7 +92,8 @@ public final void loadFromMapTest() throws IOException { map.put("initialPositionInStream", InitialPositionInStream.TRIM_HORIZON); map.put("startAtTime", DAY); - DynamoDBSourceConfig config = DynamoDBSourceConfig.load(map); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + DynamoDBSourceConfig config = DynamoDBSourceConfig.load(map, sourceContext); assertNotNull(config); assertEquals(config.getAwsEndpoint(), "https://some.endpoint.aws"); @@ -111,7 +114,46 @@ public final void loadFromMapTest() throws IOException { ZonedDateTime expected = ZonedDateTime.ofInstant(DAY.toInstant(), ZoneOffset.UTC); assertEquals(actual, expected); } - + + @Test + public final void loadFromMapCredentialFromSecretTest() throws IOException { + Map map = new HashMap (); + map.put("awsEndpoint", "https://some.endpoint.aws"); + map.put("awsRegion", "us-east-1"); + map.put("awsDynamodbStreamArn", "arn:aws:dynamodb:us-west-2:111122223333:table/TestTable/stream/2015-05-11T21:21:33.291"); + map.put("checkpointInterval", "30000"); + map.put("backoffTime", "4000"); + map.put("numRetries", "3"); + map.put("receiveQueueSize", 2000); + map.put("applicationName", "My test application"); + map.put("initialPositionInStream", InitialPositionInStream.TRIM_HORIZON); + map.put("startAtTime", DAY); + + SourceContext sourceContext = Mockito.mock(SourceContext.class); + Mockito.when(sourceContext.getSecret("awsCredentialPluginParam")) + .thenReturn("{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}"); + DynamoDBSourceConfig config = DynamoDBSourceConfig.load(map, sourceContext); + + assertNotNull(config); + assertEquals(config.getAwsEndpoint(), "https://some.endpoint.aws"); + assertEquals(config.getAwsRegion(), "us-east-1"); + assertEquals(config.getAwsDynamodbStreamArn(), "arn:aws:dynamodb:us-west-2:111122223333:table/TestTable/stream/2015-05-11T21:21:33.291"); + assertEquals(config.getAwsCredentialPluginParam(), + "{\"accessKey\":\"myKey\",\"secretKey\":\"my-Secret\"}"); + assertEquals(config.getApplicationName(), "My test application"); + assertEquals(config.getCheckpointInterval(), 30000); + assertEquals(config.getBackoffTime(), 4000); + assertEquals(config.getNumRetries(), 3); + assertEquals(config.getReceiveQueueSize(), 2000); + assertEquals(config.getInitialPositionInStream(), InitialPositionInStream.TRIM_HORIZON); + + Calendar cal = Calendar.getInstance(); + cal.setTime(config.getStartAtTime()); + ZonedDateTime actual = ZonedDateTime.ofInstant(cal.toInstant(), ZoneOffset.UTC); + ZonedDateTime expected = ZonedDateTime.ofInstant(DAY.toInstant(), ZoneOffset.UTC); + assertEquals(actual, expected); + } + @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "empty aws-credential param") public final void missingCredentialsTest() throws Exception { @@ -121,7 +163,8 @@ public final void missingCredentialsTest() throws Exception { map.put("awsDynamodbStreamArn", "arn:aws:dynamodb:us-west-2:111122223333:table/TestTable/stream/2015-05-11T21:21:33.291"); DynamoDBSource source = new DynamoDBSource(); - source.open(map, null); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + source.open(map, sourceContext); } @Test(expectedExceptions = IllegalArgumentException.class, @@ -136,7 +179,8 @@ public final void missingStartTimeTest() throws Exception { map.put("initialPositionInStream", InitialPositionInStream.AT_TIMESTAMP); DynamoDBSource source = new DynamoDBSource(); - source.open(map, null); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + source.open(map, sourceContext); } private File getFile(String name) { diff --git a/pulsar-io/influxdb/pom.xml b/pulsar-io/influxdb/pom.xml index 21306c4f90d92..04325c478a3ce 100644 --- a/pulsar-io/influxdb/pom.xml +++ b/pulsar-io/influxdb/pom.xml @@ -32,6 +32,11 @@ Pulsar IO :: InfluxDB + + ${project.groupId} + pulsar-io-common + ${project.version} + ${project.groupId} pulsar-io-core diff --git a/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/InfluxDBGenericRecordSink.java b/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/InfluxDBGenericRecordSink.java index 5b51461fc7b8e..0d431f84c52f2 100644 --- a/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/InfluxDBGenericRecordSink.java +++ b/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/InfluxDBGenericRecordSink.java @@ -46,12 +46,12 @@ public class InfluxDBGenericRecordSink implements Sink { @Override public void open(Map map, SinkContext sinkContext) throws Exception { try { - val configV2 = InfluxDBSinkConfig.load(map); + val configV2 = InfluxDBSinkConfig.load(map, sinkContext); configV2.validate(); sink = new InfluxDBSink(); } catch (Exception e) { try { - val configV1 = org.apache.pulsar.io.influxdb.v1.InfluxDBSinkConfig.load(map); + val configV1 = org.apache.pulsar.io.influxdb.v1.InfluxDBSinkConfig.load(map, sinkContext); configV1.validate(); sink = new org.apache.pulsar.io.influxdb.v1.InfluxDBGenericRecordSink(); } catch (Exception e1) { diff --git a/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v1/InfluxDBAbstractSink.java b/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v1/InfluxDBAbstractSink.java index 06856bad80edc..217c5304b24f7 100644 --- a/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v1/InfluxDBAbstractSink.java +++ b/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v1/InfluxDBAbstractSink.java @@ -43,7 +43,7 @@ public abstract class InfluxDBAbstractSink extends BatchSink { @Override public void open(Map config, SinkContext sinkContext) throws Exception { - InfluxDBSinkConfig influxDBSinkConfig = InfluxDBSinkConfig.load(config); + InfluxDBSinkConfig influxDBSinkConfig = InfluxDBSinkConfig.load(config, sinkContext); influxDBSinkConfig.validate(); super.init(influxDBSinkConfig.getBatchTimeMs(), influxDBSinkConfig.getBatchSize()); diff --git a/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v1/InfluxDBSinkConfig.java b/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v1/InfluxDBSinkConfig.java index 9b7d8e1ce905d..4ae2cf1e4a3a1 100644 --- a/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v1/InfluxDBSinkConfig.java +++ b/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v1/InfluxDBSinkConfig.java @@ -27,6 +27,8 @@ import java.util.Map; import lombok.Data; import lombok.experimental.Accessors; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.core.annotations.FieldDoc; /** @@ -94,7 +96,7 @@ public class InfluxDBSinkConfig implements Serializable { @FieldDoc( required = false, - defaultValue = "1000L", + defaultValue = "1000", help = "The InfluxDB operation time in milliseconds") private long batchTimeMs = 1000L; @@ -110,14 +112,11 @@ public static InfluxDBSinkConfig load(String yamlFile) throws IOException { return mapper.readValue(new File(yamlFile), InfluxDBSinkConfig.class); } - public static InfluxDBSinkConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), InfluxDBSinkConfig.class); + public static InfluxDBSinkConfig load(Map map, SinkContext sinkContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, InfluxDBSinkConfig.class, sinkContext); } public void validate() { - Preconditions.checkNotNull(influxdbUrl, "influxdbUrl property not set."); - Preconditions.checkNotNull(database, "database property not set."); Preconditions.checkArgument(batchSize > 0, "batchSize must be a positive integer."); Preconditions.checkArgument(batchTimeMs > 0, "batchTimeMs must be a positive long."); } diff --git a/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSink.java b/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSink.java index 08f1ab2339992..0aa43570596af 100644 --- a/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSink.java +++ b/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSink.java @@ -49,7 +49,7 @@ public class InfluxDBSink extends BatchSink { @Override public void open(Map config, SinkContext sinkContext) throws Exception { - InfluxDBSinkConfig influxDBSinkConfig = InfluxDBSinkConfig.load(config); + InfluxDBSinkConfig influxDBSinkConfig = InfluxDBSinkConfig.load(config, sinkContext); influxDBSinkConfig.validate(); super.init(influxDBSinkConfig.getBatchTimeMs(), influxDBSinkConfig.getBatchSize()); diff --git a/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSinkConfig.java b/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSinkConfig.java index 899b00c002155..ea87ee66b90a3 100644 --- a/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSinkConfig.java +++ b/pulsar-io/influxdb/src/main/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSinkConfig.java @@ -27,6 +27,8 @@ import java.util.Map; import lombok.Data; import lombok.experimental.Accessors; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.core.annotations.FieldDoc; /** @@ -87,7 +89,7 @@ public class InfluxDBSinkConfig implements Serializable { @FieldDoc( required = false, - defaultValue = "1000L", + defaultValue = "1000", help = "The InfluxDB operation time in milliseconds") private long batchTimeMs = 1000; @@ -103,17 +105,11 @@ public static InfluxDBSinkConfig load(String yamlFile) throws IOException { return mapper.readValue(new File(yamlFile), InfluxDBSinkConfig.class); } - public static InfluxDBSinkConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), InfluxDBSinkConfig.class); + public static InfluxDBSinkConfig load(Map map, SinkContext sinkContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, InfluxDBSinkConfig.class, sinkContext); } public void validate() { - Preconditions.checkNotNull(influxdbUrl, "influxdbUrl property not set."); - Preconditions.checkNotNull(token, "token property not set."); - Preconditions.checkNotNull(organization, "organization property not set."); - Preconditions.checkNotNull(bucket, "bucket property not set."); - Preconditions.checkArgument(batchSize > 0, "batchSize must be a positive integer."); Preconditions.checkArgument(batchTimeMs > 0, "batchTimeMs must be a positive long."); } diff --git a/pulsar-io/influxdb/src/test/java/org/apache/pulsar/io/influxdb/v1/InfluxDBSinkConfigTest.java b/pulsar-io/influxdb/src/test/java/org/apache/pulsar/io/influxdb/v1/InfluxDBSinkConfigTest.java index 4493dcfb24854..10b1bfb624f49 100644 --- a/pulsar-io/influxdb/src/test/java/org/apache/pulsar/io/influxdb/v1/InfluxDBSinkConfigTest.java +++ b/pulsar-io/influxdb/src/test/java/org/apache/pulsar/io/influxdb/v1/InfluxDBSinkConfigTest.java @@ -18,7 +18,9 @@ */ package org.apache.pulsar.io.influxdb.v1; +import org.apache.pulsar.io.core.SinkContext; import org.influxdb.InfluxDB; +import org.mockito.Mockito; import org.testng.annotations.Test; import java.io.File; @@ -60,8 +62,11 @@ public final void loadFromMapTest() throws IOException { map.put("gzipEnable", "false"); map.put("batchTimeMs", "1000"); map.put("batchSize", "100"); + map.put("username", "admin"); + map.put("password", "admin"); - InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map, sinkContext); assertNotNull(config); assertEquals("http://localhost:8086", config.getInfluxdbUrl()); assertEquals("test_db", config.getDatabase()); @@ -71,6 +76,39 @@ public final void loadFromMapTest() throws IOException { assertEquals(Boolean.parseBoolean("false"), config.isGzipEnable()); assertEquals(Long.parseLong("1000"), config.getBatchTimeMs()); assertEquals(Integer.parseInt("100"), config.getBatchSize()); + assertEquals("admin", config.getUsername()); + assertEquals("admin", config.getPassword()); + } + + @Test + public final void loadFromMapCredentialFromSecretTest() throws IOException { + Map map = new HashMap<>(); + map.put("influxdbUrl", "http://localhost:8086"); + map.put("database", "test_db"); + map.put("consistencyLevel", "ONE"); + map.put("logLevel", "NONE"); + map.put("retentionPolicy", "autogen"); + map.put("gzipEnable", "false"); + map.put("batchTimeMs", "1000"); + map.put("batchSize", "100"); + + SinkContext sinkContext = Mockito.mock(SinkContext.class); + Mockito.when(sinkContext.getSecret("username")) + .thenReturn("admin"); + Mockito.when(sinkContext.getSecret("password")) + .thenReturn("admin"); + InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map, sinkContext); + assertNotNull(config); + assertEquals("http://localhost:8086", config.getInfluxdbUrl()); + assertEquals("test_db", config.getDatabase()); + assertEquals("ONE", config.getConsistencyLevel()); + assertEquals("NONE", config.getLogLevel()); + assertEquals("autogen", config.getRetentionPolicy()); + assertEquals(Boolean.parseBoolean("false"), config.isGzipEnable()); + assertEquals(Long.parseLong("1000"), config.getBatchTimeMs()); + assertEquals(Integer.parseInt("100"), config.getBatchSize()); + assertEquals("admin", config.getUsername()); + assertEquals("admin", config.getPassword()); } @Test @@ -85,12 +123,13 @@ public final void validValidateTest() throws IOException { map.put("batchTimeMs", "1000"); map.put("batchSize", "100"); - InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map, sinkContext); config.validate(); } - @Test(expectedExceptions = NullPointerException.class, - expectedExceptionsMessageRegExp = "influxdbUrl property not set.") + @Test(expectedExceptions = IllegalArgumentException.class, + expectedExceptionsMessageRegExp = "influxdbUrl cannot be null") public final void missingInfluxdbUrlValidateTest() throws IOException { Map map = new HashMap<>(); map.put("database", "test_db"); @@ -101,7 +140,8 @@ public final void missingInfluxdbUrlValidateTest() throws IOException { map.put("batchTimeMs", "1000"); map.put("batchSize", "100"); - InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map, sinkContext); config.validate(); } @@ -118,7 +158,8 @@ public final void invalidBatchSizeTest() throws IOException { map.put("batchTimeMs", "1000"); map.put("batchSize", "-100"); - InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map, sinkContext); config.validate(); } @@ -135,7 +176,8 @@ public final void invalidConsistencyLevelTest() throws IOException { map.put("batchTimeMs", "1000"); map.put("batchSize", "100"); - InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map, sinkContext); config.validate(); InfluxDB.ConsistencyLevel.valueOf(config.getConsistencyLevel().toUpperCase()); diff --git a/pulsar-io/influxdb/src/test/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSinkConfigTest.java b/pulsar-io/influxdb/src/test/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSinkConfigTest.java index df1f7fd29a637..d6cee1e308d2b 100644 --- a/pulsar-io/influxdb/src/test/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSinkConfigTest.java +++ b/pulsar-io/influxdb/src/test/java/org/apache/pulsar/io/influxdb/v2/InfluxDBSinkConfigTest.java @@ -24,6 +24,8 @@ import java.io.File; import java.util.HashMap; import java.util.Map; +import org.apache.pulsar.io.core.SinkContext; +import org.mockito.Mockito; import org.testng.annotations.Test; public class InfluxDBSinkConfigTest { @@ -58,18 +60,34 @@ private Map buildValidConfigMap() { public final void testLoadFromMap() throws Exception { Map map = buildValidConfigMap(); - InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map, sinkContext); assertNotNull(config); config.validate(); verifyValues(config); } - @Test(expectedExceptions = NullPointerException.class, - expectedExceptionsMessageRegExp = "influxdbUrl property not set.") + @Test + public final void testLoadFromMapCredentialFromSecret() throws Exception { + Map map = buildValidConfigMap(); + map.remove("token"); + + SinkContext sinkContext = Mockito.mock(SinkContext.class); + Mockito.when(sinkContext.getSecret("token")) + .thenReturn("xxxx"); + InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map, sinkContext); + assertNotNull(config); + config.validate(); + verifyValues(config); + } + + @Test(expectedExceptions = IllegalArgumentException.class, + expectedExceptionsMessageRegExp = "influxdbUrl cannot be null") public void testRequiredConfigMissing() throws Exception { Map map = buildValidConfigMap(); map.remove("influxdbUrl"); - InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map, sinkContext); config.validate(); } @@ -78,7 +96,8 @@ public void testRequiredConfigMissing() throws Exception { public void testBatchConfig() throws Exception { Map map = buildValidConfigMap(); map.put("batchSize", -1); - InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + InfluxDBSinkConfig config = InfluxDBSinkConfig.load(map, sinkContext); config.validate(); } diff --git a/pulsar-io/jdbc/core/pom.xml b/pulsar-io/jdbc/core/pom.xml index e3014fa04cd2d..109be01c3670a 100644 --- a/pulsar-io/jdbc/core/pom.xml +++ b/pulsar-io/jdbc/core/pom.xml @@ -32,6 +32,12 @@ Pulsar IO :: Jdbc :: Core + + ${project.groupId} + pulsar-io-common + ${project.version} + + ${project.groupId} pulsar-io-core diff --git a/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/JdbcAbstractSink.java b/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/JdbcAbstractSink.java index 4586fcebcf167..ca33b3cfdaba9 100644 --- a/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/JdbcAbstractSink.java +++ b/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/JdbcAbstractSink.java @@ -76,7 +76,7 @@ public abstract class JdbcAbstractSink implements Sink { @Override public void open(Map config, SinkContext sinkContext) throws Exception { - jdbcSinkConfig = JdbcSinkConfig.load(config); + jdbcSinkConfig = JdbcSinkConfig.load(config, sinkContext); jdbcSinkConfig.validate(); jdbcUrl = jdbcSinkConfig.getJdbcUrl(); diff --git a/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/JdbcSinkConfig.java b/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/JdbcSinkConfig.java index f798d94f7c35e..854d68381312c 100644 --- a/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/JdbcSinkConfig.java +++ b/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/JdbcSinkConfig.java @@ -26,6 +26,8 @@ import java.util.Map; import lombok.Data; import lombok.experimental.Accessors; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.core.annotations.FieldDoc; @Data @@ -145,9 +147,8 @@ public static JdbcSinkConfig load(String yamlFile) throws IOException { return mapper.readValue(new File(yamlFile), JdbcSinkConfig.class); } - public static JdbcSinkConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), JdbcSinkConfig.class); + public static JdbcSinkConfig load(Map map, SinkContext sinkContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, JdbcSinkConfig.class, sinkContext); } public void validate() { diff --git a/pulsar-io/kafka/pom.xml b/pulsar-io/kafka/pom.xml index 6c6904da34647..3084dd076f732 100644 --- a/pulsar-io/kafka/pom.xml +++ b/pulsar-io/kafka/pom.xml @@ -46,6 +46,11 @@ + + ${project.groupId} + pulsar-io-common + ${project.version} + ${project.groupId} diff --git a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSink.java b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSink.java index 5ceea4dec8dca..2bedba928b756 100644 --- a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSink.java +++ b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSink.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.Map; -import java.util.Objects; import java.util.Properties; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; @@ -79,10 +78,7 @@ protected Properties beforeCreateProducer(Properties props) { @Override public void open(Map config, SinkContext sinkContext) throws Exception { - kafkaSinkConfig = KafkaSinkConfig.load(config); - Objects.requireNonNull(kafkaSinkConfig.getTopic(), "Kafka topic is not set"); - Objects.requireNonNull(kafkaSinkConfig.getBootstrapServers(), "Kafka bootstrapServers is not set"); - Objects.requireNonNull(kafkaSinkConfig.getAcks(), "Kafka acks mode is not set"); + kafkaSinkConfig = KafkaSinkConfig.load(config, sinkContext); if (kafkaSinkConfig.getBatchSize() <= 0) { throw new IllegalArgumentException("Invalid Kafka Producer batchSize : " + kafkaSinkConfig.getBatchSize()); diff --git a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java index f8539518851aa..782f9d5d57dbb 100644 --- a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java +++ b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java @@ -66,7 +66,7 @@ public abstract class KafkaAbstractSource extends PushSource { @Override public void open(Map config, SourceContext sourceContext) throws Exception { - kafkaSourceConfig = KafkaSourceConfig.load(config); + kafkaSourceConfig = KafkaSourceConfig.load(config, sourceContext); Objects.requireNonNull(kafkaSourceConfig.getTopic(), "Kafka topic is not set"); Objects.requireNonNull(kafkaSourceConfig.getBootstrapServers(), "Kafka bootstrapServers is not set"); Objects.requireNonNull(kafkaSourceConfig.getGroupId(), "Kafka consumer group id is not set"); diff --git a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaSinkConfig.java b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaSinkConfig.java index 755b2c89c8f20..8f772a57d0c4d 100644 --- a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaSinkConfig.java +++ b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaSinkConfig.java @@ -26,6 +26,8 @@ import java.util.Map; import lombok.Data; import lombok.experimental.Accessors; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.core.annotations.FieldDoc; @Data @@ -84,12 +86,12 @@ public class KafkaSinkConfig implements Serializable { + " before considering a request complete. This controls the durability of records that are sent.") private String acks; @FieldDoc( - defaultValue = "16384L", + defaultValue = "16384", help = "The batch size that Kafka producer will attempt to batch records together" + " before sending them to brokers.") private long batchSize = 16384L; @FieldDoc( - defaultValue = "1048576L", + defaultValue = "1048576", help = "The maximum size of a Kafka request in bytes.") private long maxRequestSize = 1048576L; @@ -122,8 +124,7 @@ public static KafkaSinkConfig load(String yamlFile) throws IOException { return mapper.readValue(new File(yamlFile), KafkaSinkConfig.class); } - public static KafkaSinkConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), KafkaSinkConfig.class); + public static KafkaSinkConfig load(Map map, SinkContext sinkContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, KafkaSinkConfig.class, sinkContext); } } \ No newline at end of file diff --git a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaSourceConfig.java b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaSourceConfig.java index 5de60d2a028c8..3edfa27190339 100644 --- a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaSourceConfig.java +++ b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaSourceConfig.java @@ -27,6 +27,7 @@ import java.util.Map; import lombok.Data; import lombok.experimental.Accessors; +import org.apache.pulsar.io.core.SourceContext; import org.apache.pulsar.io.core.annotations.FieldDoc; @Data @@ -151,8 +152,14 @@ public static KafkaSourceConfig load(String yamlFile) throws IOException { return mapper.readValue(new File(yamlFile), KafkaSourceConfig.class); } - public static KafkaSourceConfig load(Map map) throws IOException { + public static KafkaSourceConfig load(Map map, SourceContext sourceContext) throws IOException { ObjectMapper mapper = new ObjectMapper(); + // since the KafkaSourceConfig requires the ACCEPT_EMPTY_STRING_AS_NULL_OBJECT feature + // We manually set the sensitive fields here instead of calling `IOConfigUtils.loadWithSecrets` + String sslTruststorePassword = sourceContext.getSecret("sslTruststorePassword"); + if (sslTruststorePassword != null) { + map.put("sslTruststorePassword", sslTruststorePassword); + } mapper.enable(DeserializationFeature.ACCEPT_EMPTY_STRING_AS_NULL_OBJECT); return mapper.readValue(mapper.writeValueAsString(map), KafkaSourceConfig.class); } diff --git a/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/sink/KafkaAbstractSinkTest.java b/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/sink/KafkaAbstractSinkTest.java index d59cdb1d9b63d..3e31eefe0e409 100644 --- a/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/sink/KafkaAbstractSinkTest.java +++ b/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/sink/KafkaAbstractSinkTest.java @@ -188,12 +188,12 @@ public PulsarClient getPulsarClient() { sink.close(); } }; - expectThrows(NullPointerException.class, "Kafka topic is not set", openAndClose); - config.put("topic", "topic_2"); - expectThrows(NullPointerException.class, "Kafka bootstrapServers is not set", openAndClose); + expectThrows(IllegalArgumentException.class, "bootstrapServers cannot be null", openAndClose); config.put("bootstrapServers", "localhost:6667"); - expectThrows(NullPointerException.class, "Kafka acks mode is not set", openAndClose); + expectThrows(IllegalArgumentException.class, "acks cannot be null", openAndClose); config.put("acks", "1"); + expectThrows(IllegalArgumentException.class, "topic cannot be null", openAndClose); + config.put("topic", "topic_2"); config.put("batchSize", "-1"); expectThrows(IllegalArgumentException.class, "Invalid Kafka Producer batchSize : -1", openAndClose); config.put("batchSize", "16384"); diff --git a/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java b/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java index 9e0fef87a2592..7675de0636e8a 100644 --- a/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java +++ b/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java @@ -108,19 +108,39 @@ public void testInvalidConfigWillThrownException() throws Exception { public void loadConsumerConfigPropertiesFromMapTest() throws Exception { Map config = new HashMap<>(); config.put("consumerConfigProperties", ""); - KafkaSourceConfig kafkaSourceConfig = KafkaSourceConfig.load(config); + config.put("bootstrapServers", "localhost:8080"); + config.put("groupId", "test-group"); + config.put("topic", "test-topic"); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + KafkaSourceConfig kafkaSourceConfig = KafkaSourceConfig.load(config, sourceContext); assertNotNull(kafkaSourceConfig); assertNull(kafkaSourceConfig.getConsumerConfigProperties()); config.put("consumerConfigProperties", null); - kafkaSourceConfig = KafkaSourceConfig.load(config); + kafkaSourceConfig = KafkaSourceConfig.load(config, sourceContext); assertNull(kafkaSourceConfig.getConsumerConfigProperties()); config.put("consumerConfigProperties", ImmutableMap.of("foo", "bar")); - kafkaSourceConfig = KafkaSourceConfig.load(config); + kafkaSourceConfig = KafkaSourceConfig.load(config, sourceContext); assertEquals(kafkaSourceConfig.getConsumerConfigProperties(), ImmutableMap.of("foo", "bar")); } + @Test + public void loadSensitiveFieldsFromSecretTest() throws Exception { + Map config = new HashMap<>(); + config.put("consumerConfigProperties", ""); + config.put("bootstrapServers", "localhost:8080"); + config.put("groupId", "test-group"); + config.put("topic", "test-topic"); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + Mockito.when(sourceContext.getSecret("sslTruststorePassword")) + .thenReturn("xxxx"); + KafkaSourceConfig kafkaSourceConfig = KafkaSourceConfig.load(config, sourceContext); + assertNotNull(kafkaSourceConfig); + assertNull(kafkaSourceConfig.getConsumerConfigProperties()); + assertEquals("xxxx", kafkaSourceConfig.getSslTruststorePassword()); + } + @Test public final void loadFromYamlFileTest() throws IOException { File yamlFile = getFile("kafkaSourceConfig.yaml"); diff --git a/pulsar-io/mongo/pom.xml b/pulsar-io/mongo/pom.xml index 1a65e61f035ce..bf4f3e0be572c 100644 --- a/pulsar-io/mongo/pom.xml +++ b/pulsar-io/mongo/pom.xml @@ -37,6 +37,11 @@ + + ${project.groupId} + pulsar-io-common + ${project.version} + ${project.parent.groupId} pulsar-io-core diff --git a/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoAbstractConfig.java b/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoAbstractConfig.java index 35c327ed82b99..74f077da62036 100644 --- a/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoAbstractConfig.java +++ b/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoAbstractConfig.java @@ -24,7 +24,6 @@ import java.io.Serializable; import lombok.Data; import lombok.experimental.Accessors; -import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.io.core.annotations.FieldDoc; /** @@ -42,6 +41,7 @@ public abstract class MongoAbstractConfig implements Serializable { @FieldDoc( required = true, + sensitive = true, // it may contain password defaultValue = "", help = "The URI of MongoDB that the connector connects to " + "(see: https://docs.mongodb.com/manual/reference/connection-string/)" @@ -95,7 +95,6 @@ public MongoAbstractConfig( } public void validate() { - checkArgument(!StringUtils.isEmpty(getMongoUri()), "Required MongoDB URI is not set."); checkArgument(getBatchSize() > 0, "batchSize must be a positive integer."); checkArgument(getBatchTimeMs() > 0, "batchTimeMs must be a positive long."); } diff --git a/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSink.java b/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSink.java index 2206d232eaf97..61d5aeb697e01 100644 --- a/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSink.java +++ b/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSink.java @@ -86,7 +86,7 @@ public MongoSink(Supplier clientProvider) { public void open(Map config, SinkContext sinkContext) throws Exception { log.info("Open MongoDB Sink"); - mongoSinkConfig = MongoSinkConfig.load(config); + mongoSinkConfig = MongoSinkConfig.load(config, sinkContext); mongoSinkConfig.validate(); if (clientProvider != null) { diff --git a/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSinkConfig.java b/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSinkConfig.java index 285f3c97bef1a..9431fe4910800 100644 --- a/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSinkConfig.java +++ b/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSinkConfig.java @@ -30,6 +30,8 @@ import lombok.EqualsAndHashCode; import lombok.experimental.Accessors; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SinkContext; /** * Configuration class for the MongoDB Sink Connectors. @@ -59,11 +61,8 @@ public static MongoSinkConfig load(String yamlFile) throws IOException { return cfg; } - public static MongoSinkConfig load(Map map) throws IOException { - final ObjectMapper mapper = new ObjectMapper(); - final MongoSinkConfig cfg = mapper.readValue(mapper.writeValueAsString(map), MongoSinkConfig.class); - - return cfg; + public static MongoSinkConfig load(Map map, SinkContext sinkContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, MongoSinkConfig.class, sinkContext); } @Override diff --git a/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSource.java b/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSource.java index 6ee95fc4cd4b5..68a31b461a51c 100644 --- a/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSource.java +++ b/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSource.java @@ -79,7 +79,7 @@ public MongoSource(Supplier clientProvider) { public void open(Map config, SourceContext sourceContext) throws Exception { log.info("Open MongoDB Source"); - mongoSourceConfig = MongoSourceConfig.load(config); + mongoSourceConfig = MongoSourceConfig.load(config, sourceContext); mongoSourceConfig.validate(); if (clientProvider != null) { diff --git a/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSourceConfig.java b/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSourceConfig.java index cf887a93bf3c3..1c0c7f4b3657a 100644 --- a/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSourceConfig.java +++ b/pulsar-io/mongo/src/main/java/org/apache/pulsar/io/mongodb/MongoSourceConfig.java @@ -29,6 +29,8 @@ import lombok.EqualsAndHashCode; import lombok.experimental.Accessors; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SourceContext; import org.apache.pulsar.io.core.annotations.FieldDoc; /** @@ -75,12 +77,8 @@ public static MongoSourceConfig load(String yamlFile) throws IOException { return cfg; } - public static MongoSourceConfig load(Map map) throws IOException { - final ObjectMapper mapper = new ObjectMapper(); - final MongoSourceConfig cfg = - mapper.readValue(mapper.writeValueAsString(map), MongoSourceConfig.class); - - return cfg; + public static MongoSourceConfig load(Map map, SourceContext sourceContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, MongoSourceConfig.class, sourceContext); } /** diff --git a/pulsar-io/mongo/src/test/java/org/apache/pulsar/io/mongodb/MongoSinkConfigTest.java b/pulsar-io/mongo/src/test/java/org/apache/pulsar/io/mongodb/MongoSinkConfigTest.java index b1166eac5722a..c86e45feb2348 100644 --- a/pulsar-io/mongo/src/test/java/org/apache/pulsar/io/mongodb/MongoSinkConfigTest.java +++ b/pulsar-io/mongo/src/test/java/org/apache/pulsar/io/mongodb/MongoSinkConfigTest.java @@ -19,6 +19,8 @@ package org.apache.pulsar.io.mongodb; import java.util.Map; +import org.apache.pulsar.io.core.SinkContext; +import org.mockito.Mockito; import org.testng.annotations.Test; import java.io.File; @@ -34,7 +36,27 @@ public void testLoadMapConfig() throws IOException { commonConfigMap.put("batchSize", TestHelper.BATCH_SIZE); commonConfigMap.put("batchTimeMs", TestHelper.BATCH_TIME); - final MongoSinkConfig cfg = MongoSinkConfig.load(commonConfigMap); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + final MongoSinkConfig cfg = MongoSinkConfig.load(commonConfigMap, sinkContext); + + assertEquals(cfg.getMongoUri(), TestHelper.URI); + assertEquals(cfg.getDatabase(), TestHelper.DB); + assertEquals(cfg.getCollection(), TestHelper.COLL); + assertEquals(cfg.getBatchSize(), TestHelper.BATCH_SIZE); + assertEquals(cfg.getBatchTimeMs(), TestHelper.BATCH_TIME); + } + + @Test + public void testLoadMapConfigUrlFromSecret() throws IOException { + final Map commonConfigMap = TestHelper.createCommonConfigMap(); + commonConfigMap.put("batchSize", TestHelper.BATCH_SIZE); + commonConfigMap.put("batchTimeMs", TestHelper.BATCH_TIME); + commonConfigMap.remove("mongoUri"); + + SinkContext sinkContext = Mockito.mock(SinkContext.class); + Mockito.when(sinkContext.getSecret("mongoUri")) + .thenReturn(TestHelper.URI); + final MongoSinkConfig cfg = MongoSinkConfig.load(commonConfigMap, sinkContext); assertEquals(cfg.getMongoUri(), TestHelper.URI); assertEquals(cfg.getDatabase(), TestHelper.DB); @@ -44,12 +66,13 @@ public void testLoadMapConfig() throws IOException { } @Test(expectedExceptions = IllegalArgumentException.class, - expectedExceptionsMessageRegExp = "Required MongoDB URI is not set.") + expectedExceptionsMessageRegExp = "mongoUri cannot be null") public void testBadMongoUri() throws IOException { final Map configMap = TestHelper.createCommonConfigMap(); TestHelper.removeMongoUri(configMap); - final MongoSinkConfig cfg = MongoSinkConfig.load(configMap); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + final MongoSinkConfig cfg = MongoSinkConfig.load(configMap, sinkContext); cfg.validate(); } @@ -60,7 +83,8 @@ public void testBadDatabase() throws IOException { final Map configMap = TestHelper.createCommonConfigMap(); TestHelper.removeDatabase(configMap); - final MongoSinkConfig cfg = MongoSinkConfig.load(configMap); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + final MongoSinkConfig cfg = MongoSinkConfig.load(configMap, sinkContext); cfg.validate(); } @@ -71,7 +95,8 @@ public void testBadCollection() throws IOException { final Map configMap = TestHelper.createCommonConfigMap(); TestHelper.removeCollection(configMap); - final MongoSinkConfig cfg = MongoSinkConfig.load(configMap); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + final MongoSinkConfig cfg = MongoSinkConfig.load(configMap, sinkContext); cfg.validate(); } @@ -82,7 +107,8 @@ public void testBadBatchSize() throws IOException { final Map configMap = TestHelper.createCommonConfigMap(); TestHelper.putBatchSize(configMap, 0); - final MongoSinkConfig cfg = MongoSinkConfig.load(configMap); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + final MongoSinkConfig cfg = MongoSinkConfig.load(configMap, sinkContext); cfg.validate(); } @@ -93,7 +119,8 @@ public void testBadBatchTime() throws IOException { final Map configMap = TestHelper.createCommonConfigMap(); TestHelper.putBatchTime(configMap, 0L); - final MongoSinkConfig cfg = MongoSinkConfig.load(configMap); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + final MongoSinkConfig cfg = MongoSinkConfig.load(configMap, sinkContext); cfg.validate(); } diff --git a/pulsar-io/mongo/src/test/java/org/apache/pulsar/io/mongodb/MongoSourceConfigTest.java b/pulsar-io/mongo/src/test/java/org/apache/pulsar/io/mongodb/MongoSourceConfigTest.java index e7fd01549b033..528cd0237ef16 100644 --- a/pulsar-io/mongo/src/test/java/org/apache/pulsar/io/mongodb/MongoSourceConfigTest.java +++ b/pulsar-io/mongo/src/test/java/org/apache/pulsar/io/mongodb/MongoSourceConfigTest.java @@ -23,6 +23,8 @@ import java.io.File; import java.io.IOException; import java.util.Map; +import org.apache.pulsar.io.core.SourceContext; +import org.mockito.Mockito; import org.testng.annotations.Test; public class MongoSourceConfigTest { @@ -32,7 +34,27 @@ public void testLoadMapConfig() throws IOException { final Map configMap = TestHelper.createCommonConfigMap(); TestHelper.putSyncType(configMap, TestHelper.SYNC_TYPE); - final MongoSourceConfig cfg = MongoSourceConfig.load(configMap); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + final MongoSourceConfig cfg = MongoSourceConfig.load(configMap, sourceContext); + + assertEquals(cfg.getMongoUri(), TestHelper.URI); + assertEquals(cfg.getDatabase(), TestHelper.DB); + assertEquals(cfg.getCollection(), TestHelper.COLL); + assertEquals(cfg.getSyncType(), TestHelper.SYNC_TYPE); + assertEquals(cfg.getBatchSize(), TestHelper.BATCH_SIZE); + assertEquals(cfg.getBatchTimeMs(), TestHelper.BATCH_TIME); + } + + @Test + public void testLoadMapConfigUriFromSecret() throws IOException { + final Map configMap = TestHelper.createCommonConfigMap(); + TestHelper.putSyncType(configMap, TestHelper.SYNC_TYPE); + configMap.remove("mongoUri"); + + SourceContext sourceContext = Mockito.mock(SourceContext.class); + Mockito.when(sourceContext.getSecret("mongoUri")) + .thenReturn(TestHelper.URI); + final MongoSourceConfig cfg = MongoSourceConfig.load(configMap, sourceContext); assertEquals(cfg.getMongoUri(), TestHelper.URI); assertEquals(cfg.getDatabase(), TestHelper.DB); @@ -43,12 +65,13 @@ public void testLoadMapConfig() throws IOException { } @Test(expectedExceptions = IllegalArgumentException.class, - expectedExceptionsMessageRegExp = "Required MongoDB URI is not set.") + expectedExceptionsMessageRegExp = "mongoUri cannot be null") public void testBadMongoUri() throws IOException { final Map configMap = TestHelper.createCommonConfigMap(); TestHelper.removeMongoUri(configMap); - final MongoSourceConfig cfg = MongoSourceConfig.load(configMap); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + final MongoSourceConfig cfg = MongoSourceConfig.load(configMap, sourceContext); cfg.validate(); } @@ -61,7 +84,8 @@ public void testBadSyncType() throws IOException { final Map configMap = TestHelper.createCommonConfigMap(); TestHelper.putSyncType(configMap, "wrong_sync_type_str"); - final MongoSourceConfig cfg = MongoSourceConfig.load(configMap); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + final MongoSourceConfig cfg = MongoSourceConfig.load(configMap, sourceContext); cfg.validate(); } @@ -72,7 +96,8 @@ public void testBadBatchSize() throws IOException { final Map configMap = TestHelper.createCommonConfigMap(); TestHelper.putBatchSize(configMap, 0); - final MongoSourceConfig cfg = MongoSourceConfig.load(configMap); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + final MongoSourceConfig cfg = MongoSourceConfig.load(configMap, sourceContext); cfg.validate(); } @@ -83,7 +108,8 @@ public void testBadBatchTime() throws IOException { final Map configMap = TestHelper.createCommonConfigMap(); TestHelper.putBatchTime(configMap, 0L); - final MongoSourceConfig cfg = MongoSourceConfig.load(configMap); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + final MongoSourceConfig cfg = MongoSourceConfig.load(configMap, sourceContext); cfg.validate(); } diff --git a/pulsar-io/rabbitmq/pom.xml b/pulsar-io/rabbitmq/pom.xml index 6c400a7521adb..d10204dfedce3 100644 --- a/pulsar-io/rabbitmq/pom.xml +++ b/pulsar-io/rabbitmq/pom.xml @@ -32,6 +32,11 @@ + + ${project.groupId} + pulsar-io-common + ${project.version} + ${project.groupId} pulsar-io-core diff --git a/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSink.java b/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSink.java index f317a35734e69..89192c42346e8 100644 --- a/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSink.java +++ b/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSink.java @@ -53,7 +53,7 @@ public class RabbitMQSink implements Sink { @Override public void open(Map config, SinkContext sinkContext) throws Exception { - rabbitMQSinkConfig = RabbitMQSinkConfig.load(config); + rabbitMQSinkConfig = RabbitMQSinkConfig.load(config, sinkContext); rabbitMQSinkConfig.validate(); ConnectionFactory connectionFactory = rabbitMQSinkConfig.createConnectionFactory(); diff --git a/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSinkConfig.java b/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSinkConfig.java index c1f8d6b8ad3d3..39f97e5e460c8 100644 --- a/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSinkConfig.java +++ b/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSinkConfig.java @@ -20,7 +20,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; -import com.google.common.base.Preconditions; import java.io.File; import java.io.IOException; import java.io.Serializable; @@ -28,6 +27,8 @@ import lombok.Data; import lombok.EqualsAndHashCode; import lombok.experimental.Accessors; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.core.annotations.FieldDoc; @Data @@ -60,14 +61,12 @@ public static RabbitMQSinkConfig load(String yamlFile) throws IOException { return mapper.readValue(new File(yamlFile), RabbitMQSinkConfig.class); } - public static RabbitMQSinkConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), RabbitMQSinkConfig.class); + public static RabbitMQSinkConfig load(Map map, SinkContext sinkContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, RabbitMQSinkConfig.class, sinkContext); } @Override public void validate() { super.validate(); - Preconditions.checkNotNull(exchangeName, "exchangeName property not set."); } } diff --git a/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSource.java b/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSource.java index d15108c4d8288..b0b7ef31b08de 100644 --- a/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSource.java +++ b/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSource.java @@ -54,7 +54,7 @@ public class RabbitMQSource extends PushSource { @Override public void open(Map config, SourceContext sourceContext) throws Exception { - rabbitMQSourceConfig = RabbitMQSourceConfig.load(config); + rabbitMQSourceConfig = RabbitMQSourceConfig.load(config, sourceContext); rabbitMQSourceConfig.validate(); ConnectionFactory connectionFactory = rabbitMQSourceConfig.createConnectionFactory(); diff --git a/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSourceConfig.java b/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSourceConfig.java index f24018e70da13..01e23a7146080 100644 --- a/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSourceConfig.java +++ b/pulsar-io/rabbitmq/src/main/java/org/apache/pulsar/io/rabbitmq/RabbitMQSourceConfig.java @@ -28,6 +28,8 @@ import lombok.Data; import lombok.EqualsAndHashCode; import lombok.experimental.Accessors; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SourceContext; import org.apache.pulsar.io.core.annotations.FieldDoc; @Data @@ -66,9 +68,8 @@ public static RabbitMQSourceConfig load(String yamlFile) throws IOException { return mapper.readValue(new File(yamlFile), RabbitMQSourceConfig.class); } - public static RabbitMQSourceConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), RabbitMQSourceConfig.class); + public static RabbitMQSourceConfig load(Map map, SourceContext sourceContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, RabbitMQSourceConfig.class, sourceContext); } @Override diff --git a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/sink/RabbitMQSinkConfigTest.java b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/sink/RabbitMQSinkConfigTest.java index 3d4fd6f46e16f..8706cb567524f 100644 --- a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/sink/RabbitMQSinkConfigTest.java +++ b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/sink/RabbitMQSinkConfigTest.java @@ -18,7 +18,9 @@ */ package org.apache.pulsar.io.rabbitmq.sink; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.rabbitmq.RabbitMQSinkConfig; +import org.mockito.Mockito; import org.testng.annotations.Test; import java.io.File; @@ -71,7 +73,45 @@ public final void loadFromMapTest() throws IOException { map.put("exchangeName", "test-exchange"); map.put("exchangeType", "test-exchange-type"); - RabbitMQSinkConfig config = RabbitMQSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + RabbitMQSinkConfig config = RabbitMQSinkConfig.load(map, sinkContext); + assertNotNull(config); + assertEquals(config.getHost(), "localhost"); + assertEquals(config.getPort(), Integer.parseInt("5673")); + assertEquals(config.getVirtualHost(), "/"); + assertEquals(config.getUsername(), "guest"); + assertEquals(config.getPassword(), "guest"); + assertEquals(config.getConnectionName(), "test-connection"); + assertEquals(config.getRequestedChannelMax(), Integer.parseInt("0")); + assertEquals(config.getRequestedFrameMax(), Integer.parseInt("0")); + assertEquals(config.getConnectionTimeout(), Integer.parseInt("60000")); + assertEquals(config.getHandshakeTimeout(), Integer.parseInt("10000")); + assertEquals(config.getRequestedHeartbeat(), Integer.parseInt("60")); + assertEquals(config.getExchangeName(), "test-exchange"); + assertEquals(config.getExchangeType(), "test-exchange-type"); + } + + @Test + public final void loadFromMapCredentialsFromSecretTest() throws IOException { + Map map = new HashMap<>(); + map.put("host", "localhost"); + map.put("port", "5673"); + map.put("virtualHost", "/"); + map.put("connectionName", "test-connection"); + map.put("requestedChannelMax", "0"); + map.put("requestedFrameMax", "0"); + map.put("connectionTimeout", "60000"); + map.put("handshakeTimeout", "10000"); + map.put("requestedHeartbeat", "60"); + map.put("exchangeName", "test-exchange"); + map.put("exchangeType", "test-exchange-type"); + + SinkContext sinkContext = Mockito.mock(SinkContext.class); + Mockito.when(sinkContext.getSecret("username")) + .thenReturn("guest"); + Mockito.when(sinkContext.getSecret("password")) + .thenReturn("guest"); + RabbitMQSinkConfig config = RabbitMQSinkConfig.load(map, sinkContext); assertNotNull(config); assertEquals(config.getHost(), "localhost"); assertEquals(config.getPort(), Integer.parseInt("5673")); @@ -105,12 +145,13 @@ public final void validValidateTest() throws IOException { map.put("exchangeName", "test-exchange"); map.put("exchangeType", "test-exchange-type"); - RabbitMQSinkConfig config = RabbitMQSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + RabbitMQSinkConfig config = RabbitMQSinkConfig.load(map, sinkContext); config.validate(); } - @Test(expectedExceptions = NullPointerException.class, - expectedExceptionsMessageRegExp = "exchangeName property not set.") + @Test(expectedExceptions = IllegalArgumentException.class, + expectedExceptionsMessageRegExp = "exchangeName cannot be null") public final void missingExchangeValidateTest() throws IOException { Map map = new HashMap<>(); map.put("host", "localhost"); @@ -126,7 +167,8 @@ public final void missingExchangeValidateTest() throws IOException { map.put("requestedHeartbeat", "60"); map.put("exchangeType", "test-exchange-type"); - RabbitMQSinkConfig config = RabbitMQSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + RabbitMQSinkConfig config = RabbitMQSinkConfig.load(map, sinkContext); config.validate(); } diff --git a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/source/RabbitMQSourceConfigTest.java b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/source/RabbitMQSourceConfigTest.java index c33e0070c6fd0..43a90062fa453 100644 --- a/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/source/RabbitMQSourceConfigTest.java +++ b/pulsar-io/rabbitmq/src/test/java/org/apache/pulsar/io/rabbitmq/source/RabbitMQSourceConfigTest.java @@ -18,7 +18,9 @@ */ package org.apache.pulsar.io.rabbitmq.source; +import org.apache.pulsar.io.core.SourceContext; import org.apache.pulsar.io.rabbitmq.RabbitMQSourceConfig; +import org.mockito.Mockito; import org.testng.annotations.Test; import java.io.File; @@ -76,7 +78,50 @@ public final void loadFromMapTest() throws IOException { map.put("prefetchGlobal", "false"); map.put("passive", "true"); - RabbitMQSourceConfig config = RabbitMQSourceConfig.load(map); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + RabbitMQSourceConfig config = RabbitMQSourceConfig.load(map, sourceContext); + assertNotNull(config); + assertEquals("localhost", config.getHost()); + assertEquals(Integer.parseInt("5672"), config.getPort()); + assertEquals("/", config.getVirtualHost()); + assertEquals("guest", config.getUsername()); + assertEquals("guest", config.getPassword()); + assertEquals("test-queue", config.getQueueName()); + assertEquals("test-connection", config.getConnectionName()); + assertEquals(Integer.parseInt("0"), config.getRequestedChannelMax()); + assertEquals(Integer.parseInt("0"), config.getRequestedFrameMax()); + assertEquals(Integer.parseInt("60000"), config.getConnectionTimeout()); + assertEquals(Integer.parseInt("10000"), config.getHandshakeTimeout()); + assertEquals(Integer.parseInt("60"), config.getRequestedHeartbeat()); + assertEquals(Integer.parseInt("0"), config.getPrefetchCount()); + assertEquals(false, config.isPrefetchGlobal()); + assertEquals(false, config.isPrefetchGlobal()); + assertEquals(true, config.isPassive()); + } + + @Test + public final void loadFromMapCredentialsFromSecretTest() throws IOException { + Map map = new HashMap<>(); + map.put("host", "localhost"); + map.put("port", "5672"); + map.put("virtualHost", "/"); + map.put("queueName", "test-queue"); + map.put("connectionName", "test-connection"); + map.put("requestedChannelMax", "0"); + map.put("requestedFrameMax", "0"); + map.put("connectionTimeout", "60000"); + map.put("handshakeTimeout", "10000"); + map.put("requestedHeartbeat", "60"); + map.put("prefetchCount", "0"); + map.put("prefetchGlobal", "false"); + map.put("passive", "true"); + + SourceContext sourceContext = Mockito.mock(SourceContext.class); + Mockito.when(sourceContext.getSecret("username")) + .thenReturn("guest"); + Mockito.when(sourceContext.getSecret("password")) + .thenReturn("guest"); + RabbitMQSourceConfig config = RabbitMQSourceConfig.load(map, sourceContext); assertNotNull(config); assertEquals("localhost", config.getHost()); assertEquals(Integer.parseInt("5672"), config.getPort()); @@ -115,12 +160,13 @@ public final void validValidateTest() throws IOException { map.put("prefetchGlobal", "false"); map.put("passive", "false"); - RabbitMQSourceConfig config = RabbitMQSourceConfig.load(map); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + RabbitMQSourceConfig config = RabbitMQSourceConfig.load(map, sourceContext); config.validate(); } - @Test(expectedExceptions = NullPointerException.class, - expectedExceptionsMessageRegExp = "host property not set.") + @Test(expectedExceptions = IllegalArgumentException.class, + expectedExceptionsMessageRegExp = "host cannot be null") public final void missingHostValidateTest() throws IOException { Map map = new HashMap<>(); map.put("port", "5672"); @@ -138,7 +184,8 @@ public final void missingHostValidateTest() throws IOException { map.put("prefetchGlobal", "false"); map.put("passive", "false"); - RabbitMQSourceConfig config = RabbitMQSourceConfig.load(map); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + RabbitMQSourceConfig config = RabbitMQSourceConfig.load(map, sourceContext); config.validate(); } @@ -162,7 +209,8 @@ public final void invalidPrefetchCountTest() throws IOException { map.put("prefetchGlobal", "false"); map.put("passive", "false"); - RabbitMQSourceConfig config = RabbitMQSourceConfig.load(map); + SourceContext sourceContext = Mockito.mock(SourceContext.class); + RabbitMQSourceConfig config = RabbitMQSourceConfig.load(map, sourceContext); config.validate(); } diff --git a/pulsar-io/redis/pom.xml b/pulsar-io/redis/pom.xml index 5926a70f6a2ec..e828b34d74066 100644 --- a/pulsar-io/redis/pom.xml +++ b/pulsar-io/redis/pom.xml @@ -32,6 +32,11 @@ Pulsar IO :: Redis + + ${project.groupId} + pulsar-io-common + ${project.version} + ${project.groupId} pulsar-io-core diff --git a/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/RedisAbstractConfig.java b/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/RedisAbstractConfig.java index 978e7de31a51c..89ec684dded72 100644 --- a/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/RedisAbstractConfig.java +++ b/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/RedisAbstractConfig.java @@ -88,13 +88,11 @@ public class RedisAbstractConfig implements Serializable { @FieldDoc( required = false, - defaultValue = "10000L", + defaultValue = "10000", help = "The amount of time in milliseconds to wait before timing out when connecting") private long connectTimeout = 10000L; public void validate() { - Preconditions.checkNotNull(redisHosts, "redisHosts property not set."); - Preconditions.checkNotNull(redisDatabase, "redisDatabase property not set."); Preconditions.checkNotNull(clientMode, "clientMode property not set."); } @@ -105,7 +103,6 @@ public enum ClientMode { public List getHostAndPorts() { List hostAndPorts = Lists.newArrayList(); - Preconditions.checkNotNull(redisHosts, "redisHosts property not set."); String[] hosts = StringUtils.split(redisHosts, ","); for (String host : hosts) { HostAndPort hostAndPort = HostAndPort.fromString(host); diff --git a/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/sink/RedisSink.java b/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/sink/RedisSink.java index bff0a5c2da592..ebd6e9dbab272 100644 --- a/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/sink/RedisSink.java +++ b/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/sink/RedisSink.java @@ -68,7 +68,7 @@ public class RedisSink implements Sink { public void open(Map config, SinkContext sinkContext) throws Exception { log.info("Open Redis Sink"); - redisSinkConfig = RedisSinkConfig.load(config); + redisSinkConfig = RedisSinkConfig.load(config, sinkContext); redisSinkConfig.validate(); redisSession = RedisSession.create(redisSinkConfig); diff --git a/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/sink/RedisSinkConfig.java b/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/sink/RedisSinkConfig.java index a9db66812a475..f7a70cb65a826 100644 --- a/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/sink/RedisSinkConfig.java +++ b/pulsar-io/redis/src/main/java/org/apache/pulsar/io/redis/sink/RedisSinkConfig.java @@ -28,6 +28,8 @@ import lombok.Data; import lombok.EqualsAndHashCode; import lombok.experimental.Accessors; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.core.annotations.FieldDoc; import org.apache.pulsar.io.redis.RedisAbstractConfig; @@ -40,13 +42,13 @@ public class RedisSinkConfig extends RedisAbstractConfig implements Serializable @FieldDoc( required = false, - defaultValue = "10000L", + defaultValue = "10000", help = "The amount of time in milliseconds before an operation is marked as timed out") private long operationTimeout = 10000L; @FieldDoc( required = false, - defaultValue = "1000L", + defaultValue = "1000", help = "The Redis operation time in milliseconds") private long batchTimeMs = 1000L; @@ -62,9 +64,8 @@ public static RedisSinkConfig load(String yamlFile) throws IOException { return mapper.readValue(new File(yamlFile), RedisSinkConfig.class); } - public static RedisSinkConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), RedisSinkConfig.class); + public static RedisSinkConfig load(Map map, SinkContext sinkContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, RedisSinkConfig.class, sinkContext); } @Override diff --git a/pulsar-io/redis/src/test/java/org/apache/pulsar/io/redis/sink/RedisSinkConfigTest.java b/pulsar-io/redis/src/test/java/org/apache/pulsar/io/redis/sink/RedisSinkConfigTest.java index 1316d0994a1cd..39fc6e540c242 100644 --- a/pulsar-io/redis/src/test/java/org/apache/pulsar/io/redis/sink/RedisSinkConfigTest.java +++ b/pulsar-io/redis/src/test/java/org/apache/pulsar/io/redis/sink/RedisSinkConfigTest.java @@ -18,7 +18,9 @@ */ package org.apache.pulsar.io.redis.sink; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.redis.RedisAbstractConfig; +import org.mockito.Mockito; import org.testng.annotations.Test; import java.io.File; @@ -62,7 +64,34 @@ public final void loadFromMapTest() throws IOException { map.put("batchTimeMs", "1000"); map.put("connectTimeout", "3000"); - RedisSinkConfig config = RedisSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + RedisSinkConfig config = RedisSinkConfig.load(map, sinkContext); + assertNotNull(config); + assertEquals(config.getRedisHosts(), "localhost:6379"); + assertEquals(config.getRedisPassword(), "fake@123"); + assertEquals(config.getRedisDatabase(), Integer.parseInt("1")); + assertEquals(config.getClientMode(), "Standalone"); + assertEquals(config.getOperationTimeout(), Long.parseLong("2000")); + assertEquals(config.getBatchSize(), Integer.parseInt("100")); + assertEquals(config.getBatchTimeMs(), Long.parseLong("1000")); + assertEquals(config.getConnectTimeout(), Long.parseLong("3000")); + } + + @Test + public final void loadFromMapCredentialsFromSecretTest() throws IOException { + Map map = new HashMap(); + map.put("redisHosts", "localhost:6379"); + map.put("redisDatabase", "1"); + map.put("clientMode", "Standalone"); + map.put("operationTimeout", "2000"); + map.put("batchSize", "100"); + map.put("batchTimeMs", "1000"); + map.put("connectTimeout", "3000"); + + SinkContext sinkContext = Mockito.mock(SinkContext.class); + Mockito.when(sinkContext.getSecret("redisPassword")) + .thenReturn("fake@123"); + RedisSinkConfig config = RedisSinkConfig.load(map, sinkContext); assertNotNull(config); assertEquals(config.getRedisHosts(), "localhost:6379"); assertEquals(config.getRedisPassword(), "fake@123"); @@ -86,12 +115,13 @@ public final void validValidateTest() throws IOException { map.put("batchTimeMs", "1000"); map.put("connectTimeout", "3000"); - RedisSinkConfig config = RedisSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + RedisSinkConfig config = RedisSinkConfig.load(map, sinkContext); config.validate(); } - @Test(expectedExceptions = NullPointerException.class, - expectedExceptionsMessageRegExp = "redisHosts property not set.") + @Test(expectedExceptions = IllegalArgumentException.class, + expectedExceptionsMessageRegExp = "redisHosts cannot be null") public final void missingValidValidateTableNameTest() throws IOException { Map map = new HashMap(); map.put("redisPassword", "fake@123"); @@ -102,7 +132,8 @@ public final void missingValidValidateTableNameTest() throws IOException { map.put("batchTimeMs", "1000"); map.put("connectTimeout", "3000"); - RedisSinkConfig config = RedisSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + RedisSinkConfig config = RedisSinkConfig.load(map, sinkContext); config.validate(); } @@ -119,7 +150,8 @@ public final void invalidBatchTimeMsTest() throws IOException { map.put("batchTimeMs", "-100"); map.put("connectTimeout", "3000"); - RedisSinkConfig config = RedisSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + RedisSinkConfig config = RedisSinkConfig.load(map, sinkContext); config.validate(); } @@ -136,7 +168,8 @@ public final void invalidClientModeTest() throws IOException { map.put("batchTimeMs", "1000"); map.put("connectTimeout", "3000"); - RedisSinkConfig config = RedisSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + RedisSinkConfig config = RedisSinkConfig.load(map, sinkContext); config.validate(); RedisAbstractConfig.ClientMode.valueOf(config.getClientMode().toUpperCase()); diff --git a/pulsar-io/redis/src/test/java/org/apache/pulsar/io/redis/sink/RedisSinkTest.java b/pulsar-io/redis/src/test/java/org/apache/pulsar/io/redis/sink/RedisSinkTest.java index 214151345b42c..2b407fafa5e04 100644 --- a/pulsar-io/redis/src/test/java/org/apache/pulsar/io/redis/sink/RedisSinkTest.java +++ b/pulsar-io/redis/src/test/java/org/apache/pulsar/io/redis/sink/RedisSinkTest.java @@ -21,7 +21,9 @@ import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.functions.instance.SinkRecord; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.redis.EmbeddedRedisUtils; +import org.mockito.Mockito; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -66,7 +68,8 @@ public void TestOpenAndWriteSink() throws Exception { Record record = build("fakeTopic", "fakeKey", "fakeValue"); // open should success - sink.open(configs, null); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + sink.open(configs, sinkContext); // write should success. sink.write(record); diff --git a/pulsar-io/solr/pom.xml b/pulsar-io/solr/pom.xml index ca3735121898f..2c3e213d8542a 100644 --- a/pulsar-io/solr/pom.xml +++ b/pulsar-io/solr/pom.xml @@ -36,6 +36,11 @@ Pulsar IO :: Solr + + ${project.groupId} + pulsar-io-common + ${project.version} + ${project.parent.groupId} pulsar-io-core diff --git a/pulsar-io/solr/src/main/java/org/apache/pulsar/io/solr/SolrAbstractSink.java b/pulsar-io/solr/src/main/java/org/apache/pulsar/io/solr/SolrAbstractSink.java index de9cdb4a9d82a..202c782c14c49 100644 --- a/pulsar-io/solr/src/main/java/org/apache/pulsar/io/solr/SolrAbstractSink.java +++ b/pulsar-io/solr/src/main/java/org/apache/pulsar/io/solr/SolrAbstractSink.java @@ -48,7 +48,7 @@ public abstract class SolrAbstractSink implements Sink { @Override public void open(Map config, SinkContext sinkContext) throws Exception { - solrSinkConfig = SolrSinkConfig.load(config); + solrSinkConfig = SolrSinkConfig.load(config, sinkContext); solrSinkConfig.validate(); enableBasicAuth = !Strings.isNullOrEmpty(solrSinkConfig.getUsername()); diff --git a/pulsar-io/solr/src/main/java/org/apache/pulsar/io/solr/SolrSinkConfig.java b/pulsar-io/solr/src/main/java/org/apache/pulsar/io/solr/SolrSinkConfig.java index 02733d230bdcb..daa93a366b110 100644 --- a/pulsar-io/solr/src/main/java/org/apache/pulsar/io/solr/SolrSinkConfig.java +++ b/pulsar-io/solr/src/main/java/org/apache/pulsar/io/solr/SolrSinkConfig.java @@ -27,6 +27,8 @@ import java.util.Map; import lombok.Data; import lombok.experimental.Accessors; +import org.apache.pulsar.io.common.IOConfigUtils; +import org.apache.pulsar.io.core.SinkContext; import org.apache.pulsar.io.core.annotations.FieldDoc; /** @@ -84,9 +86,8 @@ public static SolrSinkConfig load(String yamlFile) throws IOException { return mapper.readValue(new File(yamlFile), SolrSinkConfig.class); } - public static SolrSinkConfig load(Map map) throws IOException { - ObjectMapper mapper = new ObjectMapper(); - return mapper.readValue(mapper.writeValueAsString(map), SolrSinkConfig.class); + public static SolrSinkConfig load(Map map, SinkContext sinkContext) throws IOException { + return IOConfigUtils.loadWithSecrets(map, SolrSinkConfig.class, sinkContext); } public void validate() { diff --git a/pulsar-io/solr/src/test/java/org/apache/pulsar/io/solr/SolrSinkConfigTest.java b/pulsar-io/solr/src/test/java/org/apache/pulsar/io/solr/SolrSinkConfigTest.java index 42d2121dbfcbd..2c2447a637d35 100644 --- a/pulsar-io/solr/src/test/java/org/apache/pulsar/io/solr/SolrSinkConfigTest.java +++ b/pulsar-io/solr/src/test/java/org/apache/pulsar/io/solr/SolrSinkConfigTest.java @@ -19,6 +19,8 @@ package org.apache.pulsar.io.solr; import com.google.common.collect.Lists; +import org.apache.pulsar.io.core.SinkContext; +import org.mockito.Mockito; import org.testng.annotations.Test; import java.io.File; @@ -61,7 +63,31 @@ public final void loadFromMapTest() throws IOException { map.put("username", "fakeuser"); map.put("password", "fake@123"); - SolrSinkConfig config = SolrSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + SolrSinkConfig config = SolrSinkConfig.load(map, sinkContext); + assertNotNull(config); + assertEquals(config.getSolrUrl(), "localhost:2181,localhost:2182/chroot"); + assertEquals(config.getSolrMode(), "SolrCloud"); + assertEquals(config.getSolrCollection(), "techproducts"); + assertEquals(config.getSolrCommitWithinMs(), Integer.parseInt("100")); + assertEquals(config.getUsername(), "fakeuser"); + assertEquals(config.getPassword(), "fake@123"); + } + + @Test + public final void loadFromMapCredentialsFromSecretTest() throws IOException { + Map map = new HashMap<>(); + map.put("solrUrl", "localhost:2181,localhost:2182/chroot"); + map.put("solrMode", "SolrCloud"); + map.put("solrCollection", "techproducts"); + map.put("solrCommitWithinMs", "100"); + + SinkContext sinkContext = Mockito.mock(SinkContext.class); + Mockito.when(sinkContext.getSecret("username")) + .thenReturn("fakeuser"); + Mockito.when(sinkContext.getSecret("password")) + .thenReturn("fake@123"); + SolrSinkConfig config = SolrSinkConfig.load(map, sinkContext); assertNotNull(config); assertEquals(config.getSolrUrl(), "localhost:2181,localhost:2182/chroot"); assertEquals(config.getSolrMode(), "SolrCloud"); @@ -81,12 +107,13 @@ public final void validValidateTest() throws IOException { map.put("username", "fakeuser"); map.put("password", "fake@123"); - SolrSinkConfig config = SolrSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + SolrSinkConfig config = SolrSinkConfig.load(map, sinkContext); config.validate(); } - @Test(expectedExceptions = NullPointerException.class, - expectedExceptionsMessageRegExp = "solrUrl property not set.") + @Test(expectedExceptions = IllegalArgumentException.class, + expectedExceptionsMessageRegExp = "solrUrl cannot be null") public final void missingValidValidateSolrModeTest() throws IOException { Map map = new HashMap<>(); map.put("solrMode", "SolrCloud"); @@ -95,7 +122,8 @@ public final void missingValidValidateSolrModeTest() throws IOException { map.put("username", "fakeuser"); map.put("password", "fake@123"); - SolrSinkConfig config = SolrSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + SolrSinkConfig config = SolrSinkConfig.load(map, sinkContext); config.validate(); } @@ -110,7 +138,8 @@ public final void invalidBatchTimeMsTest() throws IOException { map.put("username", "fakeuser"); map.put("password", "fake@123"); - SolrSinkConfig config = SolrSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + SolrSinkConfig config = SolrSinkConfig.load(map, sinkContext); config.validate(); } @@ -125,7 +154,8 @@ public final void invalidClientModeTest() throws IOException { map.put("username", "fakeuser"); map.put("password", "fake@123"); - SolrSinkConfig config = SolrSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + SolrSinkConfig config = SolrSinkConfig.load(map, sinkContext); config.validate(); SolrAbstractSink.SolrMode.valueOf(config.getSolrMode().toUpperCase()); @@ -141,7 +171,8 @@ public final void validZkChrootTest() throws IOException { map.put("username", "fakeuser"); map.put("password", "fake@123"); - SolrSinkConfig config = SolrSinkConfig.load(map); + SinkContext sinkContext = Mockito.mock(SinkContext.class); + SolrSinkConfig config = SolrSinkConfig.load(map, sinkContext); config.validate(); String url = config.getSolrUrl(); From 8f54f8b87c632554b6f311506f46773ee32ccbac Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Fri, 22 Dec 2023 23:44:53 +0800 Subject: [PATCH 206/699] [improve][ci] Upgrade Aerospike java client to 4.5.0 to avoid CVE-2023-36480 (#21792) (cherry picked from commit 9a5c2f229e11ea1dd59737afc8b2308fac78f16d) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 494584c2ee193..9ce5c7ab8b6a6 100644 --- a/pom.xml +++ b/pom.xml @@ -173,7 +173,7 @@ flexible messaging model and an intuitive client API. 0.8.3 2.2.0 3.11.2 - 4.4.20 + 4.5.0 3.4.0 5.18.0 1.12.262 From 96cfe9a761d59e2d16c3c06c0d2dd4047a6d55eb Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Wed, 27 Dec 2023 10:07:49 +0800 Subject: [PATCH 207/699] [improve][ci] Exclude jose4j to avoid CVE-2023-31582 (#21791) (cherry picked from commit d9c55b4f5c70fa13185fdf0290d4513b0380f216) --- pom.xml | 4 ++++ pulsar-io/debezium/core/pom.xml | 4 ++++ pulsar-io/kafka-connect-adaptor/pom.xml | 22 ++++++++++++++++++++++ pulsar-io/kafka/pom.xml | 6 ++++++ pulsar-io/solr/pom.xml | 6 ++++++ 5 files changed, 42 insertions(+) diff --git a/pom.xml b/pom.xml index 9ce5c7ab8b6a6..152bb8924ff54 100644 --- a/pom.xml +++ b/pom.xml @@ -547,6 +547,10 @@ flexible messaging model and an intuitive client API. com.squareup.okio okio + + jose4j + org.bitbucket.b_c + diff --git a/pulsar-io/debezium/core/pom.xml b/pulsar-io/debezium/core/pom.xml index d5ae87fd0918e..17a77ea673e6a 100644 --- a/pulsar-io/debezium/core/pom.xml +++ b/pulsar-io/debezium/core/pom.xml @@ -71,6 +71,10 @@ org.apache.kafka kafka-log4j-appender + + jose4j + org.bitbucket.b_c + diff --git a/pulsar-io/kafka-connect-adaptor/pom.xml b/pulsar-io/kafka-connect-adaptor/pom.xml index 27b57583dfe82..f9d28299dfdcd 100644 --- a/pulsar-io/kafka-connect-adaptor/pom.xml +++ b/pulsar-io/kafka-connect-adaptor/pom.xml @@ -73,6 +73,10 @@ org.eclipse.jetty * + + jose4j + org.bitbucket.b_c + @@ -80,12 +84,24 @@ org.apache.kafka connect-json ${kafka-client.version} + + + jose4j + org.bitbucket.b_c + + org.apache.kafka connect-api ${kafka-client.version} + + + jose4j + org.bitbucket.b_c + + @@ -136,6 +152,12 @@ connect-file ${kafka-client.version} test + + + jose4j + org.bitbucket.b_c + + diff --git a/pulsar-io/kafka/pom.xml b/pulsar-io/kafka/pom.xml index 3084dd076f732..b5a0e0da3ea7e 100644 --- a/pulsar-io/kafka/pom.xml +++ b/pulsar-io/kafka/pom.xml @@ -84,6 +84,12 @@ org.apache.kafka kafka-clients ${kafka-client.version} + + + jose4j + org.bitbucket.b_c + + diff --git a/pulsar-io/solr/pom.xml b/pulsar-io/solr/pom.xml index 2c3e213d8542a..40de1f1750a51 100644 --- a/pulsar-io/solr/pom.xml +++ b/pulsar-io/solr/pom.xml @@ -65,6 +65,12 @@ org.apache.solr solr-core ${solr.version} + + + jose4j + org.bitbucket.b_c + + test From 77ad5680d4f8fa12241042acd0c1887d19ed7b04 Mon Sep 17 00:00:00 2001 From: nikhilerigila09 Date: Thu, 1 Feb 2024 10:48:12 +0530 Subject: [PATCH 208/699] Release 3.1.0.1 --- 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-experimental/pom.xml | 2 +- docker/pulsar/pom.xml | 2 +- jclouds-shaded/pom.xml | 2 +- managed-ledger/pom.xml | 2 +- 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-client-1x-base/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-1x/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-2x-shaded/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/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/hdfs2/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-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-sql/pom.xml | 2 +- pulsar-sql/presto-distribution/pom.xml | 2 +- pulsar-sql/presto-pulsar-plugin/pom.xml | 2 +- pulsar-sql/presto-pulsar/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 +- 134 files changed, 136 insertions(+), 136 deletions(-) diff --git a/bouncy-castle/bc/pom.xml b/bouncy-castle/bc/pom.xml index 6058c28076814..51b34bb1648cd 100644 --- a/bouncy-castle/bc/pom.xml +++ b/bouncy-castle/bc/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.2 + 3.1.0.1 .. diff --git a/bouncy-castle/bcfips-include-test/pom.xml b/bouncy-castle/bcfips-include-test/pom.xml index 92641bdd2c49a..7d384a35cd236 100644 --- a/bouncy-castle/bcfips-include-test/pom.xml +++ b/bouncy-castle/bcfips-include-test/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.2 + 3.1.0.1 .. diff --git a/bouncy-castle/bcfips/pom.xml b/bouncy-castle/bcfips/pom.xml index 2280474524cee..9f8a5d83acb64 100644 --- a/bouncy-castle/bcfips/pom.xml +++ b/bouncy-castle/bcfips/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.2 + 3.1.0.1 .. diff --git a/bouncy-castle/pom.xml b/bouncy-castle/pom.xml index dfb955a354459..4d1341512ebb7 100644 --- a/bouncy-castle/pom.xml +++ b/bouncy-castle/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.2 + 3.1.0.1 .. diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 393a69fd1edb5..72bad5cc5343e 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -31,12 +31,12 @@ com.datastax.oss buildtools - 3.1.2 + 3.1.0.1 jar Pulsar Build Tools - 2023-11-30T15:05:59Z + 2024-02-01T05:18:08Z 1.8 1.8 3.1.0 diff --git a/distribution/io/pom.xml b/distribution/io/pom.xml index 710373712427c..f6c7456234b19 100644 --- a/distribution/io/pom.xml +++ b/distribution/io/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.2 + 3.1.0.1 .. diff --git a/distribution/offloaders/pom.xml b/distribution/offloaders/pom.xml index b6ea86580489e..af5f795133071 100644 --- a/distribution/offloaders/pom.xml +++ b/distribution/offloaders/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.2 + 3.1.0.1 .. diff --git a/distribution/pom.xml b/distribution/pom.xml index 01354a0eaac69..12260bcf61b92 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.2 + 3.1.0.1 .. diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index 6177782aad3bd..86e60b1fce4fd 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.2 + 3.1.0.1 .. diff --git a/distribution/shell/pom.xml b/distribution/shell/pom.xml index c4e6118f8bdd2..1b60b6ef70c0d 100644 --- a/distribution/shell/pom.xml +++ b/distribution/shell/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.2 + 3.1.0.1 .. diff --git a/docker/pom.xml b/docker/pom.xml index 477cfa3f221d6..480307e8d282a 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.2 + 3.1.0.1 docker-images Apache Pulsar :: Docker Images diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index 977253c228028..f9c882f3cf6c6 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.2 + 3.1.0.1 4.0.0 pulsar-all-docker-image diff --git a/docker/pulsar-experimental/pom.xml b/docker/pulsar-experimental/pom.xml index c19a278607a9b..f5ce5d4abbd22 100644 --- a/docker/pulsar-experimental/pom.xml +++ b/docker/pulsar-experimental/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.2 + 3.1.0.1 4.0.0 pulsar-experimental-docker-image diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 9c55ff898d913..cee568a29d4d9 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.2 + 3.1.0.1 4.0.0 pulsar-docker-image diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml index 9d3f6961fe550..12ed4a156a456 100644 --- a/jclouds-shaded/pom.xml +++ b/jclouds-shaded/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.2 + 3.1.0.1 .. diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index fca2d04548f43..cef24f73883e3 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.2 + 3.1.0.1 .. diff --git a/pom.xml b/pom.xml index 152bb8924ff54..9ae509cab47e5 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ com.datastax.oss pulsar - 3.1.2 + 3.1.0.1 Pulsar Pulsar is a distributed pub-sub messaging platform with a very @@ -92,7 +92,7 @@ flexible messaging model and an intuitive client API. UTF-8 UTF-8 - 2023-11-30T15:05:59Z + 2024-02-01T05:18:08Z true 1.21 - 4.16.3 + 4.16.4 3.9.1 1.5.0 1.10.0 diff --git a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java index ec5f77f79464b..970c45964f784 100644 --- a/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java +++ b/pulsar-metadata/src/test/java/org/apache/bookkeeper/replication/AuditorLedgerCheckerTest.java @@ -42,6 +42,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import lombok.Cleanup; import org.apache.bookkeeper.bookie.BookieImpl; @@ -425,8 +426,16 @@ public void testInnerDelayedAuditOfLostBookies() throws Exception { // wait for 5 seconds before starting the recovery work when a bookie fails urLedgerMgr.setLostBookieRecoveryDelay(5); - // shutdown a non auditor bookie; choosing non-auditor to avoid another election - String shutdownBookie = shutDownNonAuditorBookie(); + AtomicReference shutdownBookieRef = new AtomicReference<>(); + CountDownLatch shutdownLatch = new CountDownLatch(1); + new Thread(() -> { + try { + String shutdownBookie = shutDownNonAuditorBookie(); + shutdownBookieRef.set(shutdownBookie); + shutdownLatch.countDown(); + } catch (Exception ignore) { + } + }).start(); if (LOG.isDebugEnabled()) { LOG.debug("Waiting for ledgers to be marked as under replicated"); @@ -442,9 +451,10 @@ public void testInnerDelayedAuditOfLostBookies() throws Exception { urLedgerList.contains(ledgerId)); Map urLedgerData = getUrLedgerData(urLedgerList); String data = urLedgerData.get(ledgerId); - assertTrue("Bookie " + shutdownBookie + shutdownLatch.await(); + assertTrue("Bookie " + shutdownBookieRef.get() + "is not listed in the ledger as missing replica :" + data, - data.contains(shutdownBookie)); + data.contains(shutdownBookieRef.get())); } /** @@ -503,7 +513,16 @@ public void testRescheduleOfDelayedAuditOfLostBookiesToStartImmediately() throws urLedgerMgr.setLostBookieRecoveryDelay(50); // shutdown a non auditor bookie; choosing non-auditor to avoid another election - String shutdownBookie = shutDownNonAuditorBookie(); + AtomicReference shutdownBookieRef = new AtomicReference<>(); + CountDownLatch shutdownLatch = new CountDownLatch(1); + new Thread(() -> { + try { + String shutdownBookie = shutDownNonAuditorBookie(); + shutdownBookieRef.set(shutdownBookie); + shutdownLatch.countDown(); + } catch (Exception ignore) { + } + }).start(); if (LOG.isDebugEnabled()) { LOG.debug("Waiting for ledgers to be marked as under replicated"); @@ -522,9 +541,10 @@ public void testRescheduleOfDelayedAuditOfLostBookiesToStartImmediately() throws urLedgerList.contains(ledgerId)); Map urLedgerData = getUrLedgerData(urLedgerList); String data = urLedgerData.get(ledgerId); - assertTrue("Bookie " + shutdownBookie + shutdownLatch.await(); + assertTrue("Bookie " + shutdownBookieRef.get() + "is not listed in the ledger as missing replica :" + data, - data.contains(shutdownBookie)); + data.contains(shutdownBookieRef.get())); } @Test @@ -547,7 +567,16 @@ public void testRescheduleOfDelayedAuditOfLostBookiesToStartLater() throws Excep urLedgerMgr.setLostBookieRecoveryDelay(3); // shutdown a non auditor bookie; choosing non-auditor to avoid another election - String shutdownBookie = shutDownNonAuditorBookie(); + AtomicReference shutdownBookieRef = new AtomicReference<>(); + CountDownLatch shutdownLatch = new CountDownLatch(1); + new Thread(() -> { + try { + String shutdownBookie = shutDownNonAuditorBookie(); + shutdownBookieRef.set(shutdownBookie); + shutdownLatch.countDown(); + } catch (Exception ignore) { + } + }).start(); if (LOG.isDebugEnabled()) { LOG.debug("Waiting for ledgers to be marked as under replicated"); @@ -573,9 +602,10 @@ public void testRescheduleOfDelayedAuditOfLostBookiesToStartLater() throws Excep urLedgerList.contains(ledgerId)); Map urLedgerData = getUrLedgerData(urLedgerList); String data = urLedgerData.get(ledgerId); - assertTrue("Bookie " + shutdownBookie + shutdownLatch.await(); + assertTrue("Bookie " + shutdownBookieRef.get() + "is not listed in the ledger as missing replica :" + data, - data.contains(shutdownBookie)); + data.contains(shutdownBookieRef.get())); } @Test @@ -664,7 +694,12 @@ public void testTriggerAuditorWithPendingAuditTask() throws Exception { urLedgerMgr.setLostBookieRecoveryDelay(lostBookieRecoveryDelay); // shutdown a non auditor bookie; choosing non-auditor to avoid another election - String shutdownBookie = shutDownNonAuditorBookie(); + new Thread(() -> { + try { + shutDownNonAuditorBookie(); + } catch (Exception ignore) { + } + }).start(); if (LOG.isDebugEnabled()) { LOG.debug("Waiting for ledgers to be marked as under replicated"); @@ -715,7 +750,12 @@ public void testTriggerAuditorBySettingDelayToZeroWithPendingAuditTask() throws urLedgerMgr.setLostBookieRecoveryDelay(lostBookieRecoveryDelay); // shutdown a non auditor bookie; choosing non-auditor to avoid another election - String shutdownBookie = shutDownNonAuditorBookie(); + new Thread(() -> { + try { + shutDownNonAuditorBookie(); + } catch (Exception ignore) { + } + }).start(); if (LOG.isDebugEnabled()) { LOG.debug("Waiting for ledgers to be marked as under replicated"); @@ -767,8 +807,17 @@ public void testDelayedAuditWithMultipleBookieFailures() throws Exception { // wait for 10 seconds before starting the recovery work when a bookie fails urLedgerMgr.setLostBookieRecoveryDelay(10); - // shutdown a non auditor bookie to avoid an election - String shutdownBookie1 = shutDownNonAuditorBookie(); + // shutdown a non auditor bookie; choosing non-auditor to avoid another election + AtomicReference shutdownBookieRef1 = new AtomicReference<>(); + CountDownLatch shutdownLatch1 = new CountDownLatch(1); + new Thread(() -> { + try { + String shutdownBookie1 = shutDownNonAuditorBookie(); + shutdownBookieRef1.set(shutdownBookie1); + shutdownLatch1.countDown(); + } catch (Exception ignore) { + } + }).start(); // wait for 3 seconds and there shouldn't be any under replicated ledgers // because we have delayed the start of audit by 10 seconds @@ -780,7 +829,16 @@ public void testDelayedAuditWithMultipleBookieFailures() throws Exception { // the history about having delayed recovery remains. Hence we make sure // we bring down a non auditor bookie. This should cause the audit to take // place immediately and not wait for the remaining 7 seconds to elapse - String shutdownBookie2 = shutDownNonAuditorBookie(); + AtomicReference shutdownBookieRef2 = new AtomicReference<>(); + CountDownLatch shutdownLatch2 = new CountDownLatch(1); + new Thread(() -> { + try { + String shutdownBookie2 = shutDownNonAuditorBookie(); + shutdownBookieRef2.set(shutdownBookie2); + shutdownLatch2.countDown(); + } catch (Exception ignore) { + } + }).start(); // 2 second grace period for the ledgers to get reported as under replicated Thread.sleep(2000); @@ -793,9 +851,11 @@ public void testDelayedAuditWithMultipleBookieFailures() throws Exception { urLedgerList.contains(ledgerId)); Map urLedgerData = getUrLedgerData(urLedgerList); String data = urLedgerData.get(ledgerId); - assertTrue("Bookie " + shutdownBookie1 + shutdownBookie2 + shutdownLatch1.await(); + shutdownLatch2.await(); + assertTrue("Bookie " + shutdownBookieRef1.get() + shutdownBookieRef2.get() + " are not listed in the ledger as missing replicas :" + data, - data.contains(shutdownBookie1) && data.contains(shutdownBookie2)); + data.contains(shutdownBookieRef1.get()) && data.contains(shutdownBookieRef2.get())); } /** @@ -825,7 +885,16 @@ public void testDelayedAuditWithRollingUpgrade() throws Exception { // shutdown a non auditor bookie to avoid an election int idx1 = getShutDownNonAuditorBookieIdx(""); ServerConfiguration conf1 = confByIndex(idx1); - String shutdownBookie1 = shutdownBookie(idx1); + AtomicReference shutdownBookieRef1 = new AtomicReference<>(); + CountDownLatch shutdownLatch1 = new CountDownLatch(1); + new Thread(() -> { + try { + String shutdownBookie1 = shutdownBookie(idx1); + shutdownBookieRef1.set(shutdownBookie1); + shutdownLatch1.countDown(); + } catch (Exception ignore) { + } + }).start(); // wait for 2 seconds and there shouldn't be any under replicated ledgers // because we have delayed the start of audit by 5 seconds @@ -838,7 +907,17 @@ public void testDelayedAuditWithRollingUpgrade() throws Exception { // Now to simulate the rolling upgrade, bring down a bookie different from // the one we brought down/up above. - String shutdownBookie2 = shutDownNonAuditorBookie(shutdownBookie1); + // shutdown a non auditor bookie; choosing non-auditor to avoid another election + AtomicReference shutdownBookieRef2 = new AtomicReference<>(); + CountDownLatch shutdownLatch2 = new CountDownLatch(1); + new Thread(() -> { + try { + String shutdownBookie2 = shutDownNonAuditorBookie(); + shutdownBookieRef2.set(shutdownBookie2); + shutdownLatch2.countDown(); + } catch (Exception ignore) { + } + }).start(); // since the first bookie that was brought down/up has come up, there is only // one bookie down at this time. Hence the lost bookie check shouldn't start @@ -856,11 +935,13 @@ public void testDelayedAuditWithRollingUpgrade() throws Exception { urLedgerList.contains(ledgerId)); Map urLedgerData = getUrLedgerData(urLedgerList); String data = urLedgerData.get(ledgerId); - assertTrue("Bookie " + shutdownBookie1 + "wrongly listed as missing the ledger: " + data, - !data.contains(shutdownBookie1)); - assertTrue("Bookie " + shutdownBookie2 + shutdownLatch1.await(); + shutdownLatch2.await(); + assertTrue("Bookie " + shutdownBookieRef1.get() + "wrongly listed as missing the ledger: " + data, + !data.contains(shutdownBookieRef1.get())); + assertTrue("Bookie " + shutdownBookieRef2.get() + " is not listed in the ledger as missing replicas :" + data, - data.contains(shutdownBookie2)); + data.contains(shutdownBookieRef2.get())); LOG.info("*****************Test Complete"); } From 998d15fa7dde8f4b66a0034f9aba2abadf5cb755 Mon Sep 17 00:00:00 2001 From: thetumbled <52550727+thetumbled@users.noreply.github.com> Date: Tue, 30 Jan 2024 23:40:09 +0800 Subject: [PATCH 249/699] [fix] [broker] add timeout for health check read. (#21990) (cherry picked from commit fabcc99c9c6dd25352a53ee9b330ef27251b272b) --- .../pulsar/broker/admin/impl/BrokersBase.java | 13 +++- .../broker/admin/AdminApiHealthCheckTest.java | 63 +++++++++++++++++++ 2 files changed, 75 insertions(+), 1 deletion(-) 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 f056b18f3f1d1..61b354610ac20 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 @@ -26,6 +26,7 @@ import java.lang.management.ManagementFactory; import java.lang.management.ThreadInfo; import java.lang.management.ThreadMXBean; +import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -34,6 +35,7 @@ import java.util.Objects; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import javax.ws.rs.DELETE; import javax.ws.rs.DefaultValue; @@ -80,6 +82,12 @@ public class BrokersBase extends AdminResource { // log a full thread dump when a deadlock is detected in healthcheck once every 10 minutes // to prevent excessive logging private static final long LOG_THREADDUMP_INTERVAL_WHEN_DEADLOCK_DETECTED = 600000L; + // there is a timeout of 60 seconds default in the client(readTimeoutMs), so we need to set the timeout + // a bit shorter than 60 seconds to avoid the client timeout exception thrown before the server timeout exception. + // or we can't propagate the server timeout exception to the client. + private static final Duration HEALTH_CHECK_READ_TIMEOUT = Duration.ofSeconds(58); + private static final TimeoutException HEALTH_CHECK_TIMEOUT_EXCEPTION = + FutureUtil.createTimeoutException("Timeout", BrokersBase.class, "healthCheckRecursiveReadNext(...)"); private volatile long threadDumpLoggedTimestamp; @GET @@ -434,7 +442,10 @@ private CompletableFuture internalRunHealthCheck(TopicVersion topicVersion }); throw FutureUtil.wrapToCompletionException(createException); }).thenCompose(reader -> producer.sendAsync(messageStr) - .thenCompose(__ -> healthCheckRecursiveReadNext(reader, messageStr)) + .thenCompose(__ -> FutureUtil.addTimeoutHandling( + healthCheckRecursiveReadNext(reader, messageStr), + HEALTH_CHECK_READ_TIMEOUT, pulsar().getBrokerService().executor(), + () -> HEALTH_CHECK_TIMEOUT_EXCEPTION)) .whenComplete((__, ex) -> { closeAndReCheck(producer, reader, topicOptional.get(), subscriptionName) .whenComplete((unused, innerEx) -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiHealthCheckTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiHealthCheckTest.java index a780f889de85f..357422b11f6ce 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiHealthCheckTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiHealthCheckTest.java @@ -23,6 +23,7 @@ import static org.testng.Assert.assertTrue; import java.lang.management.ManagementFactory; import java.lang.management.ThreadMXBean; +import java.lang.reflect.Field; import java.time.Duration; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -31,13 +32,21 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.admin.PulsarAdminException; +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.impl.ProducerBuilderImpl; +import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.naming.TopicVersion; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.compaction.Compactor; import org.awaitility.Awaitility; +import org.mockito.Mockito; import org.springframework.util.CollectionUtils; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -236,4 +245,58 @@ public void testHealthCheckupV2() throws Exception { )) ); } + + class DummyProducerBuilder extends ProducerBuilderImpl { + // This is a dummy producer builder to test the health check timeout + // the producer constructed by this builder will not send any message + public DummyProducerBuilder(PulsarClientImpl client, Schema schema) { + super(client, schema); + } + + @Override + public CompletableFuture> createAsync() { + CompletableFuture> future = new CompletableFuture<>(); + super.createAsync().thenAccept(producer -> { + Producer spyProducer = Mockito.spy(producer); + Mockito.doReturn(CompletableFuture.completedFuture(MessageId.earliest)) + .when(spyProducer).sendAsync(Mockito.any()); + future.complete(spyProducer); + }).exceptionally(ex -> { + future.completeExceptionally(ex); + return null; + }); + return future; + } + } + + @Test + public void testHealthCheckTimeOut() throws Exception { + final String testHealthCheckTopic = String.format("persistent://pulsar/localhost:%s/healthcheck", + pulsar.getConfig().getWebServicePort().get()); + PulsarClient client = pulsar.getClient(); + PulsarClient spyClient = Mockito.spy(client); + Mockito.doReturn(new DummyProducerBuilder<>((PulsarClientImpl) spyClient, Schema.BYTES)) + .when(spyClient).newProducer(Schema.STRING); + // use reflection to replace the client in the broker + Field field = PulsarService.class.getDeclaredField("client"); + field.setAccessible(true); + field.set(pulsar, spyClient); + try { + admin.brokers().healthcheck(TopicVersion.V2); + throw new Exception("Should not reach here"); + } catch (PulsarAdminException e) { + log.info("Exception caught", e); + assertTrue(e.getMessage().contains("LowOverheadTimeoutException")); + } + // To ensure we don't have any subscription, the producers and readers are closed. + Awaitility.await().untilAsserted(() -> + assertTrue(CollectionUtils.isEmpty(admin.topics() + .getSubscriptions(testHealthCheckTopic).stream() + // All system topics are using compaction, even though is not explicitly set in the policies. + .filter(v -> !v.equals(Compactor.COMPACTION_SUBSCRIPTION)) + .collect(Collectors.toList()) + )) + ); + } + } From 4a2e5606087ce0608072bb34791929b2e9bd0735 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=90=A7=E6=98=93=E5=AE=A2?= Date: Tue, 30 Jan 2024 19:20:29 +0800 Subject: [PATCH 250/699] [fix][client] Fix ConsumerBuilderImpl#subscribe silent stuck when using pulsar-client:3.0.x with jackson-annotations prior to 2.12.0 (#21985) ### Motivation In summary, `jackson-annotations:2.12.0` or later is now required for `pulsar-client 3.0.x`, and this also applies to versions `3.1.x` and `3.2.x`. Otherwise, `ConsumerBuilderImpl#subscribe` may become stuck without displaying any error message. ### Modifications Modify the `whenComplete` to a combination of `thenAccept` and `exceptionally`. The modification is harmless. (cherry picked from commit 327b973002fca565e41ef27806e233f37aa48ac5) --- .../client/impl/MultiTopicsConsumerImpl.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) 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 8a515a9f9b8d7..68860cf1e2929 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 @@ -999,13 +999,13 @@ CompletableFuture subscribeAsync(String topicName, int numberPartitions) { private void subscribeTopicPartitions(CompletableFuture subscribeResult, String topicName, int numPartitions, boolean createIfDoesNotExist) { - client.preProcessSchemaBeforeSubscribe(client, schema, topicName).whenComplete((schema, cause) -> { - if (null == cause) { - doSubscribeTopicPartitions(schema, subscribeResult, topicName, numPartitions, createIfDoesNotExist); - } else { - subscribeResult.completeExceptionally(cause); - } - }); + client.preProcessSchemaBeforeSubscribe(client, schema, topicName) + .thenAccept(schema -> { + doSubscribeTopicPartitions(schema, subscribeResult, topicName, numPartitions, createIfDoesNotExist); + }).exceptionally(cause -> { + subscribeResult.completeExceptionally(cause); + return null; + }); } private void doSubscribeTopicPartitions(Schema schema, From c8b4f0b541bdde7596c61a20820df7737c78f81b Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 22 Nov 2023 01:32:10 +0800 Subject: [PATCH 251/699] [fix][broker] Correct schema deletion for parititioned topic (#21574) ### Motivation Schemas binding on the partitioned topic, but schemas will be deleted when a partition is deleted. ### Modifications Correct the behaviors of schema deleting: - Pulsar deletes schema when a non-partitioned topic is deleted. - Pulsar deletes schema when a partitioned topic metadata is deleted. - Pulsar does not delete schema when only a part of a partitioned topic is deleted. (cherry picked from commit cc22c87864d4b3092dbb14fc493507344acf5696) --- .../admin/impl/PersistentTopicsBase.java | 4 +- .../pulsar/broker/service/BrokerService.java | 2 +- .../service/persistent/PersistentTopic.java | 10 ++ .../pulsar/broker/service/TopicGCTest.java | 112 +++++++++++++++++ .../service/schema/TopicSchemaTest.java | 118 ++++++++++++++++++ 5 files changed, 244 insertions(+), 2 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/TopicSchemaTest.java 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 8e7bf71b22246..121b57fc8eea7 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 @@ -722,7 +722,9 @@ protected void internalDeletePartitionedTopic(AsyncResponse asyncResponse, .thenCompose(unused -> internalRemovePartitionsTopicAsync(numPartitions, force)); }) // Only tries to delete the znode for partitioned topic when all its partitions are successfully deleted - ).thenCompose(__ -> getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + ).thenCompose(ignore -> + pulsar().getBrokerService().deleteSchema(topicName).exceptionally(ex -> null)) + .thenCompose(__ -> getPulsarResources().getNamespaceResources().getPartitionedTopicResources() .runWithMarkDeleteAsync(topicName, () -> namespaceResources() .getPartitionedTopicResources().deletePartitionedTopicAsync(topicName))) .thenAccept(__ -> { 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 8992c091a50e1..4713061e0b18b 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 @@ -3518,7 +3518,7 @@ public CompletableFuture deleteTopicPolicies(TopicName topicName) { .deleteTopicPoliciesAsync(TopicName.get(topicName.getPartitionedTopicName())); } - CompletableFuture deleteSchema(TopicName topicName) { + public CompletableFuture deleteSchema(TopicName topicName) { String base = topicName.getPartitionedTopicName(); String id = TopicName.get(base).getSchemaName(); SchemaRegistryService schemaRegistryService = getPulsar().getSchemaRegistryService(); 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 c4658d55a067c..f07578667e8d4 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 @@ -162,6 +162,7 @@ import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.protocol.schema.SchemaData; +import org.apache.pulsar.common.protocol.schema.SchemaVersion; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.common.topics.TopicCompactionStrategy; import org.apache.pulsar.common.util.Codec; @@ -2372,6 +2373,15 @@ private Optional getCompactorMXBean() { return Optional.ofNullable(compactor).map(c -> c.getStats()); } + @Override + public CompletableFuture deleteSchema() { + if (TopicName.get(getName()).isPartitioned()) { + // Only delete schema when partitioned metadata is deleting. + return CompletableFuture.completedFuture(null); + } + return brokerService.deleteSchema(TopicName.get(getName())); + } + @Override public CompletableFuture getInternalStats(boolean includeLedgerMetadata) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java new file mode 100644 index 0000000000000..7790940c1327f --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java @@ -0,0 +1,112 @@ +/* + * 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.assertTrue; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import lombok.EqualsAndHashCode; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +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.Schema; +import org.apache.pulsar.common.policies.data.InactiveTopicDeleteMode; +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 TopicGCTest 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(); + } + + @EqualsAndHashCode.Include + protected void doInitConf() throws Exception { + super.doInitConf(); + this.conf.setBrokerDeleteInactiveTopicsEnabled(true); + this.conf.setBrokerDeleteInactiveTopicsMode(InactiveTopicDeleteMode.delete_when_subscriptions_caught_up); + this.conf.setBrokerDeleteInactiveTopicsFrequencySeconds(10); + } + + @Test + public void testCreateConsumerAfterOnePartDeleted() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String partition0 = topic + "-partition-0"; + final String partition1 = topic + "-partition-1"; + final String subscription = "s1"; + admin.topics().createPartitionedTopic(topic, 2); + admin.topics().createSubscription(topic, subscription, MessageId.earliest); + + // create consumers and producers. + Producer producer0 = pulsarClient.newProducer(Schema.STRING).topic(partition0) + .enableBatching(false).create(); + Producer producer1 = pulsarClient.newProducer(Schema.STRING).topic(partition1) + .enableBatching(false).create(); + org.apache.pulsar.client.api.Consumer consumer1 = pulsarClient.newConsumer(Schema.STRING).topic(topic) + .subscriptionName(subscription).isAckReceiptEnabled(true).subscribe(); + + // Make consume all messages for one topic, do not consume any messages for another one. + producer0.send("1"); + producer1.send("2"); + admin.topics().skipAllMessages(partition0, subscription); + + // Wait for topic GC. + // Partition 0 will be deleted about 20s later, left 2min to avoid flaky. + producer0.close(); + consumer1.close(); + Awaitility.await().atMost(2, TimeUnit.MINUTES).untilAsserted(() -> { + CompletableFuture> tp1 = pulsar.getBrokerService().getTopic(partition0, false); + CompletableFuture> tp2 = pulsar.getBrokerService().getTopic(partition1, false); + assertTrue(tp1 == null || !tp1.get().isPresent()); + assertTrue(tp2 != null && tp2.get().isPresent()); + }); + + // Verify that the consumer subscribed with partitioned topic can be created successful. + Consumer consumerAllPartition = pulsarClient.newConsumer(Schema.STRING).topic(topic) + .subscriptionName(subscription).isAckReceiptEnabled(true).subscribe(); + Message msg = consumerAllPartition.receive(2, TimeUnit.SECONDS); + String receivedMsgValue = msg.getValue(); + log.info("received msg: {}", receivedMsgValue); + consumerAllPartition.acknowledge(msg); + + // cleanup. + consumerAllPartition.close(); + producer0.close(); + producer1.close(); + admin.topics().deletePartitionedTopic(topic); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/TopicSchemaTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/TopicSchemaTest.java new file mode 100644 index 0000000000000..66bfd1c3ec2b0 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/schema/TopicSchemaTest.java @@ -0,0 +1,118 @@ +/* + * 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.schema; + +import static org.apache.pulsar.broker.service.schema.SchemaRegistry.SchemaAndMetadata; +import static org.testng.Assert.assertTrue; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; +import org.apache.pulsar.broker.BrokerTestUtil; +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.common.naming.TopicDomain; +import org.apache.pulsar.common.naming.TopicName; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Test(groups = "broker") +public class TopicSchemaTest 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 = "topicDomains") + public Object[][] topicDomains() { + return new Object[][]{ + {TopicDomain.non_persistent}, + {TopicDomain.persistent} + }; + } + + @Test(dataProvider = "topicDomains") + public void testDeleteNonPartitionedTopicWithSchema(TopicDomain topicDomain) throws Exception { + final String topic = BrokerTestUtil.newUniqueName(topicDomain.value() + "://public/default/tp"); + final String schemaId = TopicName.get(TopicName.get(topic).getPartitionedTopicName()).getSchemaName(); + admin.topics().createNonPartitionedTopic(topic); + + // Add schema. + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic) + .enableBatching(false).create(); + producer.close(); + List schemaList1 = pulsar.getSchemaRegistryService().getAllSchemas(schemaId).join() + .stream().map(s -> s.join()).filter(Objects::nonNull).collect(Collectors.toList()); + assertTrue(schemaList1 != null && schemaList1.size() > 0); + + // Verify the schema has been deleted with topic. + admin.topics().delete(topic, false); + List schemaList2 = pulsar.getSchemaRegistryService().getAllSchemas(schemaId).join() + .stream().map(s -> s.join()).filter(Objects::nonNull).collect(Collectors.toList()); + assertTrue(schemaList2 == null || schemaList2.isEmpty()); + } + + @Test + public void testDeletePartitionedTopicWithoutSchema() throws Exception { + // Non-persistent topic does not support partitioned topic now, so only write a test case for persistent topic. + TopicDomain topicDomain = TopicDomain.persistent; + final String topic = BrokerTestUtil.newUniqueName(topicDomain.value() + "://public/default/tp"); + final String partition0 = topic + "-partition-0"; + final String partition1 = topic + "-partition-1"; + final String schemaId = TopicName.get(TopicName.get(topic).getPartitionedTopicName()).getSchemaName(); + admin.topics().createPartitionedTopic(topic, 2); + + // Add schema. + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic) + .enableBatching(false).create(); + producer.close(); + List schemaList1 = pulsar.getSchemaRegistryService().getAllSchemas(schemaId).join() + .stream().map(s -> s.join()).filter(Objects::nonNull).collect(Collectors.toList()); + assertTrue(schemaList1 != null && schemaList1.size() > 0); + + // Verify the schema will not been deleted with partition-0. + admin.topics().delete(partition0, false); + List schemaList2 = pulsar.getSchemaRegistryService().getAllSchemas(schemaId).join() + .stream().map(s -> s.join()).filter(Objects::nonNull).collect(Collectors.toList()); + assertTrue(schemaList2 != null && schemaList2.size() > 0); + + // Verify the schema will not been deleted with partition-0 & partition-1. + admin.topics().delete(partition1, false); + List schemaList3 = pulsar.getSchemaRegistryService().getAllSchemas(schemaId).join() + .stream().map(s -> s.join()).filter(Objects::nonNull).collect(Collectors.toList()); + assertTrue(schemaList3 != null && schemaList3.size() > 0); + + // Verify the schema will be deleted with partitioned metadata. + admin.topics().deletePartitionedTopic(topic, false); + List schemaList4 = pulsar.getSchemaRegistryService().getAllSchemas(schemaId).join() + .stream().map(s -> s.join()).filter(Objects::nonNull).collect(Collectors.toList()); + assertTrue(schemaList4 == null || schemaList4.isEmpty()); + } +} From ddf99cc26ea92a7befcfd197e978e11e2b1308dd Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Mon, 29 Jan 2024 20:16:05 -0800 Subject: [PATCH 252/699] [fix][broker] Fix schema deletion error when deleting a partitioned topic with many partitions and schema (#21977) (cherry picked from commit 75e21420c7b716f28c62c3f42b0fec8eec4e07b0) --- .../pulsar/broker/service/BrokerService.java | 29 +++++++++---------- .../schema/BookkeeperSchemaStorage.java | 6 ++-- .../tests/integration/schema/SchemaTest.java | 11 +++++++ 3 files changed, 28 insertions(+), 18 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 4713061e0b18b..a6db3e6aeaa33 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 @@ -121,8 +121,6 @@ import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.persistent.SystemTopic; import org.apache.pulsar.broker.service.plugin.EntryFilterProvider; -import org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage; -import org.apache.pulsar.broker.service.schema.SchemaRegistryService; 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; @@ -3519,22 +3517,21 @@ public CompletableFuture deleteTopicPolicies(TopicName topicName) { } public CompletableFuture deleteSchema(TopicName topicName) { + // delete schema at the upper level when deleting the partitioned topic. + if (topicName.isPartitioned()) { + return CompletableFuture.completedFuture(null); + } String base = topicName.getPartitionedTopicName(); String id = TopicName.get(base).getSchemaName(); - SchemaRegistryService schemaRegistryService = getPulsar().getSchemaRegistryService(); - return BookkeeperSchemaStorage.ignoreUnrecoverableBKException(schemaRegistryService.getSchema(id)) - .thenCompose(schema -> { - if (schema != null) { - // It's different from `SchemasResource.deleteSchema` - // because when we delete a topic, the schema - // history is meaningless. But when we delete a schema of a topic, a new schema could be - // registered in the future. - log.info("Delete schema storage of id: {}", id); - return getPulsar().getSchemaRegistryService().deleteSchemaStorage(id); - } else { - return CompletableFuture.completedFuture(null); - } - }); + return getPulsar().getSchemaRegistryService().deleteSchemaStorage(id).whenComplete((vid, ex) -> { + if (vid != null && ex == null) { + // It's different from `SchemasResource.deleteSchema` + // because when we delete a topic, the schema + // history is meaningless. But when we delete a schema of a topic, a new schema could be + // registered in the future. + log.info("Deleted schema storage of id: {}", id); + } + }); } private CompletableFuture checkMaxTopicsPerNamespace(TopicName topicName, int numPartitions) { 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 78e30f6fff827..c509764bf6710 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 @@ -707,7 +707,8 @@ public static Exception bkException(String operation, int rc, long ledgerId, lon message += " - entry=" + entryId; } boolean recoverable = rc != BKException.Code.NoSuchLedgerExistsException - && rc != BKException.Code.NoSuchEntryException; + && rc != BKException.Code.NoSuchEntryException + && rc != BKException.Code.NoSuchLedgerExistsOnMetadataServerException; return new SchemaException(recoverable, message); } @@ -716,7 +717,8 @@ public static CompletableFuture ignoreUnrecoverableBKException(Completabl if (t.getCause() != null && (t.getCause() instanceof SchemaException) && !((SchemaException) t.getCause()).isRecoverable()) { - // Meeting NoSuchLedgerExistsException or NoSuchEntryException when reading schemas in + // Meeting NoSuchLedgerExistsException, NoSuchEntryException or + // NoSuchLedgerExistsOnMetadataServerException when reading schemas in // bookkeeper. This also means that the data has already been deleted by other operations // in deleting schema. if (log.isDebugEnabled()) { diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/schema/SchemaTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/schema/SchemaTest.java index 8bb6de74c661d..d0421063b2d90 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/schema/SchemaTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/schema/SchemaTest.java @@ -31,6 +31,8 @@ import org.apache.pulsar.client.api.schema.SchemaDefinition; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.tests.integration.schema.Schemas.Person; import org.apache.pulsar.tests.integration.schema.Schemas.PersonConsumeSchema; import org.apache.pulsar.tests.integration.schema.Schemas.Student; @@ -316,5 +318,14 @@ public void testPrimitiveSchemaTypeCompatibilityCheck() { } + @Test + public void testDeletePartitionedTopicWhenTopicReferenceIsNotReady() throws Exception { + final String topic = "persistent://public/default/tp-ref"; + admin.topics().createPartitionedTopic(topic, 20); + admin.schemas().createSchema(topic, + SchemaInfo.builder().type(SchemaType.STRING).schema(new byte[0]).build()); + admin.topics().deletePartitionedTopic(topic, false); + } + } From 20e7e9510bff9c7d67fde0776f00d42a7a319dd9 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 30 Jan 2024 19:34:01 +0800 Subject: [PATCH 253/699] [improve] [proxy] Add a check for brokerServiceURL that does not support multi uri yet (#21972) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit At the beginning of the design, these two configurations(`brokerServiceURL & brokerServiceURLTLS`) do not support setting multiple broker addresses, which should instead be set to a “discovery service provider.” see: https://github.com/apache/pulsar/pull/1002 and https://github.com/apache/pulsar/pull/14682 Users will get the below error if they set A to a multi-broker URLs ``` "2024-01-09 00:20:10,261 -0800 [pulsar-proxy-io-4-7] WARN io.netty.channel.DefaultChannelPipeline - An exceptionCaught() event was fired, and it reached at the tail of the pipeline. It usually means the last handler in the pipeline did not handle the exception. java.lang.IllegalArgumentException: port out of range:-1 at java.net.InetSocketAddress.checkPort(InetSocketAddress.java:143) ~[?:?] at java.net.InetSocketAddress.createUnresolved(InetSocketAddress.java:254) ~[?:?] at org.apache.pulsar.proxy.server.LookupProxyHandler.getAddr(LookupProxyHandler.java:432) ~[org.apache.pulsar-pulsar-proxy-2.9.0.jar:2.9.0] at org.apache.pulsar.proxy.server.LookupProxyHandler.handleGetSchema(LookupProxyHandler.java:357) ~[org.apache.pulsar-pulsar-proxy-2.9.0.jar:2.9.0] at org.apache.pulsar.proxy.server.ProxyConnection.handleGetSchema(ProxyConnection.java:463) ~[org.apache.pulsar-pulsar-proxy-2.9.0.jar:2.9.0] at org.apache.pulsar.common.protocol.PulsarDecoder.channelRead(PulsarDecoder.java:326) ~[io.streamnative-pulsar-common-2.9.2.12.jar:2.9.2.12] at org.apache.pulsar.proxy.server.ProxyConnection.channelRead(ProxyConnection.java:221) ~[org.apache.pulsar-pulsar-proxy-2.9.0.jar:2.9.0] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) ~[io.netty-netty-transport-4.1.74.Final.jar:4.1.74.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365) ~[io.netty-netty-transport-4.1.74.Final.jar:4.1.74.Final] at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357) ~[io.netty-netty-transport-4.1.74.Final.jar:4.1.74.Final] at io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:327) ~[io.netty-netty-codec-4.1.74.Final.jar:4.1.74.Final] at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:299) ~[io.netty-netty-codec-4.1.74.Final.jar:4.1.74.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) ~[io.netty-netty-transport-4.1.74.Final.jar:4.1.74.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365) ~[io.netty-netty-transport-4.1.74.Final.jar:4.1.74.Final] at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357) ~[io.netty-netty-transport-4.1.74.Final.jar:4.1.74.Final] at io.netty.handler.ssl.SslHandler.unwrap(SslHandler.java:1372) ~[io.netty-netty-handler-4.1.74.Final.jar:4.1.74.Final] at io.netty.handler.ssl.SslHandler.decodeNonJdkCompatible(SslHandler.java:1246) ~[io.netty-netty-handler-4.1.74.Final.jar:4.1.74.Final] at io.netty.handler.ssl.SslHandler.decode(SslHandler.java:1286) ~[io.netty-netty-handler-4.1.74.Final.jar:4.1.74.Final] at io.netty.handler.codec.ByteToMessageDecoder.decodeRemovalReentryProtection(ByteToMessageDecoder.java:510) ~[io.netty-netty-codec-4.1.74.Final.jar:4.1.74.Final] at io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:449) ~[io.netty-netty-codec-4.1.74.Final.jar:4.1.74.Final] at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:279) ~[io.netty-netty-codec-4.1.74.Final.jar:4.1.74.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) ~[io.netty-netty-transport-4.1.74.Final.jar:4.1.74.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365) ~[io.netty-netty-transport-4.1.74.Final.jar:4.1.74.Final] at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357) ~[io.netty-netty-transport-4.1.74.Final.jar:4.1.74.Final] at io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410) ~[io.netty-netty-transport-4.1.74.Final.jar:4.1.74.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) ~[io.netty-netty-transport-4.1.74.Final.jar:4.1.74.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365) ~[io.netty-netty-transport-4.1.74.Final.jar:4.1.74.Final] at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919) ~[io.netty-netty-transport-4.1.74.Final.jar:4.1.74.Final] at io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:795) ~[io.netty-netty-transport-classes-epoll-4.1.74.Final.jar:4.1.74.Final] at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:480) ~[io.netty-netty-transport-classes-epoll-4.1.74.Final.jar:4.1.74.Final] at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:378) ~[io.netty-netty-transport-classes-epoll-4.1.74.Final.jar:4.1.74.Final] at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:986) ~[io.netty-netty-common-4.1.74.Final.jar:4.1.74.Final] at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74) ~[io.netty-netty-common-4.1.74.Final.jar:4.1.74.Final] at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) ~[io.netty-netty-common-4.1.74.Final.jar:4.1.74.Final] ``` - Improve the description - Add a check to prevent wrong settings (cherry picked from commit 8a568736a20a40d9b03e4753089ca2bd2220cf57) --- conf/proxy.conf | 10 +- .../proxy/server/ProxyConfiguration.java | 20 ++- .../proxy/server/ProxyServiceStarter.java | 24 +++- .../proxy/server/ProxyConfigurationTest.java | 119 ++++++++++++++++++ .../proxy/server/ProxyServiceStarterTest.java | 2 +- 5 files changed, 163 insertions(+), 12 deletions(-) diff --git a/conf/proxy.conf b/conf/proxy.conf index 4194bf7621985..8285e1cb75320 100644 --- a/conf/proxy.conf +++ b/conf/proxy.conf @@ -28,17 +28,19 @@ 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. +# If does not set metadataStoreUrl or configurationMetadataStoreUrl, this url should point to the discovery service +# provider, and does not support multi urls yet. # The URL must begin with pulsar:// for plaintext or with pulsar+ssl:// for TLS. brokerServiceURL= brokerServiceURLTLS= -# These settings are unnecessary if `zookeeperServers` is specified +# If does not set metadataStoreUrl or configurationMetadataStoreUrl, this url should point to the discovery service +# provider, and does not support multi urls yet. brokerWebServiceURL= brokerWebServiceURLTLS= -# If function workers are setup in a separate cluster, configure the following 2 settings -# to point to the function workers cluster +# If function workers are setup in a separate cluster, configure the following 2 settings. This url should point to +# the discovery service provider of the function workers cluster, and does not support multi urls yet. functionWorkerWebServiceURL= functionWorkerWebServiceURLTLS= 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 148eb579ed196..4ec5b3f77a3e1 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 @@ -173,23 +173,29 @@ public class ProxyConfiguration implements PulsarConfiguration { @FieldContext( category = CATEGORY_BROKER_DISCOVERY, - doc = "The service url points to the broker cluster. URL must have the pulsar:// prefix." + doc = "If does not set metadataStoreUrl or configurationMetadataStoreUrl, this url should point to the" + + " discovery service provider." + + " URL must have the pulsar:// prefix. And does not support multi url yet." ) private String brokerServiceURL; @FieldContext( category = CATEGORY_BROKER_DISCOVERY, - doc = "The tls service url points to the broker cluster. URL must have the pulsar+ssl:// prefix." + doc = "If does not set metadataStoreUrl or configurationMetadataStoreUrl, this url should point to the" + + " discovery service provider." + + " URL must have the pulsar+ssl:// prefix. And does not support multi url yet." ) private String brokerServiceURLTLS; @FieldContext( category = CATEGORY_BROKER_DISCOVERY, - doc = "The web service url points to the broker cluster" + doc = "The web service url points to the discovery service provider of the broker cluster, and does not support" + + " multi url yet." ) private String brokerWebServiceURL; @FieldContext( category = CATEGORY_BROKER_DISCOVERY, - doc = "The tls web service url points to the broker cluster" + doc = "The tls web service url points to the discovery service provider of the broker cluster, and does not" + + " support multi url yet." ) private String brokerWebServiceURLTLS; @@ -201,13 +207,15 @@ public class ProxyConfiguration implements PulsarConfiguration { @FieldContext( category = CATEGORY_BROKER_DISCOVERY, - doc = "The web service url points to the function worker cluster." + doc = "The web service url points to the discovery service provider of the function worker cluster, and does" + + " not support multi url yet." + " Only configure it when you setup function workers in a separate cluster" ) private String functionWorkerWebServiceURL; @FieldContext( category = CATEGORY_BROKER_DISCOVERY, - doc = "The tls web service url points to the function worker cluster." + doc = "The tls web service url points to the discovery service provider of the function worker cluster, and" + + " does not support multi url yet." + " Only configure it when you setup function workers in a separate cluster" ) private String functionWorkerWebServiceURLTLS; diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java index 7427331641318..e623d4b85aa09 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java @@ -159,11 +159,28 @@ public ProxyServiceStarter(String[] args) throws Exception { if (isNotBlank(config.getBrokerServiceURL())) { checkArgument(config.getBrokerServiceURL().startsWith("pulsar://"), "brokerServiceURL must start with pulsar://"); + ensureUrlNotContainsComma("brokerServiceURL", config.getBrokerServiceURL()); } - if (isNotBlank(config.getBrokerServiceURLTLS())) { checkArgument(config.getBrokerServiceURLTLS().startsWith("pulsar+ssl://"), "brokerServiceURLTLS must start with pulsar+ssl://"); + ensureUrlNotContainsComma("brokerServiceURLTLS", config.getBrokerServiceURLTLS()); + } + + if (isNotBlank(config.getBrokerWebServiceURL())) { + ensureUrlNotContainsComma("brokerWebServiceURL", config.getBrokerWebServiceURL()); + } + if (isNotBlank(config.getBrokerWebServiceURLTLS())) { + ensureUrlNotContainsComma("brokerWebServiceURLTLS", config.getBrokerWebServiceURLTLS()); + } + + if (isNotBlank(config.getFunctionWorkerWebServiceURL())) { + ensureUrlNotContainsComma("functionWorkerWebServiceURLTLS", + config.getFunctionWorkerWebServiceURL()); + } + if (isNotBlank(config.getFunctionWorkerWebServiceURLTLS())) { + ensureUrlNotContainsComma("functionWorkerWebServiceURLTLS", + config.getFunctionWorkerWebServiceURLTLS()); } if ((isBlank(config.getBrokerServiceURL()) && isBlank(config.getBrokerServiceURLTLS())) @@ -184,6 +201,11 @@ public ProxyServiceStarter(String[] args) throws Exception { } } + private void ensureUrlNotContainsComma(String paramName, String paramValue) { + checkArgument(!paramValue.contains(","), paramName + " does not support multi urls yet," + + " it should point to the discovery service provider."); + } + public static void main(String[] args) throws Exception { ProxyServiceStarter serviceStarter = new ProxyServiceStarter(args); try { 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 97a73c20b60d0..a9a562e04c899 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 @@ -20,6 +20,8 @@ import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; +import org.mockito.MockedStatic; +import org.mockito.Mockito; import org.testng.annotations.Test; import java.beans.Introspector; @@ -36,6 +38,8 @@ import java.util.Properties; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; @Test(groups = "broker") public class ProxyConfigurationTest { @@ -134,4 +138,119 @@ public void testConvert() throws IOException { } } + @Test + public void testBrokerUrlCheck() throws IOException { + ProxyConfiguration configuration = new ProxyConfiguration(); + // brokerServiceURL must start with pulsar:// + configuration.setBrokerServiceURL("127.0.0.1:6650"); + try (MockedStatic theMock = Mockito.mockStatic(PulsarConfigurationLoader.class)) { + theMock.when(PulsarConfigurationLoader.create(Mockito.anyString(), Mockito.any())) + .thenReturn(configuration); + try { + new ProxyServiceStarter(ProxyServiceStarterTest.ARGS); + fail("brokerServiceURL must start with pulsar://"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("brokerServiceURL must start with pulsar://")); + } + } + configuration.setBrokerServiceURL("pulsar://127.0.0.1:6650"); + + // brokerServiceURLTLS must start with pulsar+ssl:// + configuration.setBrokerServiceURLTLS("pulsar://127.0.0.1:6650"); + try (MockedStatic theMock = Mockito.mockStatic(PulsarConfigurationLoader.class)) { + theMock.when(PulsarConfigurationLoader.create(Mockito.anyString(), Mockito.any())) + .thenReturn(configuration); + try { + new ProxyServiceStarter(ProxyServiceStarterTest.ARGS); + fail("brokerServiceURLTLS must start with pulsar+ssl://"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("brokerServiceURLTLS must start with pulsar+ssl://")); + } + } + + // brokerServiceURL did not support multi urls yet. + configuration.setBrokerServiceURL("pulsar://127.0.0.1:6650,pulsar://127.0.0.2:6650"); + try (MockedStatic theMock = Mockito.mockStatic(PulsarConfigurationLoader.class)) { + theMock.when(PulsarConfigurationLoader.create(Mockito.anyString(), Mockito.any())) + .thenReturn(configuration); + try { + new ProxyServiceStarter(ProxyServiceStarterTest.ARGS); + fail("brokerServiceURL does not support multi urls yet"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("does not support multi urls yet")); + } + } + configuration.setBrokerServiceURL("pulsar://127.0.0.1:6650"); + + // brokerServiceURLTLS did not support multi urls yet. + configuration.setBrokerServiceURLTLS("pulsar+ssl://127.0.0.1:6650,pulsar+ssl:127.0.0.2:6650"); + try (MockedStatic theMock = Mockito.mockStatic(PulsarConfigurationLoader.class)) { + theMock.when(PulsarConfigurationLoader.create(Mockito.anyString(), Mockito.any())) + .thenReturn(configuration); + try { + new ProxyServiceStarter(ProxyServiceStarterTest.ARGS); + fail("brokerServiceURLTLS does not support multi urls yet"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("does not support multi urls yet")); + } + } + configuration.setBrokerServiceURLTLS("pulsar+ssl://127.0.0.1:6650"); + + // brokerWebServiceURL did not support multi urls yet. + configuration.setBrokerWebServiceURL("http://127.0.0.1:8080,http://127.0.0.2:8080"); + try (MockedStatic theMock = Mockito.mockStatic(PulsarConfigurationLoader.class)) { + theMock.when(PulsarConfigurationLoader.create(Mockito.anyString(), Mockito.any())) + .thenReturn(configuration); + try { + new ProxyServiceStarter(ProxyServiceStarterTest.ARGS); + fail("brokerWebServiceURL does not support multi urls yet"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("does not support multi urls yet")); + } + } + configuration.setBrokerWebServiceURL("http://127.0.0.1:8080"); + + // brokerWebServiceURLTLS did not support multi urls yet. + configuration.setBrokerWebServiceURLTLS("https://127.0.0.1:443,https://127.0.0.2:443"); + try (MockedStatic theMock = Mockito.mockStatic(PulsarConfigurationLoader.class)) { + theMock.when(PulsarConfigurationLoader.create(Mockito.anyString(), Mockito.any())) + .thenReturn(configuration); + try { + new ProxyServiceStarter(ProxyServiceStarterTest.ARGS); + fail("brokerWebServiceURLTLS does not support multi urls yet"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("does not support multi urls yet")); + } + } + configuration.setBrokerWebServiceURLTLS("https://127.0.0.1:443"); + + // functionWorkerWebServiceURL did not support multi urls yet. + configuration.setFunctionWorkerWebServiceURL("http://127.0.0.1:8080,http://127.0.0.2:8080"); + try (MockedStatic theMock = Mockito.mockStatic(PulsarConfigurationLoader.class)) { + theMock.when(PulsarConfigurationLoader.create(Mockito.anyString(), Mockito.any())) + .thenReturn(configuration); + try { + new ProxyServiceStarter(ProxyServiceStarterTest.ARGS); + fail("functionWorkerWebServiceURL does not support multi urls yet"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("does not support multi urls yet")); + } + } + configuration.setFunctionWorkerWebServiceURL("http://127.0.0.1:8080"); + + // functionWorkerWebServiceURLTLS did not support multi urls yet. + configuration.setFunctionWorkerWebServiceURLTLS("http://127.0.0.1:443,http://127.0.0.2:443"); + try (MockedStatic theMock = Mockito.mockStatic(PulsarConfigurationLoader.class)) { + theMock.when(PulsarConfigurationLoader.create(Mockito.anyString(), Mockito.any())) + .thenReturn(configuration); + try { + new ProxyServiceStarter(ProxyServiceStarterTest.ARGS); + fail("functionWorkerWebServiceURLTLS does not support multi urls yet"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("does not support multi urls yet")); + } + } + configuration.setFunctionWorkerWebServiceURLTLS("http://127.0.0.1:443"); + } + } \ No newline at end of file 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 def58be6df372..a9bead706a373 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 @@ -45,7 +45,7 @@ public class ProxyServiceStarterTest extends MockedPulsarServiceBaseTest { - static final String[] ARGS = new String[]{"-c", "./src/test/resources/proxy.conf"}; + public static final String[] ARGS = new String[]{"-c", "./src/test/resources/proxy.conf"}; protected ProxyServiceStarter serviceStarter; protected String serviceUrl; From 820423203cd91afe4d728feeefd188d30d2e21ed Mon Sep 17 00:00:00 2001 From: thetumbled <52550727+thetumbled@users.noreply.github.com> Date: Wed, 31 Jan 2024 00:11:07 +0800 Subject: [PATCH 254/699] [fix] [broker] Fix reader stuck when read from compacted topic with read compact mode disable (#21969) (cherry picked from commit 313eae5eff25df2d584773f059365f0edb82b35a) --- .../pulsar/broker/service/ServerCnx.java | 32 +++++++++++++++---- .../GetLastMessageIdCompactedTest.java | 27 ++++++++++++++++ 2 files changed, 52 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 cca53bf9d6aa3..5057b7b045a92 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 @@ -2079,7 +2079,8 @@ protected void handleGetLastMessageId(CommandGetLastMessageId getLastMessageId) (PositionImpl) markDeletePosition, partitionIndex, requestId, - consumer.getSubscription().getName()); + consumer.getSubscription().getName(), + consumer.readCompacted()); }).exceptionally(e -> { writeAndFlush(Commands.newError(getLastMessageId.getRequestId(), ServerError.UnknownError, "Failed to recover Transaction Buffer.")); @@ -2097,15 +2098,17 @@ private void getLargestBatchIndexWhenPossible( PositionImpl markDeletePosition, int partitionIndex, long requestId, - String subscriptionName) { + String subscriptionName, + boolean readCompacted) { PersistentTopic persistentTopic = (PersistentTopic) topic; ManagedLedgerImpl ml = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); // If it's not pointing to a valid entry, respond messageId of the current position. // If the compaction cursor reach the end of the topic, respond messageId from compacted ledger - CompletableFuture compactionHorizonFuture = - persistentTopic.getTopicCompactionService().getLastCompactedPosition(); + CompletableFuture compactionHorizonFuture = readCompacted + ? persistentTopic.getTopicCompactionService().getLastCompactedPosition() : + CompletableFuture.completedFuture(null); compactionHorizonFuture.whenComplete((compactionHorizon, ex) -> { if (ex != null) { @@ -2114,8 +2117,22 @@ private void getLargestBatchIndexWhenPossible( return; } - if (lastPosition.getEntryId() == -1 || (compactionHorizon != null - && lastPosition.compareTo((PositionImpl) compactionHorizon) <= 0)) { + if (lastPosition.getEntryId() == -1 || !ml.ledgerExists(lastPosition.getLedgerId())) { + // there is no entry in the original topic + if (compactionHorizon != null) { + // if readCompacted is true, we need to read the last entry from compacted topic + handleLastMessageIdFromCompactionService(persistentTopic, requestId, partitionIndex, + markDeletePosition); + } else { + // if readCompacted is false, we need to return MessageId.earliest + writeAndFlush(Commands.newGetLastMessageIdResponse(requestId, -1, -1, partitionIndex, -1, + markDeletePosition != null ? markDeletePosition.getLedgerId() : -1, + markDeletePosition != null ? markDeletePosition.getEntryId() : -1)); + } + return; + } + + if (compactionHorizon != null && lastPosition.compareTo((PositionImpl) compactionHorizon) <= 0) { handleLastMessageIdFromCompactionService(persistentTopic, requestId, partitionIndex, markDeletePosition); return; @@ -2144,7 +2161,8 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { batchSizeFuture.whenComplete((batchSize, e) -> { if (e != null) { - if (e.getCause() instanceof ManagedLedgerException.NonRecoverableLedgerException) { + if (e.getCause() instanceof ManagedLedgerException.NonRecoverableLedgerException + && readCompacted) { handleLastMessageIdFromCompactionService(persistentTopic, requestId, partitionIndex, markDeletePosition); } else { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/GetLastMessageIdCompactedTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/GetLastMessageIdCompactedTest.java index 317b1a227e585..6c2d848bb7c2d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/GetLastMessageIdCompactedTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/GetLastMessageIdCompactedTest.java @@ -20,6 +20,8 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotEquals; + import java.util.ArrayList; import java.util.List; import java.util.Optional; @@ -32,6 +34,7 @@ import org.apache.pulsar.broker.service.persistent.PersistentTopic; 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.ProducerBuilder; @@ -415,4 +418,28 @@ public void testGetLastMessageIdAfterCompactionAllNullMsg(boolean enabledBatch) producer.close(); admin.topics().delete(topicName, false); } + + @Test(dataProvider = "enabledBatch") + public void testReaderStuckWithCompaction(boolean enabledBatch) throws Exception { + String topicName = "persistent://public/default/" + BrokerTestUtil.newUniqueName("tp"); + String subName = "sub"; + Producer producer = createProducer(enabledBatch, topicName); + producer.newMessage().key("k0").value("v0").sendAsync(); + producer.newMessage().key("k0").value("v1").sendAsync(); + producer.flush(); + + triggerCompactionAndWait(topicName); + triggerLedgerSwitch(topicName); + clearAllTheLedgersOutdated(topicName); + + var reader = pulsarClient.newReader(Schema.STRING) + .topic(topicName) + .subscriptionName(subName) + .startMessageId(MessageId.earliest) + .create(); + while (reader.hasMessageAvailable()) { + Message message = reader.readNext(5, TimeUnit.SECONDS); + assertNotEquals(message, null); + } + } } From 41c74db7b5627d079888c7f000081c13f622e8b6 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 31 Jan 2024 00:31:15 +0800 Subject: [PATCH 255/699] [fix][client] Fix multi-topics consumer could receive old messages after seek (#21945) (cherry picked from commit dc1b107319cccb4920ac70053dd25fce239f5bca) --- .../client/impl/TopicsConsumerImplTest.java | 80 ++++++++++++++++++- .../client/impl/MultiTopicsConsumerImpl.java | 66 ++++++++++----- 2 files changed, 125 insertions(+), 21 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 51b32c2b44ecf..c343ab0d6e294 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 @@ -34,6 +34,7 @@ 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.MessageListener; import org.apache.pulsar.client.api.MessageRouter; import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; @@ -57,22 +58,27 @@ import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; - import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; import java.util.Set; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.TreeSet; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; 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.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -1394,4 +1400,76 @@ public Map getActiveConsumers() { } } + @DataProvider + public static Object[][] seekByFunction() { + return new Object[][] { + { true }, { false } + }; + } + + @Test(timeOut = 30000, dataProvider = "seekByFunction") + public void testSeekToNewerPosition(boolean seekByFunction) throws Exception { + final var topic1 = TopicName.get(newTopicName()).toString() + .replace("my-property", "public").replace("my-ns", "default"); + final var topic2 = TopicName.get(newTopicName()).toString() + .replace("my-property", "public").replace("my-ns", "default"); + @Cleanup final var producer1 = pulsarClient.newProducer(Schema.STRING).topic(topic1).create(); + @Cleanup final var producer2 = pulsarClient.newProducer(Schema.STRING).topic(topic2).create(); + producer1.send("1-0"); + producer2.send("2-0"); + producer1.send("1-1"); + producer2.send("2-1"); + final var consumer1 = pulsarClient.newConsumer(Schema.STRING) + .topics(Arrays.asList(topic1, topic2)).subscriptionName("sub") + .ackTimeout(1, TimeUnit.SECONDS) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest).subscribe(); + final var timestamps = new ArrayList(); + for (int i = 0; i < 4; i++) { + timestamps.add(consumer1.receive().getPublishTime()); + } + timestamps.sort(Comparator.naturalOrder()); + final var timestamp = timestamps.get(2); + consumer1.close(); + + final Function, CompletableFuture> seekAsync = consumer -> { + final var future = seekByFunction ? consumer.seekAsync(__ -> timestamp) : consumer.seekAsync(timestamp); + assertEquals(((ConsumerBase) consumer).getIncomingMessageSize(), 0L); + assertEquals(((ConsumerBase) consumer).getTotalIncomingMessages(), 0); + assertTrue(((ConsumerBase) consumer).getUnAckedMessageTracker().isEmpty()); + return future; + }; + + @Cleanup final var consumer2 = pulsarClient.newConsumer(Schema.STRING) + .topics(Arrays.asList(topic1, topic2)).subscriptionName("sub-2") + .ackTimeout(1, TimeUnit.SECONDS) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest).subscribe(); + seekAsync.apply(consumer2).get(); + final var values = new TreeSet(); + for (int i = 0; i < 2; i++) { + values.add(consumer2.receive().getValue()); + } + assertEquals(values, new TreeSet<>(Arrays.asList("1-1", "2-1"))); + + final var valuesInListener = new CopyOnWriteArrayList(); + @Cleanup final var consumer3 = pulsarClient.newConsumer(Schema.STRING) + .topics(Arrays.asList(topic1, topic2)).subscriptionName("sub-3") + .messageListener((MessageListener) (__, msg) -> valuesInListener.add(msg.getValue())) + .ackTimeout(1, TimeUnit.SECONDS) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest).subscribe(); + seekAsync.apply(consumer3).get(); + if (valuesInListener.isEmpty()) { + Awaitility.await().untilAsserted(() -> assertEquals(valuesInListener.size(), 2)); + assertEquals(valuesInListener.stream().sorted().toList(), Arrays.asList("1-1", "2-1")); + } // else: consumer3 has passed messages to the listener before seek, in this case we cannot assume anything + + @Cleanup final var consumer4 = pulsarClient.newConsumer(Schema.STRING) + .topics(Arrays.asList(topic1, topic2)).subscriptionName("sub-4") + .ackTimeout(1, TimeUnit.SECONDS) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest).subscribe(); + seekAsync.apply(consumer4).get(); + final var valuesInReceiveAsync = new ArrayList(); + valuesInReceiveAsync.add(consumer4.receiveAsync().get().getValue()); + valuesInReceiveAsync.add(consumer4.receiveAsync().get().getValue()); + assertEquals(valuesInReceiveAsync.stream().sorted().toList(), Arrays.asList("1-1", "2-1")); + } } 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 68860cf1e2929..4240ad55b096b 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 @@ -49,6 +49,7 @@ import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; +import javax.annotation.Nullable; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.api.BatchReceivePolicy; import org.apache.pulsar.client.api.Consumer; @@ -101,7 +102,8 @@ public class MultiTopicsConsumerImpl extends ConsumerBase { private final MultiTopicConsumerStatsRecorderImpl stats; private final ConsumerConfigurationData internalConfig; - private volatile MessageIdAdv startMessageId; + private final MessageIdAdv startMessageId; + private volatile boolean duringSeek = false; private final long startMessageRollbackDurationInSec; MultiTopicsConsumerImpl(PulsarClientImpl client, ConsumerConfigurationData conf, ExecutorProvider executorProvider, CompletableFuture> subscribeFuture, Schema schema, @@ -235,6 +237,10 @@ private void startReceivingMessages(List> newConsumers) { } private void receiveMessageFromConsumer(ConsumerImpl consumer, boolean batchReceive) { + if (duringSeek) { + log.info("[{}] Pause receiving messages for topic {} due to seek", subscription, consumer.getTopic()); + return; + } CompletableFuture>> messagesFuture; if (batchReceive) { messagesFuture = consumer.batchReceiveAsync().thenApply(msgs -> ((MessagesImpl) msgs).getMessageList()); @@ -252,8 +258,12 @@ private void receiveMessageFromConsumer(ConsumerImpl consumer, boolean batchR } // Process the message, add to the queue and trigger listener or async callback messages.forEach(msg -> { - if (isValidConsumerEpoch((MessageImpl) msg)) { + final boolean skipDueToSeek = duringSeek; + if (isValidConsumerEpoch((MessageImpl) msg) && !skipDueToSeek) { messageReceived(consumer, msg); + } else if (skipDueToSeek) { + log.info("[{}] [{}] Skip processing message {} received during seek", topic, subscription, + msg.getMessageId()); } }); @@ -748,17 +758,12 @@ public void seek(Function function) throws PulsarClientException @Override public CompletableFuture seekAsync(Function function) { - List> futures = new ArrayList<>(consumers.size()); - consumers.values().forEach(consumer -> futures.add(consumer.seekAsync(function))); - unAckedMessageTracker.clear(); - incomingMessages.clear(); - resetIncomingMessageSize(); - return FutureUtil.waitForAll(futures); + return seekAllAsync(consumer -> consumer.seekAsync(function)); } @Override public CompletableFuture seekAsync(MessageId messageId) { - final Consumer internalConsumer; + final ConsumerImpl internalConsumer; if (messageId instanceof TopicMessageId) { TopicMessageId topicMessageId = (TopicMessageId) messageId; internalConsumer = consumers.get(topicMessageId.getOwnerTopic()); @@ -775,25 +780,46 @@ public CompletableFuture seekAsync(MessageId messageId) { ); } - final CompletableFuture seekFuture; if (internalConsumer == null) { - List> futures = new ArrayList<>(consumers.size()); - consumers.values().forEach(consumerImpl -> futures.add(consumerImpl.seekAsync(messageId))); - seekFuture = FutureUtil.waitForAll(futures); + return seekAllAsync(consumer -> consumer.seekAsync(messageId)); } else { - seekFuture = internalConsumer.seekAsync(messageId); + return seekAsyncInternal(Collections.singleton(internalConsumer), __ -> __.seekAsync(messageId)); } + } + + @Override + public CompletableFuture seekAsync(long timestamp) { + return seekAllAsync(consumer -> consumer.seekAsync(timestamp)); + } + private CompletableFuture seekAsyncInternal(Collection> consumers, + Function, CompletableFuture> seekFunc) { + beforeSeek(); + final CompletableFuture future = new CompletableFuture<>(); + FutureUtil.waitForAll(consumers.stream().map(seekFunc).collect(Collectors.toList())) + .whenComplete((__, e) -> afterSeek(future, e)); + return future; + } + + private CompletableFuture seekAllAsync(Function, CompletableFuture> seekFunc) { + return seekAsyncInternal(consumers.values(), seekFunc); + } + + private void beforeSeek() { + duringSeek = true; unAckedMessageTracker.clear(); clearIncomingMessages(); - return seekFuture; } - @Override - public CompletableFuture seekAsync(long timestamp) { - List> futures = new ArrayList<>(consumers.size()); - consumers.values().forEach(consumer -> futures.add(consumer.seekAsync(timestamp))); - return FutureUtil.waitForAll(futures); + private void afterSeek(CompletableFuture seekFuture, @Nullable Throwable throwable) { + duringSeek = false; + log.info("[{}] Resume receiving messages for {} since seek is done", subscription, consumers.keySet()); + startReceivingMessages(new ArrayList<>(consumers.values())); + if (throwable == null) { + seekFuture.complete(null); + } else { + seekFuture.completeExceptionally(throwable); + } } @Override From 08e3cbe8c69790648965014ed64d52d0519b40c2 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Fri, 19 Jan 2024 11:18:02 +0800 Subject: [PATCH 256/699] [fix][broker] Fix getMessageById throws 500 (#21919) Signed-off-by: Zixuan Liu (cherry picked from commit 6f7b9d9a6147c0de3550a091f5cfba0389d98cec) --- .../admin/impl/PersistentTopicsBase.java | 3 +++ .../broker/admin/PersistentTopicsTest.java | 21 ++++++------------- .../client/admin/internal/TopicsImpl.java | 16 +------------- 3 files changed, 10 insertions(+), 30 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 121b57fc8eea7..758b8048f3d13 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 @@ -2829,6 +2829,9 @@ protected CompletableFuture internalGetMessageById(long ledgerId, long @Override public void readEntryFailed(ManagedLedgerException exception, Object ctx) { + if (exception instanceof ManagedLedgerException.LedgerNotExistException) { + throw new RestException(Status.NOT_FOUND, "Message id not found"); + } throw new RestException(exception); } 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 25ad6cab94272..10321cb776101 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 @@ -1358,21 +1358,12 @@ public void testGetMessageById() throws Exception { Message message2 = admin.topics().getMessageById(topicName2, id2.getLedgerId(), id2.getEntryId()); Assert.assertEquals(message2.getData(), data2.getBytes()); - Message message3 = null; - try { - message3 = admin.topics().getMessageById(topicName2, id1.getLedgerId(), id1.getEntryId()); - Assert.fail(); - } catch (Exception e) { - Assert.assertNull(message3); - } - - Message message4 = null; - try { - message4 = admin.topics().getMessageById(topicName1, id2.getLedgerId(), id2.getEntryId()); - Assert.fail(); - } catch (Exception e) { - Assert.assertNull(message4); - } + Assert.expectThrows(PulsarAdminException.NotFoundException.class, () -> { + admin.topics().getMessageById(topicName2, id1.getLedgerId(), id1.getEntryId()); + }); + Assert.expectThrows(PulsarAdminException.NotFoundException.class, () -> { + admin.topics().getMessageById(topicName1, id2.getLedgerId(), id2.getEntryId()); + }); } @Test diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java index e0c64319ea2d9..a89da628372bd 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java @@ -965,21 +965,7 @@ public CompletableFuture truncateAsync(String topic) { @Override public CompletableFuture> getMessageByIdAsync(String topic, long ledgerId, long entryId) { - CompletableFuture> future = new CompletableFuture<>(); - getRemoteMessageById(topic, ledgerId, entryId).handle((r, ex) -> { - if (ex != null) { - if (ex instanceof NotFoundException) { - log.warn("Exception '{}' occurred while trying to get message.", ex.getMessage()); - future.complete(r); - } else { - future.completeExceptionally(ex); - } - return null; - } - future.complete(r); - return null; - }); - return future; + return getRemoteMessageById(topic, ledgerId, entryId); } private CompletableFuture> getRemoteMessageById(String topic, long ledgerId, long entryId) { From 06506a12cbf79511e0790bd1876d738542d68923 Mon Sep 17 00:00:00 2001 From: hrzzzz <64506104+hrzzzz@users.noreply.github.com> Date: Mon, 22 Jan 2024 14:23:53 +0800 Subject: [PATCH 257/699] [fix][broker] Fix deadlock while skip non-recoverable ledgers. (#21915) ### Motivation The sequence of events leading to the deadlock when methods from org.apache.bookkeeper.mledger.impl.ManagedCursorImpl are invoked concurrently is as follows: 1. Thread A calls asyncDelete, which then goes on to internally call internalAsyncMarkDelete. This results in acquiring a lock on pendingMarkDeleteOps through synchronized (pendingMarkDeleteOps). 2. Inside internalAsyncMarkDelete, internalMarkDelete is called which subsequently calls persistPositionToLedger. At the start of persistPositionToLedger, buildIndividualDeletedMessageRanges is invoked, where it tries to acquire a read lock using lock.readLock().lock(). At this point, if the write lock is being held by another thread, Thread A will block waiting for the read lock. 3. Concurrently, Thread B executes skipNonRecoverableLedger which first obtains a write lock using lock.writeLock().lock() and then proceeds to call asyncDelete. 4. At this moment, Thread B already holds the write lock and is attempting to acquire the synchronized lock on pendingMarkDeleteOps that Thread A already holds, while Thread A is waiting for the read lock that Thread B needs to release. In code, the deadlock appears as follows: Thread A: synchronized (pendingMarkDeleteOps) -> lock.readLock().lock() (waiting) Thread B: lock.writeLock().lock() -> synchronized (pendingMarkDeleteOps) (waiting) ### Modifications Avoid using a long-range lock. Co-authored-by: ruihongzhou Co-authored-by: Jiwe Guo Co-authored-by: Lari Hotari (cherry picked from commit 37fc40cd9db7c85e547eed02346f90e08da05103) --- .../mledger/impl/ManagedCursorImpl.java | 36 ++++++++----------- 1 file changed, 15 insertions(+), 21 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 ea013d2da7dd7..322034c3cbbf8 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.LongStream; import org.apache.bookkeeper.client.AsyncCallback.CloseCallback; import org.apache.bookkeeper.client.AsyncCallback.OpenCallback; import org.apache.bookkeeper.client.BKException; @@ -2755,30 +2756,23 @@ public void skipNonRecoverableLedger(final long ledgerId){ if (ledgerInfo == null) { return; } - lock.writeLock().lock(); log.warn("[{}] [{}] Since the ledger [{}] is lost and the autoSkipNonRecoverableData is true, this ledger will" + " be auto acknowledge in subscription", ledger.getName(), name, ledgerId); - try { - for (int i = 0; i < ledgerInfo.getEntries(); i++) { - if (!individualDeletedMessages.contains(ledgerId, i)) { - asyncDelete(PositionImpl.get(ledgerId, i), new AsyncCallbacks.DeleteCallback() { - @Override - public void deleteComplete(Object ctx) { - // ignore. - } + asyncDelete(() -> LongStream.range(0, ledgerInfo.getEntries()) + .mapToObj(i -> (Position) PositionImpl.get(ledgerId, i)).iterator(), + 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(); - } + @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); } // ////////////////////////////////////////////////// From 8134790dca8839ec8554eb399fe019b9be7e7a89 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 25 Jan 2024 17:02:26 +0800 Subject: [PATCH 258/699] [fix] [broker] Replication stopped due to unload topic failed (#21947) ### Motivation **Steps to reproduce the issue** - Enable replication. - Send `10` messages to the local cluster then close the producer. - Call `pulsar-admin topics unload ` and get an error due to the internal producer of the replicator close failing. - The topic closed failed, so we assumed the topic could work as expected, but the replication stopped. **Root cause** - `pulsar-admin topics unload ` will wait for the clients(including `consumers & producers & replicators`) to close successfully, and it will fail if clients can not be closed successfully. - `replicator.producer` close failed causing the Admin API to fail, but there is a scheduled task that will retry to close `replicator.producer` which causes replication to stop. see https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java#L209 ### Modifications since the "replicator.producer.closeAsync()" will retry after it fails, the topic unload should be successful. (cherry picked from commit 291bbb531c24fd00d6850b0b5bd88f3438e1b775) --- .../broker/service/AbstractReplicator.java | 3 +- .../broker/service/OneWayReplicatorTest.java | 70 ++++++++++++++++--- 2 files changed, 63 insertions(+), 10 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java index 6dd296d16b53b..1b5b2824257b0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java @@ -194,7 +194,7 @@ protected synchronized CompletableFuture closeProducerAsync() { return CompletableFuture.completedFuture(null); } CompletableFuture future = producer.closeAsync(); - future.thenRun(() -> { + return future.thenRun(() -> { STATE_UPDATER.set(this, State.Stopped); this.producer = null; // deactivate further read @@ -209,7 +209,6 @@ protected synchronized CompletableFuture closeProducerAsync() { brokerService.executor().schedule(this::closeProducerAsync, waitTimeMs, TimeUnit.MILLISECONDS); return null; }); - return future; } 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 e068cd55e9055..5d748a9689e2f 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 @@ -21,16 +21,22 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; +import java.lang.reflect.Field; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.persistent.PersistentReplicator; 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.Schema; +import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.common.policies.data.TopicStats; import org.awaitility.Awaitility; +import org.mockito.Mockito; +import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -50,6 +56,29 @@ public void cleanup() throws Exception { super.cleanup(); } + private void waitReplicatorStarted(String topicName) { + Awaitility.await().untilAsserted(() -> { + Optional topicOptional2 = pulsar2.getBrokerService().getTopic(topicName, false).get(); + assertTrue(topicOptional2.isPresent()); + PersistentTopic persistentTopic2 = (PersistentTopic) topicOptional2.get(); + assertFalse(persistentTopic2.getProducers().isEmpty()); + }); + } + + /** + * Override "AbstractReplicator.producer" by {@param producer} and return the original value. + */ + private ProducerImpl overrideProducerForReplicator(AbstractReplicator replicator, ProducerImpl newProducer) + throws Exception { + Field producerField = AbstractReplicator.class.getDeclaredField("producer"); + producerField.setAccessible(true); + ProducerImpl originalValue = (ProducerImpl) producerField.get(replicator); + synchronized (replicator) { + producerField.set(replicator, newProducer); + } + return originalValue; + } + @Test public void testReplicatorProducerStatInTopic() throws Exception { final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); @@ -85,18 +114,13 @@ public void testReplicatorProducerStatInTopic() throws Exception { public void testCreateRemoteConsumerFirst() throws Exception { final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); Producer producer1 = client1.newProducer(Schema.STRING).topic(topicName).create(); - // Wait for replicator started. - Awaitility.await().untilAsserted(() -> { - Optional topicOptional2 = pulsar2.getBrokerService().getTopic(topicName, false).get(); - assertTrue(topicOptional2.isPresent()); - PersistentTopic persistentTopic2 = (PersistentTopic) topicOptional2.get(); - assertFalse(persistentTopic2.getProducers().isEmpty()); - }); + // The topic in cluster2 has a replicator created producer(schema Auto_Produce), but does not have any schema。 // Verify: the consumer of this cluster2 can create successfully. Consumer consumer2 = client2.newConsumer(Schema.STRING).topic(topicName).subscriptionName("s1") .subscribe();; - + // Wait for replicator started. + waitReplicatorStarted(topicName); // cleanup. producer1.close(); consumer2.close(); @@ -105,4 +129,34 @@ public void testCreateRemoteConsumerFirst() throws Exception { admin2.topics().delete(topicName); }); } + + @Test + public void testTopicCloseWhenInternalProducerCloseErrorOnce() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp_"); + admin1.topics().createNonPartitionedTopic(topicName); + // Wait for replicator started. + waitReplicatorStarted(topicName); + PersistentTopic persistentTopic = + (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get(); + PersistentReplicator replicator = + (PersistentReplicator) persistentTopic.getReplicators().values().iterator().next(); + // Mock an error when calling "replicator.disconnect()" + ProducerImpl mockProducer = Mockito.mock(ProducerImpl.class); + Mockito.when(mockProducer.closeAsync()).thenReturn(CompletableFuture.failedFuture(new Exception("mocked ex"))); + ProducerImpl originalProducer = overrideProducerForReplicator(replicator, mockProducer); + // Verify: since the "replicator.producer.closeAsync()" will retry after it failed, the topic unload should be + // successful. + admin1.topics().unload(topicName); + // Verify: After "replicator.producer.closeAsync()" retry again, the "replicator.producer" will be closed + // successful. + overrideProducerForReplicator(replicator, originalProducer); + Awaitility.await().untilAsserted(() -> { + Assert.assertFalse(replicator.isConnected()); + }); + // cleanup. + cleanupTopics(() -> { + admin1.topics().delete(topicName); + admin2.topics().delete(topicName); + }); + } } From f343bc39e9a4baa0598edac5f601cedd706cbab2 Mon Sep 17 00:00:00 2001 From: thetumbled <52550727+thetumbled@users.noreply.github.com> Date: Sun, 21 Jan 2024 19:47:40 +0800 Subject: [PATCH 259/699] [fix] [broker] Fix write all compacted out entry into compacted topic (#21917) (cherry picked from commit e7f1d03f4a0b6ad11886f81ed5f10c3069c00c32) --- .../pulsar/client/impl/RawBatchConverter.java | 6 +- .../pulsar/compaction/CompactorTest.java | 56 +++++++++++++++++++ 2 files changed, 61 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java index b6be228788a7f..008f45e63cb3c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/client/impl/RawBatchConverter.java @@ -140,7 +140,11 @@ public static Optional rebatchMessage(RawMessage msg, msg.getMessageIdData().getEntryId(), msg.getMessageIdData().getPartition(), i); - if (!singleMessageMetadata.hasPartitionKey()) { + if (singleMessageMetadata.isCompactedOut()) { + // we may read compacted out message from the compacted topic + Commands.serializeSingleMessageInBatchWithPayload(emptyMetadata, + Unpooled.EMPTY_BUFFER, batchBuffer); + } else if (!singleMessageMetadata.hasPartitionKey()) { if (retainNullKey) { messagesRetained++; Commands.serializeSingleMessageInBatchWithPayload(singleMessageMetadata, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java index 4e442ac051326..71700ef83a443 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactorTest.java @@ -36,6 +36,8 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + import lombok.Cleanup; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.client.LedgerEntry; @@ -46,10 +48,15 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.admin.LongRunningProcessStatus; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.RawMessage; +import org.apache.pulsar.client.api.Reader; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.RawMessageImpl; @@ -177,6 +184,55 @@ public void testCompaction() throws Exception { compactAndVerify(topic, expected, true); } + @Test + public void testAllCompactedOut() throws Exception { + String topicName = "persistent://my-property/use/my-ns/testAllCompactedOut"; + // set retain null key to true + boolean oldRetainNullKey = pulsar.getConfig().isTopicCompactionRetainNullKey(); + pulsar.getConfig().setTopicCompactionRetainNullKey(true); + this.restartBroker(); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .enableBatching(true).topic(topicName).batchingMaxMessages(3).create(); + + producer.newMessage().key("K1").value("V1").sendAsync(); + producer.newMessage().key("K2").value("V2").sendAsync(); + producer.newMessage().key("K2").value(null).sendAsync(); + producer.flush(); + + admin.topics().triggerCompaction(topicName); + + Awaitility.await().untilAsserted(() -> { + Assert.assertEquals(admin.topics().compactionStatus(topicName).status, + LongRunningProcessStatus.Status.SUCCESS); + }); + + producer.newMessage().key("K1").value(null).sendAsync(); + producer.flush(); + + admin.topics().triggerCompaction(topicName); + + Awaitility.await().untilAsserted(() -> { + Assert.assertEquals(admin.topics().compactionStatus(topicName).status, + LongRunningProcessStatus.Status.SUCCESS); + }); + + @Cleanup + Reader reader = pulsarClient.newReader(Schema.STRING) + .subscriptionName("reader-test") + .topic(topicName) + .readCompacted(true) + .startMessageId(MessageId.earliest) + .create(); + while (reader.hasMessageAvailable()) { + Message message = reader.readNext(3, TimeUnit.SECONDS); + Assert.assertNotNull(message); + } + // set retain null key back to avoid affecting other tests + pulsar.getConfig().setTopicCompactionRetainNullKey(oldRetainNullKey); + } + @Test public void testCompactAddCompact() throws Exception { String topic = "persistent://my-property/use/my-ns/my-topic1"; From 0b621d63d899d36b35209abe487e6bd5e2ad3f70 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Fri, 26 Jan 2024 17:26:45 +0800 Subject: [PATCH 260/699] [fix][broker] Avoid consumers receiving acknowledged messages from compacted topic after reconnection (#21187) (cherry picked from commit 24d8d9aaccbe3d3c435b08cd5e3ee96f2f31b2d0) --- .../bookkeeper/mledger/ManagedCursor.java | 4 + .../mledger/impl/ManagedCursorImpl.java | 10 +- ...sistentDispatcherSingleActiveConsumer.java | 24 ++- .../service/persistent/PersistentTopic.java | 5 +- .../pulsar/compaction/CompactedTopicImpl.java | 6 +- .../service/ReplicatorSubscriptionTest.java | 2 + .../broker/transaction/TransactionTest.java | 59 +++++++ .../apache/pulsar/client/impl/ReaderTest.java | 28 +++ .../pulsar/compaction/CompactionTest.java | 164 +++++++++++++++++- 9 files changed, 287 insertions(+), 15 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 d1ffdf6d2d763..bc6a1e9a782d6 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 @@ -517,6 +517,10 @@ void markDelete(Position position, Map properties) */ void rewind(); + default void rewind(boolean readCompacted) { + rewind(); + } + /** * Move the cursor to a different read position. * 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 322034c3cbbf8..7a58bf7dd3698 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 @@ -677,7 +677,7 @@ private void recoveredCursor(PositionImpl position, Map properties LedgerHandle recoveredFromCursorLedger) { // if the position was at a ledger that didn't exist (since it will be deleted if it was previously empty), // we need to move to the next existing ledger - if (!ledger.ledgerExists(position.getLedgerId())) { + if (position.getEntryId() == -1L && !ledger.ledgerExists(position.getLedgerId())) { Long nextExistingLedger = ledger.getNextValidLedger(position.getLedgerId()); if (nextExistingLedger == null) { log.info("[{}] [{}] Couldn't find next next valid ledger for recovery {}", ledger.getName(), name, @@ -2499,9 +2499,15 @@ public Position getPersistentMarkDeletedPosition() { @Override public void rewind() { + rewind(false); + } + + @Override + public void rewind(boolean readCompacted) { lock.writeLock().lock(); try { - PositionImpl newReadPosition = ledger.getNextValidPosition(markDeletePosition); + PositionImpl newReadPosition = + readCompacted ? markDeletePosition.getNext() : ledger.getNextValidPosition(markDeletePosition); PositionImpl oldReadPosition = readPosition; log.info("[{}-{}] Rewind from {} to {}", ledger.getName(), name, oldReadPosition, newReadPosition); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java index 5e9183df0b1df..15f34258768c5 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java @@ -55,6 +55,7 @@ import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; import org.apache.pulsar.common.util.Codec; import org.apache.pulsar.compaction.CompactedTopicUtils; +import org.apache.pulsar.compaction.Compactor; import org.apache.pulsar.compaction.TopicCompactionService; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -108,9 +109,9 @@ protected void scheduleReadOnActiveConsumer() { if (log.isDebugEnabled()) { log.debug("[{}] Rewind cursor and read more entries without delay", name); } - cursor.rewind(); - Consumer activeConsumer = ACTIVE_CONSUMER_UPDATER.get(this); + cursor.rewind(activeConsumer != null && activeConsumer.readCompacted()); + notifyActiveConsumerChanged(activeConsumer); readMoreEntries(activeConsumer); return; @@ -128,9 +129,9 @@ protected void scheduleReadOnActiveConsumer() { log.debug("[{}] Rewind cursor and read more entries after {} ms delay", name, serviceConfig.getActiveConsumerFailoverDelayTimeMillis()); } - cursor.rewind(); - Consumer activeConsumer = ACTIVE_CONSUMER_UPDATER.get(this); + cursor.rewind(activeConsumer != null && activeConsumer.readCompacted()); + notifyActiveConsumerChanged(activeConsumer); readMoreEntries(activeConsumer); readOnActiveConsumerTask = null; @@ -201,7 +202,7 @@ public synchronized void internalReadEntriesComplete(final List entries, log.debug("[{}] rewind because no available consumer found", name); } entries.forEach(Entry::release); - cursor.rewind(); + cursor.rewind(currentConsumer != null ? currentConsumer.readCompacted() : readConsumer.readCompacted()); if (currentConsumer != null) { notifyActiveConsumerChanged(currentConsumer); readMoreEntries(currentConsumer); @@ -296,7 +297,7 @@ private synchronized void internalRedeliverUnacknowledgedMessages(Consumer consu } cursor.cancelPendingReadRequest(); havePendingRead = false; - cursor.rewind(); + cursor.rewind(consumer.readCompacted()); if (log.isDebugEnabled()) { log.debug("[{}-{}] Cursor rewinded, redelivering unacknowledged messages. ", name, consumer); } @@ -350,7 +351,9 @@ protected void readMoreEntries(Consumer consumer) { } havePendingRead = true; if (consumer.readCompacted()) { - boolean readFromEarliest = isFirstRead && MessageId.earliest.equals(consumer.getStartMessageId()); + boolean readFromEarliest = isFirstRead && MessageId.earliest.equals(consumer.getStartMessageId()) + && (!cursor.isDurable() || cursor.getName().equals(Compactor.COMPACTION_SUBSCRIPTION) + || hasValidMarkDeletePosition(cursor)); TopicCompactionService topicCompactionService = topic.getTopicCompactionService(); CompactedTopicUtils.asyncReadCompactedEntries(topicCompactionService, cursor, messagesToRead, bytesToRead, topic.getMaxReadPosition(), readFromEarliest, this, true, consumer); @@ -368,6 +371,13 @@ protected void readMoreEntries(Consumer consumer) { } } + private boolean hasValidMarkDeletePosition(ManagedCursor cursor) { + // If `markDeletedPosition.entryID == -1L` then the md-position is an invalid position, + // since the initial md-position of the consumer will be set to it. + // See ManagedLedgerImpl#asyncOpenCursor and ManagedLedgerImpl#getFirstPosition + return cursor.getMarkDeletedPosition() != null && cursor.getMarkDeletedPosition().getEntryId() == -1L; + } + @Override protected void reScheduleRead() { if (isRescheduleReadInProgress.compareAndSet(false, true)) { 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 f07578667e8d4..12f0f738bb816 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 @@ -978,7 +978,9 @@ public CompletableFuture subscribe(final TransportCnx cnx, String subs } private CompletableFuture getDurableSubscription(String subscriptionName, - InitialPosition initialPosition, long startMessageRollbackDurationSec, boolean replicated, + InitialPosition initialPosition, + long startMessageRollbackDurationSec, + boolean replicated, Map subscriptionProperties) { CompletableFuture subscriptionFuture = new CompletableFuture<>(); if (checkMaxSubscriptionsPerTopicExceed(subscriptionName)) { @@ -988,7 +990,6 @@ private CompletableFuture getDurableSubscription(String subscripti } Map properties = PersistentSubscription.getBaseCursorProperties(replicated); - ledger.asyncOpenCursor(Codec.encode(subscriptionName), initialPosition, properties, subscriptionProperties, new OpenCursorCallback() { @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java index a8e124c84a250..ab54e96b71f1b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java @@ -100,7 +100,11 @@ public void asyncReadEntriesOrWait(ManagedCursor cursor, boolean isFirstRead, ReadEntriesCallback callback, Consumer consumer) { PositionImpl cursorPosition; - if (isFirstRead && MessageId.earliest.equals(consumer.getStartMessageId())){ + boolean readFromEarliest = isFirstRead && MessageId.earliest.equals(consumer.getStartMessageId()) + && (!cursor.isDurable() || cursor.getName().equals(Compactor.COMPACTION_SUBSCRIPTION) + || cursor.getMarkDeletedPosition() == null + || cursor.getMarkDeletedPosition().getEntryId() == -1L); + if (readFromEarliest){ cursorPosition = PositionImpl.EARLIEST; } else { cursorPosition = (PositionImpl) cursor.getReadPosition(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java index fe519827be74a..4cc3a9ada7d04 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java @@ -52,6 +52,7 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; 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.common.policies.data.PartitionedTopicStats; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; @@ -868,6 +869,7 @@ public void testReplicatedSubscriptionWithCompaction() throws Exception { .topic(topicName) .subscriptionName("sub2") .subscriptionType(SubscriptionType.Exclusive) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) .readCompacted(true) .subscribe(); List result = new ArrayList<>(); 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 c41776ec69408..3f0ef0ea74208 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 @@ -1838,4 +1838,63 @@ public void testReadCommittedWithReadCompacted() throws Exception{ Assert.assertEquals(messages, List.of("V2", "V3")); } + + @Test + public void testReadCommittedWithCompaction() throws Exception{ + final String namespace = "tnx/ns-prechecks"; + final String topic = "persistent://" + namespace + "/test_transaction_topic" + UUID.randomUUID(); + admin.namespaces().createNamespace(namespace); + admin.topics().createNonPartitionedTopic(topic); + + admin.topicPolicies().setCompactionThreshold(topic, 100 * 1024 * 1024); + + @Cleanup + Producer producer = this.pulsarClient.newProducer(Schema.STRING) + .topic(topic) + .create(); + + producer.newMessage().key("K1").value("V1").send(); + + Transaction txn = pulsarClient.newTransaction() + .withTransactionTimeout(1, TimeUnit.MINUTES).build().get(); + producer.newMessage(txn).key("K2").value("V2").send(); + producer.newMessage(txn).key("K3").value("V3").send(); + txn.commit().get(); + + producer.newMessage().key("K1").value("V4").send(); + + Transaction txn2 = pulsarClient.newTransaction() + .withTransactionTimeout(1, TimeUnit.MINUTES).build().get(); + producer.newMessage(txn2).key("K2").value("V5").send(); + producer.newMessage(txn2).key("K3").value("V6").send(); + txn2.commit().get(); + + admin.topics().triggerCompaction(topic); + + Awaitility.await().untilAsserted(() -> { + assertEquals(admin.topics().compactionStatus(topic).status, + LongRunningProcessStatus.Status.SUCCESS); + }); + + @Cleanup + Consumer consumer = this.pulsarClient.newConsumer(Schema.STRING) + .topic(topic) + .subscriptionName("sub") + .subscriptionType(SubscriptionType.Exclusive) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .readCompacted(true) + .subscribe(); + List result = new ArrayList<>(); + while (true) { + Message receive = consumer.receive(2, TimeUnit.SECONDS); + if (receive == null) { + break; + } + + result.add(receive.getValue()); + } + + Assert.assertEquals(result, List.of("V4", "V5", "V6")); + } + } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java index a50c92f7ab8f4..85ff87593cffa 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java @@ -733,4 +733,32 @@ public void testReaderListenerAcknowledgement() admin.topics().deletePartitionedTopic(partitionedTopic); } + @Test + public void testReaderReconnectedFromNextEntry() throws Exception { + final String topic = "persistent://my-property/my-ns/testReaderReconnectedFromNextEntry"; + Reader reader = pulsarClient.newReader(Schema.STRING).topic(topic).receiverQueueSize(1) + .startMessageId(MessageId.earliest).create(); + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + + // Send 3 and consume 1. + producer.send("1"); + producer.send("2"); + producer.send("3"); + Message msg1 = reader.readNext(2, TimeUnit.SECONDS); + assertEquals(msg1.getValue(), "1"); + + // Trigger reader reconnect. + admin.topics().unload(topic); + + // For non-durable we are going to restart from the next entry. + Message msg2 = reader.readNext(2, TimeUnit.SECONDS); + assertEquals(msg2.getValue(), "2"); + Message msg3 = reader.readNext(2, TimeUnit.SECONDS); + assertEquals(msg3.getValue(), "3"); + + // cleanup. + reader.close(); + producer.close(); + admin.topics().delete(topic, false); + } } 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 38e559c3fe602..a07f2ca8391af 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 @@ -1876,6 +1876,7 @@ public void testReceiverQueueSize() throws Exception { ConsumerImpl consumer = (ConsumerImpl) client.newConsumer(Schema.STRING) .topic(topicName).readCompacted(true).receiverQueueSize(receiveQueueSize).subscriptionName(subName) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) .subscribe(); //Give some time to consume @@ -1919,6 +1920,7 @@ public void testDispatcherMaxReadSizeBytes() throws Exception { ConsumerImpl consumer = (ConsumerImpl) client.newConsumer(Schema.BYTES) .topic(topicName).readCompacted(true).receiverQueueSize(receiveQueueSize).subscriptionName(subName) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) .subscribe(); Awaitility.await().untilAsserted(() -> { @@ -2168,9 +2170,11 @@ public void testCompactionWithTTL() throws Exception { }); @Cleanup - Consumer consumer = - pulsarClient.newConsumer(Schema.STRING).topic(topicName).subscriptionName(subName).readCompacted(true) - .subscribe(); + Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName) + .subscriptionName("sub-2") + .readCompacted(true) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); List result = new ArrayList<>(); while (true) { @@ -2184,4 +2188,158 @@ public void testCompactionWithTTL() throws Exception { Assert.assertEquals(result, List.of("V3", "V4", "V5")); } + + @Test + public void testAcknowledgeWithReconnection() throws Exception { + final String topicName = "persistent://my-property/use/my-ns/testAcknowledge" + UUID.randomUUID(); + final String subName = "my-sub"; + @Cleanup + PulsarClient client = newPulsarClient(lookupUrl.toString(), 100); + Producer producer = pulsarClient.newProducer(Schema.STRING) + .enableBatching(false).topic(topicName).create(); + + List expected = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + producer.newMessage().key(String.valueOf(i)).value(String.valueOf(i)).send(); + expected.add(String.valueOf(i)); + } + producer.flush(); + + admin.topics().triggerCompaction(topicName); + + Awaitility.await().untilAsserted(() -> { + assertEquals(admin.topics().compactionStatus(topicName).status, + LongRunningProcessStatus.Status.SUCCESS); + }); + + // trim the topic + admin.topics().unload(topicName); + + Awaitility.await().untilAsserted(() -> { + PersistentTopicInternalStats internalStats = admin.topics().getInternalStats(topicName, false); + assertEquals(internalStats.numberOfEntries, 0); + }); + + ConsumerImpl consumer = (ConsumerImpl) client.newConsumer(Schema.STRING) + .topic(topicName).readCompacted(true).receiverQueueSize(1).subscriptionName(subName) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .isAckReceiptEnabled(true) + .subscribe(); + + List results = new ArrayList<>(); + for (int i = 0; i < 5; i++) { + Message message = consumer.receive(3, TimeUnit.SECONDS); + if (message == null) { + break; + } + results.add(message.getValue()); + consumer.acknowledge(message); + } + + Awaitility.await().untilAsserted(() -> + assertEquals(admin.topics().getStats(topicName, true).getSubscriptions().get(subName).getMsgBacklog(), + 5)); + + // Make consumer reconnect to broker + admin.topics().unload(topicName); + + // Wait for consumer to reconnect and clear incomingMessages + consumer.pause(); + Awaitility.await().untilAsserted(() -> { + Assert.assertEquals(consumer.numMessagesInQueue(), 0); + }); + consumer.resume(); + + for (int i = 0; i < 5; i++) { + Message message = consumer.receive(3, TimeUnit.SECONDS); + if (message == null) { + break; + } + results.add(message.getValue()); + consumer.acknowledge(message); + } + + Awaitility.await().untilAsserted(() -> + assertEquals(admin.topics().getStats(topicName, true).getSubscriptions().get(subName).getMsgBacklog(), + 0)); + + Assert.assertEquals(results, expected); + + Message message = consumer.receive(3, TimeUnit.SECONDS); + Assert.assertNull(message); + + // Make consumer reconnect to broker + admin.topics().unload(topicName); + + producer.newMessage().key("K").value("V").send(); + Message message2 = consumer.receive(3, TimeUnit.SECONDS); + Assert.assertEquals(message2.getValue(), "V"); + consumer.acknowledge(message2); + + Awaitility.await().untilAsserted(() -> { + PersistentTopicInternalStats internalStats = admin.topics().getInternalStats(topicName); + Assert.assertEquals(internalStats.lastConfirmedEntry, + internalStats.cursors.get(subName).markDeletePosition); + }); + + consumer.close(); + producer.close(); + } + + @Test + public void testEarliestSubsAfterRollover() throws Exception { + final String topicName = "persistent://my-property/use/my-ns/testEarliestSubsAfterRollover" + UUID.randomUUID(); + final String subName = "my-sub"; + @Cleanup + PulsarClient client = newPulsarClient(lookupUrl.toString(), 100); + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .enableBatching(false).topic(topicName).create(); + + List expected = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + producer.newMessage().key(String.valueOf(i)).value(String.valueOf(i)).send(); + expected.add(String.valueOf(i)); + } + producer.flush(); + + admin.topics().triggerCompaction(topicName); + + Awaitility.await().untilAsserted(() -> { + assertEquals(admin.topics().compactionStatus(topicName).status, + LongRunningProcessStatus.Status.SUCCESS); + }); + + // trim the topic + admin.topics().unload(topicName); + + Awaitility.await().untilAsserted(() -> { + PersistentTopicInternalStats internalStats = admin.topics().getInternalStats(topicName, false); + assertEquals(internalStats.numberOfEntries, 0); + }); + + // Make ml.getFirstPosition() return new ledger first position + producer.newMessage().key("K").value("V").send(); + expected.add("V"); + + @Cleanup + ConsumerImpl consumer = (ConsumerImpl) client.newConsumer(Schema.STRING) + .topic(topicName).readCompacted(true).receiverQueueSize(1).subscriptionName(subName) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .isAckReceiptEnabled(true) + .subscribe(); + + List results = new ArrayList<>(); + while (true) { + Message message = consumer.receive(3, TimeUnit.SECONDS); + if (message == null) { + break; + } + + results.add(message.getValue()); + consumer.acknowledge(message); + } + + Assert.assertEquals(results, expected); + } } From 9ae40c6530afcf2ff4fc275d03abce7ae21020bf Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Sat, 20 Jan 2024 09:35:56 +0800 Subject: [PATCH 261/699] [fix][broker] Fix issue with GetMessageIdByTimestamp can't find match messageId from compacted ledger (#21600) (cherry picked from commit b5e4c36848016310ad3ec56b6aee6488b540a027) --- .../admin/impl/PersistentTopicsBase.java | 60 +++++++++++++---- .../pulsar/compaction/CompactedTopicImpl.java | 50 ++++++++++++++ .../broker/admin/PersistentTopicsTest.java | 67 +++++++++++++++++++ 3 files changed, 164 insertions(+), 13 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 758b8048f3d13..ec22dc140983e 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 @@ -2882,28 +2882,62 @@ protected CompletableFuture internalGetMessageIdByTimestampAsync(long throw new RestException(Status.METHOD_NOT_ALLOWED, "Get message ID by timestamp on a non-persistent topic is not allowed"); } - ManagedLedger ledger = ((PersistentTopic) topic).getManagedLedger(); - return ledger.asyncFindPosition(entry -> { + final PersistentTopic persistentTopic = (PersistentTopic) topic; + + return persistentTopic.getTopicCompactionService().readLastCompactedEntry().thenCompose(lastEntry -> { + if (lastEntry == null) { + return findMessageIdByPublishTime(timestamp, persistentTopic.getManagedLedger()); + } + MessageMetadata metadata; + Position position = lastEntry.getPosition(); try { - long entryTimestamp = Commands.getEntryTimestamp(entry.getDataBuffer()); - return MessageImpl.isEntryPublishedEarlierThan(entryTimestamp, timestamp); - } catch (Exception e) { - log.error("[{}] Error deserializing message for message position find", topicName, e); + metadata = Commands.parseMessageMetadata(lastEntry.getDataBuffer()); } finally { - entry.release(); + lastEntry.release(); } - return false; - }).thenApply(position -> { - if (position == null) { - return null; + if (timestamp == metadata.getPublishTime()) { + return CompletableFuture.completedFuture(new MessageIdImpl(position.getLedgerId(), + position.getEntryId(), topicName.getPartitionIndex())); + } else if (timestamp < metadata.getPublishTime()) { + return persistentTopic.getTopicCompactionService().findEntryByPublishTime(timestamp) + .thenApply(compactedEntry -> { + try { + return new MessageIdImpl(compactedEntry.getLedgerId(), + compactedEntry.getEntryId(), topicName.getPartitionIndex()); + } finally { + compactedEntry.release(); + } + }); } else { - return new MessageIdImpl(position.getLedgerId(), position.getEntryId(), - topicName.getPartitionIndex()); + return findMessageIdByPublishTime(timestamp, persistentTopic.getManagedLedger()); } }); }); } + private CompletableFuture findMessageIdByPublishTime(long timestamp, ManagedLedger managedLedger) { + return managedLedger.asyncFindPosition(entry -> { + try { + long entryTimestamp = Commands.getEntryTimestamp(entry.getDataBuffer()); + return MessageImpl.isEntryPublishedEarlierThan(entryTimestamp, timestamp); + } catch (Exception e) { + log.error("[{}] Error deserializing message for message position find", + topicName, + e); + } finally { + entry.release(); + } + return false; + }).thenApply(position -> { + if (position == null) { + return null; + } else { + return new MessageIdImpl(position.getLedgerId(), position.getEntryId(), + topicName.getPartitionIndex()); + } + }); + } + protected CompletableFuture internalPeekNthMessageAsync(String subName, int messagePosition, boolean authoritative) { CompletableFuture ret; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java index ab54e96b71f1b..dfafbc41cb45c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java @@ -32,6 +32,7 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.function.Predicate; import javax.annotation.Nullable; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; @@ -324,6 +325,55 @@ public CompletableFuture readLastEntryOfCompactedLedger() { }); } + CompletableFuture findFirstMatchEntry(final Predicate predicate) { + var compactedTopicContextFuture = this.getCompactedTopicContextFuture(); + + if (compactedTopicContextFuture == null) { + return CompletableFuture.completedFuture(null); + } + return compactedTopicContextFuture.thenCompose(compactedTopicContext -> { + LedgerHandle lh = compactedTopicContext.getLedger(); + CompletableFuture promise = new CompletableFuture<>(); + findFirstMatchIndexLoop(predicate, 0L, lh.getLastAddConfirmed(), promise, null, lh); + return promise.thenCompose(index -> { + if (index == null) { + return CompletableFuture.completedFuture(null); + } + return readEntries(lh, index, index).thenApply(entries -> entries.get(0)); + }); + }); + } + private static void findFirstMatchIndexLoop(final Predicate predicate, + final long start, final long end, + final CompletableFuture promise, + final Long lastMatchIndex, + final LedgerHandle lh) { + if (start > end) { + promise.complete(lastMatchIndex); + return; + } + + long mid = (start + end) / 2; + readEntries(lh, mid, mid).thenAccept(entries -> { + Entry entry = entries.get(0); + final boolean isMatch; + try { + isMatch = predicate.test(entry); + } finally { + entry.release(); + } + + if (isMatch) { + findFirstMatchIndexLoop(predicate, start, mid - 1, promise, mid, lh); + } else { + findFirstMatchIndexLoop(predicate, mid + 1, end, promise, lastMatchIndex, lh); + } + }).exceptionally(ex -> { + promise.completeExceptionally(ex); + return null; + }); + } + private static int comparePositionAndMessageId(PositionImpl p, MessageIdData m) { return ComparisonChain.start() .compare(p.getLedgerId(), m.getLedgerId()) 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 10321cb776101..2ac1816a6723d 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 @@ -31,6 +31,8 @@ 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.assertSame; import static org.testng.Assert.assertTrue; import java.lang.reflect.Field; import java.util.ArrayList; @@ -65,6 +67,7 @@ import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.web.PulsarWebResource; import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.Topics; @@ -87,6 +90,7 @@ import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -1444,6 +1448,69 @@ public void onSendAcknowledgement(Producer producer, Message message, MessageId .compareTo(id2) > 0); } + @Test + public void testGetMessageIdByTimestampWithCompaction() throws Exception { + TenantInfoImpl tenantInfo = new TenantInfoImpl(Set.of("role1", "role2"), Set.of("test")); + admin.tenants().createTenant("tenant-xyz", tenantInfo); + admin.namespaces().createNamespace("tenant-xyz/ns-abc", Set.of("test")); + final String topicName = "persistent://tenant-xyz/ns-abc/testGetMessageIdByTimestampWithCompaction"; + admin.topics().createNonPartitionedTopic(topicName); + + Map publishTimeMap = new ConcurrentHashMap<>(); + @Cleanup + ProducerBase producer = (ProducerBase) pulsarClient.newProducer().topic(topicName) + .enableBatching(false) + .intercept(new ProducerInterceptor() { + @Override + public void close() { + + } + + @Override + public boolean eligible(Message message) { + return true; + } + + @Override + public Message beforeSend(Producer producer, Message message) { + return message; + } + + @Override + public void onSendAcknowledgement(Producer producer, Message message, MessageId msgId, + Throwable exception) { + publishTimeMap.put(message.getMessageId(), message.getPublishTime()); + } + }) + .create(); + + MessageId id1 = producer.newMessage().key("K1").value("test1".getBytes()).send(); + MessageId id2 = producer.newMessage().key("K2").value("test2".getBytes()).send(); + + long publish1 = publishTimeMap.get(id1); + long publish2 = publishTimeMap.get(id2); + Assert.assertTrue(publish1 < publish2); + + admin.topics().triggerCompaction(topicName); + Awaitility.await().untilAsserted(() -> + assertSame(admin.topics().compactionStatus(topicName).status, + LongRunningProcessStatus.Status.SUCCESS)); + + admin.topics().unload(topicName); + Awaitility.await().untilAsserted(() -> { + PersistentTopicInternalStats internalStats = admin.topics().getInternalStats(topicName, false); + assertEquals(internalStats.ledgers.size(), 1); + assertEquals(internalStats.ledgers.get(0).entries, 0); + }); + + Assert.assertEquals(admin.topics().getMessageIdByTimestamp(topicName, publish1 - 1), id1); + Assert.assertEquals(admin.topics().getMessageIdByTimestamp(topicName, publish1), id1); + Assert.assertEquals(admin.topics().getMessageIdByTimestamp(topicName, publish1 + 1), id2); + Assert.assertEquals(admin.topics().getMessageIdByTimestamp(topicName, publish2), id2); + Assert.assertTrue(admin.topics().getMessageIdByTimestamp(topicName, publish2 + 1) + .compareTo(id2) > 0); + } + @Test public void testGetBatchMessageIdByTimestamp() throws Exception { TenantInfoImpl tenantInfo = new TenantInfoImpl(Set.of("role1", "role2"), Set.of("test")); From 616b97c68536290cec05f88aa830df54185481bb Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Tue, 16 Jan 2024 18:18:31 +0800 Subject: [PATCH 262/699] [improve][ci] Upgrade pulsar-client-python to 3.4.0 to avoid CVE-2023-1428 (#21899) (cherry picked from commit 65ce9d6ea3babf66880142c602cdb7fe59f61725) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 563cf7634fd68..030b65af39204 100644 --- a/pom.xml +++ b/pom.xml @@ -82,7 +82,7 @@ flexible messaging model and an intuitive client API. ${maven.compiler.target} 8 - 3.2.0 + 3.4.0 **/Test*.java,**/*Test.java,**/*Tests.java,**/*TestCase.java From 48d2f37bbb075bec2d731c641c19079a1fe43879 Mon Sep 17 00:00:00 2001 From: coderzc Date: Thu, 1 Feb 2024 01:08:09 +0800 Subject: [PATCH 263/699] Revert "[fix][broker] Fix issue with GetMessageIdByTimestamp can't find match messageId from compacted ledger (#21600)" This reverts commit b5e4c36848016310ad3ec56b6aee6488b540a027. (cherry picked from commit e53c19a003c3d91a65e0518a2b9e083c21f62eb7) --- .../admin/impl/PersistentTopicsBase.java | 60 ++++------------- .../pulsar/compaction/CompactedTopicImpl.java | 50 -------------- .../broker/admin/PersistentTopicsTest.java | 67 ------------------- 3 files changed, 13 insertions(+), 164 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 ec22dc140983e..758b8048f3d13 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 @@ -2882,62 +2882,28 @@ protected CompletableFuture internalGetMessageIdByTimestampAsync(long throw new RestException(Status.METHOD_NOT_ALLOWED, "Get message ID by timestamp on a non-persistent topic is not allowed"); } - final PersistentTopic persistentTopic = (PersistentTopic) topic; - - return persistentTopic.getTopicCompactionService().readLastCompactedEntry().thenCompose(lastEntry -> { - if (lastEntry == null) { - return findMessageIdByPublishTime(timestamp, persistentTopic.getManagedLedger()); - } - MessageMetadata metadata; - Position position = lastEntry.getPosition(); + ManagedLedger ledger = ((PersistentTopic) topic).getManagedLedger(); + return ledger.asyncFindPosition(entry -> { try { - metadata = Commands.parseMessageMetadata(lastEntry.getDataBuffer()); + long entryTimestamp = Commands.getEntryTimestamp(entry.getDataBuffer()); + return MessageImpl.isEntryPublishedEarlierThan(entryTimestamp, timestamp); + } catch (Exception e) { + log.error("[{}] Error deserializing message for message position find", topicName, e); } finally { - lastEntry.release(); + entry.release(); } - if (timestamp == metadata.getPublishTime()) { - return CompletableFuture.completedFuture(new MessageIdImpl(position.getLedgerId(), - position.getEntryId(), topicName.getPartitionIndex())); - } else if (timestamp < metadata.getPublishTime()) { - return persistentTopic.getTopicCompactionService().findEntryByPublishTime(timestamp) - .thenApply(compactedEntry -> { - try { - return new MessageIdImpl(compactedEntry.getLedgerId(), - compactedEntry.getEntryId(), topicName.getPartitionIndex()); - } finally { - compactedEntry.release(); - } - }); + return false; + }).thenApply(position -> { + if (position == null) { + return null; } else { - return findMessageIdByPublishTime(timestamp, persistentTopic.getManagedLedger()); + return new MessageIdImpl(position.getLedgerId(), position.getEntryId(), + topicName.getPartitionIndex()); } }); }); } - private CompletableFuture findMessageIdByPublishTime(long timestamp, ManagedLedger managedLedger) { - return managedLedger.asyncFindPosition(entry -> { - try { - long entryTimestamp = Commands.getEntryTimestamp(entry.getDataBuffer()); - return MessageImpl.isEntryPublishedEarlierThan(entryTimestamp, timestamp); - } catch (Exception e) { - log.error("[{}] Error deserializing message for message position find", - topicName, - e); - } finally { - entry.release(); - } - return false; - }).thenApply(position -> { - if (position == null) { - return null; - } else { - return new MessageIdImpl(position.getLedgerId(), position.getEntryId(), - topicName.getPartitionIndex()); - } - }); - } - protected CompletableFuture internalPeekNthMessageAsync(String subName, int messagePosition, boolean authoritative) { CompletableFuture ret; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java index dfafbc41cb45c..ab54e96b71f1b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java @@ -32,7 +32,6 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import java.util.function.Predicate; import javax.annotation.Nullable; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; @@ -325,55 +324,6 @@ public CompletableFuture readLastEntryOfCompactedLedger() { }); } - CompletableFuture findFirstMatchEntry(final Predicate predicate) { - var compactedTopicContextFuture = this.getCompactedTopicContextFuture(); - - if (compactedTopicContextFuture == null) { - return CompletableFuture.completedFuture(null); - } - return compactedTopicContextFuture.thenCompose(compactedTopicContext -> { - LedgerHandle lh = compactedTopicContext.getLedger(); - CompletableFuture promise = new CompletableFuture<>(); - findFirstMatchIndexLoop(predicate, 0L, lh.getLastAddConfirmed(), promise, null, lh); - return promise.thenCompose(index -> { - if (index == null) { - return CompletableFuture.completedFuture(null); - } - return readEntries(lh, index, index).thenApply(entries -> entries.get(0)); - }); - }); - } - private static void findFirstMatchIndexLoop(final Predicate predicate, - final long start, final long end, - final CompletableFuture promise, - final Long lastMatchIndex, - final LedgerHandle lh) { - if (start > end) { - promise.complete(lastMatchIndex); - return; - } - - long mid = (start + end) / 2; - readEntries(lh, mid, mid).thenAccept(entries -> { - Entry entry = entries.get(0); - final boolean isMatch; - try { - isMatch = predicate.test(entry); - } finally { - entry.release(); - } - - if (isMatch) { - findFirstMatchIndexLoop(predicate, start, mid - 1, promise, mid, lh); - } else { - findFirstMatchIndexLoop(predicate, mid + 1, end, promise, lastMatchIndex, lh); - } - }).exceptionally(ex -> { - promise.completeExceptionally(ex); - return null; - }); - } - private static int comparePositionAndMessageId(PositionImpl p, MessageIdData m) { return ComparisonChain.start() .compare(p.getLedgerId(), m.getLedgerId()) 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 2ac1816a6723d..10321cb776101 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 @@ -31,8 +31,6 @@ 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.assertSame; import static org.testng.Assert.assertTrue; import java.lang.reflect.Field; import java.util.ArrayList; @@ -67,7 +65,6 @@ import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.web.PulsarWebResource; import org.apache.pulsar.broker.web.RestException; -import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.Topics; @@ -90,7 +87,6 @@ import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.ClusterData; -import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -1448,69 +1444,6 @@ public void onSendAcknowledgement(Producer producer, Message message, MessageId .compareTo(id2) > 0); } - @Test - public void testGetMessageIdByTimestampWithCompaction() throws Exception { - TenantInfoImpl tenantInfo = new TenantInfoImpl(Set.of("role1", "role2"), Set.of("test")); - admin.tenants().createTenant("tenant-xyz", tenantInfo); - admin.namespaces().createNamespace("tenant-xyz/ns-abc", Set.of("test")); - final String topicName = "persistent://tenant-xyz/ns-abc/testGetMessageIdByTimestampWithCompaction"; - admin.topics().createNonPartitionedTopic(topicName); - - Map publishTimeMap = new ConcurrentHashMap<>(); - @Cleanup - ProducerBase producer = (ProducerBase) pulsarClient.newProducer().topic(topicName) - .enableBatching(false) - .intercept(new ProducerInterceptor() { - @Override - public void close() { - - } - - @Override - public boolean eligible(Message message) { - return true; - } - - @Override - public Message beforeSend(Producer producer, Message message) { - return message; - } - - @Override - public void onSendAcknowledgement(Producer producer, Message message, MessageId msgId, - Throwable exception) { - publishTimeMap.put(message.getMessageId(), message.getPublishTime()); - } - }) - .create(); - - MessageId id1 = producer.newMessage().key("K1").value("test1".getBytes()).send(); - MessageId id2 = producer.newMessage().key("K2").value("test2".getBytes()).send(); - - long publish1 = publishTimeMap.get(id1); - long publish2 = publishTimeMap.get(id2); - Assert.assertTrue(publish1 < publish2); - - admin.topics().triggerCompaction(topicName); - Awaitility.await().untilAsserted(() -> - assertSame(admin.topics().compactionStatus(topicName).status, - LongRunningProcessStatus.Status.SUCCESS)); - - admin.topics().unload(topicName); - Awaitility.await().untilAsserted(() -> { - PersistentTopicInternalStats internalStats = admin.topics().getInternalStats(topicName, false); - assertEquals(internalStats.ledgers.size(), 1); - assertEquals(internalStats.ledgers.get(0).entries, 0); - }); - - Assert.assertEquals(admin.topics().getMessageIdByTimestamp(topicName, publish1 - 1), id1); - Assert.assertEquals(admin.topics().getMessageIdByTimestamp(topicName, publish1), id1); - Assert.assertEquals(admin.topics().getMessageIdByTimestamp(topicName, publish1 + 1), id2); - Assert.assertEquals(admin.topics().getMessageIdByTimestamp(topicName, publish2), id2); - Assert.assertTrue(admin.topics().getMessageIdByTimestamp(topicName, publish2 + 1) - .compareTo(id2) > 0); - } - @Test public void testGetBatchMessageIdByTimestamp() throws Exception { TenantInfoImpl tenantInfo = new TenantInfoImpl(Set.of("role1", "role2"), Set.of("test")); From 915f139c7bae717673188a350d1d1b71714e9727 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Sat, 20 Jan 2024 09:35:56 +0800 Subject: [PATCH 264/699] [fix][broker] Fix issue with GetMessageIdByTimestamp can't find match messageId from compacted ledger (#21600) (cherry picked from commit 5bb2264dd910125a427b61530f29e537c0ea0708) --- .../admin/impl/PersistentTopicsBase.java | 68 ++++++++++++++---- .../pulsar/compaction/CompactedTopicImpl.java | 50 ++++++++++++++ .../broker/admin/PersistentTopicsTest.java | 69 ++++++++++++++++++- 3 files changed, 173 insertions(+), 14 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 758b8048f3d13..7892861d5df1e 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 @@ -46,6 +46,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; +import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.IntStream; import javax.annotation.Nonnull; @@ -148,6 +149,7 @@ import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.collections.BitSetRecyclable; import org.apache.pulsar.compaction.Compactor; +import org.apache.pulsar.compaction.PulsarTopicCompactionService; import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.transaction.coordinator.TransactionCoordinatorID; import org.slf4j.Logger; @@ -2882,28 +2884,68 @@ protected CompletableFuture internalGetMessageIdByTimestampAsync(long throw new RestException(Status.METHOD_NOT_ALLOWED, "Get message ID by timestamp on a non-persistent topic is not allowed"); } - ManagedLedger ledger = ((PersistentTopic) topic).getManagedLedger(); - return ledger.asyncFindPosition(entry -> { + final PersistentTopic persistentTopic = (PersistentTopic) topic; + + return persistentTopic.getTopicCompactionService().readLastCompactedEntry().thenCompose(lastEntry -> { + if (lastEntry == null) { + return findMessageIdByPublishTime(timestamp, persistentTopic.getManagedLedger()); + } + MessageMetadata metadata; + Position position = lastEntry.getPosition(); try { - long entryTimestamp = Commands.getEntryTimestamp(entry.getDataBuffer()); - return MessageImpl.isEntryPublishedEarlierThan(entryTimestamp, timestamp); - } catch (Exception e) { - log.error("[{}] Error deserializing message for message position find", topicName, e); + metadata = Commands.parseMessageMetadata(lastEntry.getDataBuffer()); } finally { - entry.release(); + lastEntry.release(); } - return false; - }).thenApply(position -> { - if (position == null) { - return null; + if (timestamp == metadata.getPublishTime()) { + return CompletableFuture.completedFuture(new MessageIdImpl(position.getLedgerId(), + position.getEntryId(), topicName.getPartitionIndex())); + } else if (timestamp < metadata.getPublishTime() + && persistentTopic.getTopicCompactionService() instanceof PulsarTopicCompactionService) { + var compactedTopic = + ((PulsarTopicCompactionService) persistentTopic.getTopicCompactionService()) + .getCompactedTopic(); + final Predicate predicate = entry -> + Commands.parseMessageMetadata(entry.getDataBuffer()).getPublishTime() >= timestamp; + return compactedTopic.findFirstMatchEntry(predicate) + .thenApply(compactedEntry -> { + try { + return new MessageIdImpl(compactedEntry.getLedgerId(), + compactedEntry.getEntryId(), topicName.getPartitionIndex()); + } finally { + compactedEntry.release(); + } + }); } else { - return new MessageIdImpl(position.getLedgerId(), position.getEntryId(), - topicName.getPartitionIndex()); + return findMessageIdByPublishTime(timestamp, persistentTopic.getManagedLedger()); } }); }); } + private CompletableFuture findMessageIdByPublishTime(long timestamp, ManagedLedger managedLedger) { + return managedLedger.asyncFindPosition(entry -> { + try { + long entryTimestamp = Commands.getEntryTimestamp(entry.getDataBuffer()); + return MessageImpl.isEntryPublishedEarlierThan(entryTimestamp, timestamp); + } catch (Exception e) { + log.error("[{}] Error deserializing message for message position find", + topicName, + e); + } finally { + entry.release(); + } + return false; + }).thenApply(position -> { + if (position == null) { + return null; + } else { + return new MessageIdImpl(position.getLedgerId(), position.getEntryId(), + topicName.getPartitionIndex()); + } + }); + } + protected CompletableFuture internalPeekNthMessageAsync(String subName, int messagePosition, boolean authoritative) { CompletableFuture ret; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java index ab54e96b71f1b..0352808110190 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java @@ -32,6 +32,7 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.function.Predicate; import javax.annotation.Nullable; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.BookKeeper; @@ -324,6 +325,55 @@ public CompletableFuture readLastEntryOfCompactedLedger() { }); } + public CompletableFuture findFirstMatchEntry(final Predicate predicate) { + var compactedTopicContextFuture = this.getCompactedTopicContextFuture(); + + if (compactedTopicContextFuture == null) { + return CompletableFuture.completedFuture(null); + } + return compactedTopicContextFuture.thenCompose(compactedTopicContext -> { + LedgerHandle lh = compactedTopicContext.getLedger(); + CompletableFuture promise = new CompletableFuture<>(); + findFirstMatchIndexLoop(predicate, 0L, lh.getLastAddConfirmed(), promise, null, lh); + return promise.thenCompose(index -> { + if (index == null) { + return CompletableFuture.completedFuture(null); + } + return readEntries(lh, index, index).thenApply(entries -> entries.get(0)); + }); + }); + } + private static void findFirstMatchIndexLoop(final Predicate predicate, + final long start, final long end, + final CompletableFuture promise, + final Long lastMatchIndex, + final LedgerHandle lh) { + if (start > end) { + promise.complete(lastMatchIndex); + return; + } + + long mid = (start + end) / 2; + readEntries(lh, mid, mid).thenAccept(entries -> { + Entry entry = entries.get(0); + final boolean isMatch; + try { + isMatch = predicate.test(entry); + } finally { + entry.release(); + } + + if (isMatch) { + findFirstMatchIndexLoop(predicate, start, mid - 1, promise, mid, lh); + } else { + findFirstMatchIndexLoop(predicate, mid + 1, end, promise, lastMatchIndex, lh); + } + }).exceptionally(ex -> { + promise.completeExceptionally(ex); + return null; + }); + } + private static int comparePositionAndMessageId(PositionImpl p, MessageIdData m) { return ComparisonChain.start() .compare(p.getLedgerId(), m.getLedgerId()) 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 10321cb776101..1f755234009ba 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 @@ -31,6 +31,8 @@ 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.assertSame; import static org.testng.Assert.assertTrue; import java.lang.reflect.Field; import java.util.ArrayList; @@ -52,9 +54,9 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.admin.v2.ExtPersistentTopics; import org.apache.pulsar.broker.admin.v2.NonPersistentTopics; import org.apache.pulsar.broker.admin.v2.PersistentTopics; -import org.apache.pulsar.broker.admin.v2.ExtPersistentTopics; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationDataHttps; import org.apache.pulsar.broker.namespace.NamespaceService; @@ -65,6 +67,7 @@ import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.web.PulsarWebResource; import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.Topics; @@ -87,6 +90,7 @@ import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AuthAction; import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -1444,6 +1448,69 @@ public void onSendAcknowledgement(Producer producer, Message message, MessageId .compareTo(id2) > 0); } + @Test + public void testGetMessageIdByTimestampWithCompaction() throws Exception { + TenantInfoImpl tenantInfo = new TenantInfoImpl(Set.of("role1", "role2"), Set.of("test")); + admin.tenants().createTenant("tenant-xyz", tenantInfo); + admin.namespaces().createNamespace("tenant-xyz/ns-abc", Set.of("test")); + final String topicName = "persistent://tenant-xyz/ns-abc/testGetMessageIdByTimestampWithCompaction"; + admin.topics().createNonPartitionedTopic(topicName); + + Map publishTimeMap = new ConcurrentHashMap<>(); + @Cleanup + ProducerBase producer = (ProducerBase) pulsarClient.newProducer().topic(topicName) + .enableBatching(false) + .intercept(new ProducerInterceptor() { + @Override + public void close() { + + } + + @Override + public boolean eligible(Message message) { + return true; + } + + @Override + public Message beforeSend(Producer producer, Message message) { + return message; + } + + @Override + public void onSendAcknowledgement(Producer producer, Message message, MessageId msgId, + Throwable exception) { + publishTimeMap.put(message.getMessageId(), message.getPublishTime()); + } + }) + .create(); + + MessageId id1 = producer.newMessage().key("K1").value("test1".getBytes()).send(); + MessageId id2 = producer.newMessage().key("K2").value("test2".getBytes()).send(); + + long publish1 = publishTimeMap.get(id1); + long publish2 = publishTimeMap.get(id2); + Assert.assertTrue(publish1 < publish2); + + admin.topics().triggerCompaction(topicName); + Awaitility.await().untilAsserted(() -> + assertSame(admin.topics().compactionStatus(topicName).status, + LongRunningProcessStatus.Status.SUCCESS)); + + admin.topics().unload(topicName); + Awaitility.await().untilAsserted(() -> { + PersistentTopicInternalStats internalStats = admin.topics().getInternalStats(topicName, false); + assertEquals(internalStats.ledgers.size(), 1); + assertEquals(internalStats.ledgers.get(0).entries, 0); + }); + + Assert.assertEquals(admin.topics().getMessageIdByTimestamp(topicName, publish1 - 1), id1); + Assert.assertEquals(admin.topics().getMessageIdByTimestamp(topicName, publish1), id1); + Assert.assertEquals(admin.topics().getMessageIdByTimestamp(topicName, publish1 + 1), id2); + Assert.assertEquals(admin.topics().getMessageIdByTimestamp(topicName, publish2), id2); + Assert.assertTrue(admin.topics().getMessageIdByTimestamp(topicName, publish2 + 1) + .compareTo(id2) > 0); + } + @Test public void testGetBatchMessageIdByTimestamp() throws Exception { TenantInfoImpl tenantInfo = new TenantInfoImpl(Set.of("role1", "role2"), Set.of("test")); From 43f02172e7fe28e757a048a322b5f6cd7803c143 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Fri, 19 Jan 2024 14:38:14 +0800 Subject: [PATCH 265/699] [improve][ml] Filter out deleted entries before read entries from ledger. (#21739) (cherry picked from commit c66167be55e9ed14261174a672952136c6fdb441) (cherry picked from commit 84ed73e821ea0a7d81f238a4cf0cb459a857c7ce) --- .../mledger/impl/ManagedCursorImpl.java | 4 + .../mledger/impl/ManagedLedgerImpl.java | 2 +- .../mledger/impl/ReadOnlyCursorImpl.java | 6 + .../mledger/impl/ManagedCursorTest.java | 232 ++++++++++++++++-- 4 files changed, 227 insertions(+), 17 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 7a58bf7dd3698..ffa92c4f92162 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 @@ -787,6 +787,8 @@ public void asyncReadEntriesWithSkip(int numberOfEntriesToRead, long maxSizeByte int numOfEntriesToRead = applyMaxSizeCap(numberOfEntriesToRead, maxSizeBytes); PENDING_READ_OPS_UPDATER.incrementAndGet(this); + // Skip deleted entries. + skipCondition = skipCondition == null ? this::isMessageDeleted : skipCondition.or(this::isMessageDeleted); OpReadEntry op = OpReadEntry.create(this, readPosition, numOfEntriesToRead, callback, ctx, maxPosition, skipCondition); ledger.asyncReadEntries(op); @@ -940,6 +942,8 @@ public void asyncReadEntriesWithSkipOrWait(int maxEntries, long maxSizeBytes, Re asyncReadEntriesWithSkip(numberOfEntriesToRead, NO_MAX_SIZE_LIMIT, callback, ctx, maxPosition, skipCondition); } else { + // Skip deleted entries. + skipCondition = skipCondition == null ? this::isMessageDeleted : skipCondition.or(this::isMessageDeleted); OpReadEntry op = OpReadEntry.create(this, readPosition, numberOfEntriesToRead, callback, ctx, maxPosition, skipCondition); 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 948d1b5ef9e45..e2f0739fae1ee 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 @@ -4530,4 +4530,4 @@ public Position getTheSlowestNonDurationReadPosition() { } return theSlowestNonDurableReadPosition; } -} +} \ No newline at end of file diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java index 9102339b2904e..1661613f07d7d 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java @@ -23,6 +23,7 @@ import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; +import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.ReadOnlyCursor; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.PositionBound; import org.apache.bookkeeper.mledger.proto.MLDataFormats; @@ -70,4 +71,9 @@ public MLDataFormats.ManagedLedgerInfo.LedgerInfo getCurrentLedgerInfo() { public long getNumberOfEntries(Range range) { return this.ledger.getNumberOfEntries(range); } + + @Override + public boolean isMessageDeleted(Position position) { + return false; + } } 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 627ae73d928bd..644f53c3a522d 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 @@ -43,6 +43,7 @@ import java.util.Arrays; import java.util.BitSet; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -65,6 +66,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Predicate; import java.util.function.Supplier; import java.util.stream.Collectors; import lombok.Cleanup; @@ -72,6 +74,7 @@ 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.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCallback; @@ -766,7 +769,7 @@ void testResetCursor() throws Exception { @Test(timeOut = 20000) void testResetCursor1() throws Exception { ManagedLedger ledger = factory.open("my_test_move_cursor_ledger", - new ManagedLedgerConfig().setMaxEntriesPerLedger(2)); + new ManagedLedgerConfig().setMaxEntriesPerLedger(2)); ManagedCursor cursor = ledger.openCursor("trc1"); PositionImpl actualEarliest = (PositionImpl) ledger.addEntry("dummy-entry-1".getBytes(Encoding)); ledger.addEntry("dummy-entry-2".getBytes(Encoding)); @@ -2286,7 +2289,7 @@ void testFindNewestMatchingEdgeCase1() throws Exception { ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor("c1"); assertNull(c1.findNewestMatching( - entry -> Arrays.equals(entry.getDataAndRelease(), "expired".getBytes(Encoding)))); + entry -> Arrays.equals(entry.getDataAndRelease(), "expired".getBytes(Encoding)))); } @Test(timeOut = 20000) @@ -2595,7 +2598,7 @@ public void findEntryComplete(Position position, Object ctx) { @Override public void findEntryFailed(ManagedLedgerException exception, Optional failedReadPosition, - Object ctx) { + Object ctx) { result.exception = exception; counter.countDown(); } @@ -2621,7 +2624,7 @@ public void findEntryFailed(ManagedLedgerException exception, Optional } void internalTestFindNewestMatchingAllEntries(final String name, final int entriesPerLedger, - final int expectedEntryId) throws Exception { + final int expectedEntryId) throws Exception { final String ledgerAndCursorName = name; ManagedLedgerConfig config = new ManagedLedgerConfig(); config.setRetentionSizeInMB(10); @@ -2715,7 +2718,7 @@ void testReplayEntries() throws Exception { assertTrue((Arrays.equals(entries.get(0).getData(), "entry1".getBytes(Encoding)) && Arrays.equals(entries.get(1).getData(), "entry3".getBytes(Encoding))) || (Arrays.equals(entries.get(0).getData(), "entry3".getBytes(Encoding)) - && Arrays.equals(entries.get(1).getData(), "entry1".getBytes(Encoding)))); + && Arrays.equals(entries.get(1).getData(), "entry1".getBytes(Encoding)))); entries.forEach(Entry::release); // 3. Fail on reading non-existing position @@ -3142,7 +3145,7 @@ public void operationFailed(ManagedLedgerException exception) { try { bkc.openLedgerNoRecovery(ledgerId, DigestType.fromApiDigestType(mlConfig.getDigestType()), - mlConfig.getPassword()); + mlConfig.getPassword()); fail("ledger should have deleted due to update-cursor failure"); } catch (BKException e) { // ok @@ -3761,17 +3764,17 @@ private void deleteBatchIndex(ManagedCursor cursor, Position position, int batch pos.ackSet = bitSet.toLongArray(); cursor.asyncDelete(pos, - new DeleteCallback() { - @Override - public void deleteComplete(Object ctx) { - latch.countDown(); - } + new DeleteCallback() { + @Override + public void deleteComplete(Object ctx) { + latch.countDown(); + } - @Override - public void deleteFailed(ManagedLedgerException exception, Object ctx) { - latch.countDown(); - } - }, null); + @Override + public void deleteFailed(ManagedLedgerException exception, Object ctx) { + latch.countDown(); + } + }, null); latch.await(); pos.ackSet = null; } @@ -4484,5 +4487,202 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { ledger.close(); } + + @Test + public void testReadEntriesWithSkipDeletedEntries() throws Exception { + @Cleanup + ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("testReadEntriesWithSkipDeletedEntries"); + ledger = Mockito.spy(ledger); + List actualReadEntryIds = new ArrayList<>(); + Mockito.doAnswer(inv -> { + long start = inv.getArgument(1); + long end = inv.getArgument(2); + for (long i = start; i <= end; i++) { + actualReadEntryIds.add(i); + } + return inv.callRealMethod(); + }) + .when(ledger) + .asyncReadEntry(Mockito.any(ReadHandle.class), Mockito.anyLong(), Mockito.anyLong(), Mockito.any(), Mockito.any()); + @Cleanup + ManagedCursor cursor = ledger.openCursor("c"); + + int entries = 20; + Position maxReadPosition = null; + Map map = new HashMap<>(); + for (int i = 0; i < entries; i++) { + maxReadPosition = ledger.addEntry(new byte[1024]); + map.put(i, maxReadPosition); + } + + + Set deletedPositions = new HashSet<>(); + deletedPositions.add(map.get(1)); + deletedPositions.add(map.get(4)); + deletedPositions.add(map.get(5)); + deletedPositions.add(map.get(8)); + deletedPositions.add(map.get(9)); + deletedPositions.add(map.get(10)); + deletedPositions.add(map.get(15)); + deletedPositions.add(map.get(17)); + deletedPositions.add(map.get(19)); + cursor.delete(deletedPositions); + + CompletableFuture f0 = new CompletableFuture<>(); + List readEntries = new ArrayList<>(); + cursor.asyncReadEntries(5, -1L, new ReadEntriesCallback() { + @Override + public void readEntriesComplete(List entries, Object ctx) { + readEntries.addAll(entries); + f0.complete(null); + } + + @Override + public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { + f0.completeExceptionally(exception); + } + }, null, PositionImpl.get(maxReadPosition.getLedgerId(), maxReadPosition.getEntryId()).getNext()); + + f0.get(); + + CompletableFuture f1 = new CompletableFuture<>(); + cursor.asyncReadEntries(5, -1L, new ReadEntriesCallback() { + @Override + public void readEntriesComplete(List entries, Object ctx) { + readEntries.addAll(entries); + f1.complete(null); + } + + @Override + public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { + f1.completeExceptionally(exception); + } + }, null, PositionImpl.get(maxReadPosition.getLedgerId(), maxReadPosition.getEntryId()).getNext()); + + + f1.get(); + CompletableFuture f2 = new CompletableFuture<>(); + cursor.asyncReadEntries(100, -1L, new ReadEntriesCallback() { + @Override + public void readEntriesComplete(List entries, Object ctx) { + readEntries.addAll(entries); + f2.complete(null); + } + + @Override + public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { + f2.completeExceptionally(exception); + } + }, null, PositionImpl.get(maxReadPosition.getLedgerId(), maxReadPosition.getEntryId()).getNext()); + + f2.get(); + + Position cursorReadPosition = cursor.getReadPosition(); + Position expectReadPosition = maxReadPosition.getNext(); + assertTrue(cursorReadPosition.getLedgerId() == expectReadPosition.getLedgerId() + && cursorReadPosition.getEntryId() == expectReadPosition.getEntryId()); + + assertEquals(readEntries.size(), actualReadEntryIds.size()); + assertEquals(entries - deletedPositions.size(), actualReadEntryIds.size()); + for (Entry entry : readEntries) { + long entryId = entry.getEntryId(); + assertTrue(actualReadEntryIds.contains(entryId)); + } + } + + + @Test + public void testReadEntriesWithSkipDeletedEntriesAndWithSkipConditions() throws Exception { + @Cleanup + ManagedLedgerImpl ledger = (ManagedLedgerImpl) + factory.open("testReadEntriesWithSkipDeletedEntriesAndWithSkipConditions"); + ledger = Mockito.spy(ledger); + + List actualReadEntryIds = new ArrayList<>(); + Mockito.doAnswer(inv -> { + long start = inv.getArgument(1); + long end = inv.getArgument(2); + for (long i = start; i <= end; i++) { + actualReadEntryIds.add(i); + } + return inv.callRealMethod(); + }) + .when(ledger) + .asyncReadEntry(Mockito.any(ReadHandle.class), Mockito.anyLong(), Mockito.anyLong(), Mockito.any(), Mockito.any()); + @Cleanup + ManagedCursor cursor = ledger.openCursor("c"); + + int entries = 20; + Position maxReadPosition0 = null; + Map map = new HashMap<>(); + for (int i = 0; i < entries; i++) { + maxReadPosition0 = ledger.addEntry(new byte[1024]); + map.put(i, maxReadPosition0); + } + + PositionImpl maxReadPosition = + PositionImpl.get(maxReadPosition0.getLedgerId(), maxReadPosition0.getEntryId()).getNext(); + + Set deletedPositions = new HashSet<>(); + deletedPositions.add(map.get(1)); + deletedPositions.add(map.get(3)); + deletedPositions.add(map.get(5)); + cursor.delete(deletedPositions); + + Set skippedPositions = new HashSet<>(); + skippedPositions.add(map.get(6).getEntryId()); + skippedPositions.add(map.get(7).getEntryId()); + skippedPositions.add(map.get(8).getEntryId()); + skippedPositions.add(map.get(11).getEntryId()); + skippedPositions.add(map.get(15).getEntryId()); + skippedPositions.add(map.get(16).getEntryId()); + + Predicate skipCondition = position -> skippedPositions.contains(position.getEntryId()); + List readEntries = new ArrayList<>(); + + CompletableFuture f0 = new CompletableFuture<>(); + cursor.asyncReadEntriesWithSkip(10, -1L, new ReadEntriesCallback() { + @Override + public void readEntriesComplete(List entries, Object ctx) { + readEntries.addAll(entries); + f0.complete(null); + } + + @Override + public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { + f0.completeExceptionally(exception); + } + }, null, maxReadPosition, skipCondition); + + f0.get(); + CompletableFuture f1 = new CompletableFuture<>(); + cursor.asyncReadEntriesWithSkip(100, -1L, new ReadEntriesCallback() { + @Override + public void readEntriesComplete(List entries, Object ctx) { + readEntries.addAll(entries); + f1.complete(null); + } + + @Override + public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { + f1.completeExceptionally(exception); + } + }, null, maxReadPosition, skipCondition); + f1.get(); + + + assertEquals(actualReadEntryIds.size(), readEntries.size()); + assertEquals(entries - deletedPositions.size() - skippedPositions.size(), actualReadEntryIds.size()); + for (Entry entry : readEntries) { + long entryId = entry.getEntryId(); + assertTrue(actualReadEntryIds.contains(entryId)); + } + + Position cursorReadPosition = cursor.getReadPosition(); + Position expectReadPosition = maxReadPosition; + assertTrue(cursorReadPosition.getLedgerId() == expectReadPosition.getLedgerId() + && cursorReadPosition.getEntryId() == expectReadPosition.getEntryId()); + } + private static final Logger log = LoggerFactory.getLogger(ManagedCursorTest.class); } From 4697fb03abfb2d577d1302b5459d600626cd65a8 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 8 Feb 2024 21:43:26 -0800 Subject: [PATCH 266/699] [fix][broker] Sanitize values before logging in apply-config-from-env.py script (#22044) (cherry picked from commit 303678364eab538c16041214cae1588a5b2111d9) (cherry picked from commit 9a8761e204f90a6763ea24a460059c11725494ac) --- .../apply-config-from-env-with-prefix.py | 85 ++----------------- .../pulsar/scripts/apply-config-from-env.py | 57 ++++++------- 2 files changed, 32 insertions(+), 110 deletions(-) diff --git a/docker/pulsar/scripts/apply-config-from-env-with-prefix.py b/docker/pulsar/scripts/apply-config-from-env-with-prefix.py index 58f6c98975005..9943b283a9f89 100755 --- a/docker/pulsar/scripts/apply-config-from-env-with-prefix.py +++ b/docker/pulsar/scripts/apply-config-from-env-with-prefix.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python3 +#!/usr/bin/env bash # # Licensed to the Apache Software Foundation (ASF) under one # or more contributor license agreements. See the NOTICE file @@ -32,83 +32,8 @@ # update if they exist and ignored if they don't. ############################################################ -import os -import sys - -if len(sys.argv) < 3: - print('Usage: %s [...]' % (sys.argv[0])) - sys.exit(1) - -# Always apply env config to env scripts as well -prefix = sys.argv[1] -conf_files = sys.argv[2:] - -PF_ENV_DEBUG = (os.environ.get('PF_ENV_DEBUG','0') == '1') - -for conf_filename in conf_files: - lines = [] # List of config file lines - keys = {} # Map a key to its line number in the file - - # Load conf file - for line in open(conf_filename): - lines.append(line) - line = line.strip() - if not line or line.startswith('#'): - continue - - try: - k,v = line.split('=', 1) - keys[k] = len(lines) - 1 - except: - if PF_ENV_DEBUG: - print("[%s] skip Processing %s" % (conf_filename, line)) - - # Update values from Env - for k in sorted(os.environ.keys()): - v = os.environ[k].strip() - - # Hide the value in logs if is password. - if "password" in k.lower(): - displayValue = "********" - else: - displayValue = v - - if k.startswith(prefix): - k = k[len(prefix):] - if k in keys: - print('[%s] Applying config %s = %s' % (conf_filename, k, displayValue)) - idx = keys[k] - lines[idx] = '%s=%s\n' % (k, v) - - - # Ensure we have a new-line at the end of the file, to avoid issue - # when appending more lines to the config - lines.append('\n') - - # Add new keys from Env - for k in sorted(os.environ.keys()): - v = os.environ[k] - if not k.startswith(prefix): - continue - - # Hide the value in logs if is password. - if "password" in k.lower(): - displayValue = "********" - else: - displayValue = v - - k = k[len(prefix):] - if k not in keys: - print('[%s] Adding config %s = %s' % (conf_filename, k, displayValue)) - lines.append('%s=%s\n' % (k, v)) - else: - print('[%s] Updating config %s = %s' % (conf_filename, k, displayValue)) - lines[keys[k]] = '%s=%s\n' % (k, v) - - - # Store back the updated config in the same file - f = open(conf_filename, 'w') - for line in lines: - f.write(line) - f.close() +# DEPRECATED: Use "apply-config-from-env.py --prefix MY_PREFIX_ conf_file" instead +# this is not a python script, but a bash script. Call apply-config-from-env.py with the prefix argument +SCRIPT_DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" >/dev/null 2>&1 && pwd)" +"${SCRIPT_DIR}/apply-config-from-env.py" --prefix "$1" "${@:2}" diff --git a/docker/pulsar/scripts/apply-config-from-env.py b/docker/pulsar/scripts/apply-config-from-env.py index b8b479fc15b85..da51f05f8be66 100755 --- a/docker/pulsar/scripts/apply-config-from-env.py +++ b/docker/pulsar/scripts/apply-config-from-env.py @@ -25,18 +25,29 @@ ## ./apply-config-from-env file.conf ## -import os, sys - -if len(sys.argv) < 2: - print('Usage: %s' % (sys.argv[0])) +import os, sys, argparse + +parser = argparse.ArgumentParser(description='Pulsar configuration file customizer based on environment variables') +parser.add_argument('--prefix', default='PULSAR_PREFIX_', help='Prefix for environment variables, default is PULSAR_PREFIX_') +parser.add_argument('conf_files', nargs='*', help='Configuration files') +args = parser.parse_args() +if not args.conf_files: + parser.print_help() sys.exit(1) -# Always apply env config to env scripts as well -conf_files = sys.argv[1:] +env_prefix = args.prefix +conf_files = args.conf_files -PF_ENV_PREFIX = 'PULSAR_PREFIX_' PF_ENV_DEBUG = (os.environ.get('PF_ENV_DEBUG','0') == '1') +# List of keys where the value should not be displayed in logs +sensitive_keys = ["brokerClientAuthenticationParameters", "bookkeeperClientAuthenticationParameters", "tokenSecretKey"] + +def sanitize_display_value(k, v): + if "password" in k.lower() or k in sensitive_keys or (k == "tokenSecretKey" and v.startswith("data:")): + return "********" + return v + for conf_filename in conf_files: lines = [] # List of config file lines keys = {} # Map a key to its line number in the file @@ -47,7 +58,6 @@ line = line.strip() if not line: continue - try: k,v = line.split('=', 1) if k.startswith('#'): @@ -61,37 +71,26 @@ for k in sorted(os.environ.keys()): v = os.environ[k].strip() - # Hide the value in logs if is password. - if "password" in k.lower(): - displayValue = "********" - else: - displayValue = v - - if k.startswith(PF_ENV_PREFIX): - k = k[len(PF_ENV_PREFIX):] if k in keys: + displayValue = sanitize_display_value(k, v) print('[%s] Applying config %s = %s' % (conf_filename, k, displayValue)) idx = keys[k] lines[idx] = '%s=%s\n' % (k, v) - # Ensure we have a new-line at the end of the file, to avoid issue # when appending more lines to the config lines.append('\n') - - # Add new keys from Env + + # Add new keys from Env for k in sorted(os.environ.keys()): - v = os.environ[k] - if not k.startswith(PF_ENV_PREFIX): + if not k.startswith(env_prefix): continue - # Hide the value in logs if is password. - if "password" in k.lower(): - displayValue = "********" - else: - displayValue = v + v = os.environ[k].strip() + k = k[len(env_prefix):] + + displayValue = sanitize_display_value(k, v) - k = k[len(PF_ENV_PREFIX):] if k not in keys: print('[%s] Adding config %s = %s' % (conf_filename, k, displayValue)) lines.append('%s=%s\n' % (k, v)) @@ -99,10 +98,8 @@ print('[%s] Updating config %s = %s' % (conf_filename, k, displayValue)) lines[keys[k]] = '%s=%s\n' % (k, v) - # Store back the updated config in the same file f = open(conf_filename, 'w') for line in lines: f.write(line) - f.close() - + f.close() \ No newline at end of file From abb4cf30a4183b4d235d04b87b1ac801f07a6281 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Mon, 19 Feb 2024 14:57:18 +0800 Subject: [PATCH 267/699] [fix][broker][branch-3.1] Avoid PublishRateLimiter use an already closed RateLimiter (#22011) (cherry picked from commit 8cce14c31f787e17f96222a3676a010c9157cd8c) --- .../ResourceGroupPublishLimiter.java | 2 +- .../ResourceGroupRateLimiterTest.java | 87 +++++++++++++++++-- .../PrecisTopicPublishRateThrottleTest.java | 75 +++++++++++++++- .../pulsar/common/util/RateLimiter.java | 7 +- 4 files changed, 161 insertions(+), 10 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupPublishLimiter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupPublishLimiter.java index 85e00bb2f87dc..0377ec86488d4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupPublishLimiter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupPublishLimiter.java @@ -135,7 +135,7 @@ public void update(long publishRateInMsgs, long publishRateInBytes) { public boolean tryAcquire(int numbers, long bytes) { return (publishRateLimiterOnMessage == null || publishRateLimiterOnMessage.tryAcquire(numbers)) - && (publishRateLimiterOnByte == null || publishRateLimiterOnByte.tryAcquire(bytes)); + && (publishRateLimiterOnByte == null || publishRateLimiterOnByte.tryAcquire(bytes)); } public void registerRateLimitFunction(String name, RateLimitFunction func) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupRateLimiterTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupRateLimiterTest.java index fed827b1517e6..9efd2c109f355 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupRateLimiterTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupRateLimiterTest.java @@ -21,23 +21,30 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; - +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.BiFunction; +import java.util.function.Function; +import lombok.Cleanup; +import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.client.admin.PulsarAdminException; 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.common.naming.NamespaceName; +import org.apache.pulsar.common.util.RateLimiter; import org.awaitility.Awaitility; +import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - public class ResourceGroupRateLimiterTest extends BrokerTestBase { final String rgName = "testRG"; @@ -147,6 +154,76 @@ public void testResourceGroupPublishRateLimit() throws Exception { testRateLimit(); } + @Test + public void testWithConcurrentUpdate() throws Exception { + cleanup(); + setup(); + createResourceGroup(rgName, testAddRg); + admin.namespaces().setNamespaceResourceGroup(namespaceName, rgName); + + Awaitility.await().untilAsserted(() -> + assertNotNull(pulsar.getResourceGroupServiceManager() + .getNamespaceResourceGroup(NamespaceName.get(namespaceName)))); + + Awaitility.await().untilAsserted(() -> + assertNotNull(pulsar.getResourceGroupServiceManager() + .resourceGroupGet(rgName).getResourceGroupPublishLimiter())); + + ResourceGroupPublishLimiter resourceGroupPublishLimiter = Mockito.spy(pulsar.getResourceGroupServiceManager() + .resourceGroupGet(rgName).getResourceGroupPublishLimiter()); + + AtomicBoolean blocking = new AtomicBoolean(false); + BiFunction, Long, Boolean> blockFunc = (function, acquirePermit) -> { + blocking.set(true); + while (blocking.get()) { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + return function.apply(acquirePermit); + }; + + Mockito.doAnswer(invocation -> { + RateLimiter publishRateLimiterOnMessage = + (RateLimiter) FieldUtils.readDeclaredField(resourceGroupPublishLimiter, + "publishRateLimiterOnMessage", true); + RateLimiter publishRateLimiterOnByte = + (RateLimiter) FieldUtils.readDeclaredField(resourceGroupPublishLimiter, + "publishRateLimiterOnByte", true); + int numbers = invocation.getArgument(0); + long bytes = invocation.getArgument(1); + return (publishRateLimiterOnMessage == null || publishRateLimiterOnMessage.tryAcquire(numbers)) + && (publishRateLimiterOnByte == null || blockFunc.apply(publishRateLimiterOnByte::tryAcquire, bytes)); + }).when(resourceGroupPublishLimiter).tryAcquire(Mockito.anyInt(), Mockito.anyLong()); + + ConcurrentHashMap resourceGroupsMap = + (ConcurrentHashMap) FieldUtils.readDeclaredField(pulsar.getResourceGroupServiceManager(), + "resourceGroupsMap", true); + FieldUtils.writeDeclaredField(resourceGroupsMap.get(rgName), "resourceGroupPublishLimiter", + resourceGroupPublishLimiter, true); + @Cleanup + Producer producer = pulsarClient.newProducer() + .topic(namespaceName + "/test-topic") + .create(); + + CompletableFuture sendFuture = producer.sendAsync(new byte[MESSAGE_SIZE]); + + Awaitility.await().untilAsserted(() -> Assert.assertTrue(blocking.get())); + + testAddRg.setPublishRateInBytes(Long.valueOf(MESSAGE_SIZE) + 1); + admin.resourcegroups().updateResourceGroup(rgName, testAddRg); + blocking.set(false); + + sendFuture.join(); + + // Now detach the namespace + admin.namespaces().removeNamespaceResourceGroup(namespaceName); + deleteResourceGroup(rgName); + } + + private void prepareData() { testAddRg.setPublishRateInBytes(Long.valueOf(MESSAGE_SIZE)); testAddRg.setPublishRateInMsgs(1); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PrecisTopicPublishRateThrottleTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PrecisTopicPublishRateThrottleTest.java index c22ed41fc1533..b13f150387bee 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PrecisTopicPublishRateThrottleTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/PrecisTopicPublishRateThrottleTest.java @@ -18,16 +18,25 @@ */ package org.apache.pulsar.broker.service; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.BiFunction; +import java.util.function.Function; +import lombok.Cleanup; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.common.policies.data.PublishRate; +import org.apache.pulsar.common.util.RateLimiter; import org.awaitility.Awaitility; +import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.Test; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - @Test(groups = "broker") +@Slf4j public class PrecisTopicPublishRateThrottleTest extends BrokerTestBase{ @Override @@ -180,4 +189,64 @@ public void testBrokerLevelPublishRateDynamicUpdate() throws Exception{ producer.close(); super.internalCleanup(); } + + @Test + public void testWithConcurrentUpdate() throws Exception { + PublishRate publishRate = new PublishRate(-1,10); + conf.setPreciseTopicPublishRateLimiterEnable(true); + conf.setMaxPendingPublishRequestsPerConnection(1000); + super.baseSetup(); + admin.namespaces().setPublishRate("prop/ns-abc", publishRate); + final String topic = "persistent://prop/ns-abc/testWithConcurrentUpdate"; + @Cleanup + org.apache.pulsar.client.api.Producer producer = pulsarClient.newProducer() + .topic(topic) + .producerName("producer-name") + .create(); + + AbstractTopic topicRef = (AbstractTopic) pulsar.getBrokerService().getTopicReference(topic).get(); + Assert.assertNotNull(topicRef); + + PublishRateLimiter topicPublishRateLimiter = Mockito.spy(topicRef.getTopicPublishRateLimiter()); + + AtomicBoolean blocking = new AtomicBoolean(false); + BiFunction, Long, Boolean> blockFunc = (function, acquirePermit) -> { + blocking.set(true); + while (blocking.get()) { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + return function.apply(acquirePermit); + }; + + Mockito.doAnswer(invocation -> { + log.info("tryAcquire: {}, {}", invocation.getArgument(0), invocation.getArgument(1)); + RateLimiter publishRateLimiterOnMessage = + (RateLimiter) FieldUtils.readDeclaredField(topicPublishRateLimiter, + "topicPublishRateLimiterOnMessage", true); + RateLimiter publishRateLimiterOnByte = + (RateLimiter) FieldUtils.readDeclaredField(topicPublishRateLimiter, + "topicPublishRateLimiterOnByte", true); + int numbers = invocation.getArgument(0); + long bytes = invocation.getArgument(1); + return (publishRateLimiterOnMessage == null || publishRateLimiterOnMessage.tryAcquire(numbers)) + && (publishRateLimiterOnByte == null || blockFunc.apply(publishRateLimiterOnByte::tryAcquire, bytes)); + }).when(topicPublishRateLimiter).tryAcquire(Mockito.anyInt(), Mockito.anyLong()); + + FieldUtils.writeField(topicRef, "topicPublishRateLimiter", topicPublishRateLimiter, true); + + CompletableFuture sendFuture = producer.sendAsync(new byte[10]); + + Awaitility.await().untilAsserted(() -> Assert.assertTrue(blocking.get())); + publishRate.publishThrottlingRateInByte = 20; + admin.namespaces().setPublishRate("prop/ns-abc", publishRate); + blocking.set(false); + + sendFuture.join(); + + super.internalCleanup(); + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/RateLimiter.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/RateLimiter.java index 4ecb29b2462cc..17199664f12d1 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/RateLimiter.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/RateLimiter.java @@ -28,6 +28,7 @@ import java.util.concurrent.TimeUnit; import java.util.function.Supplier; import lombok.Builder; +import lombok.extern.slf4j.Slf4j; /** * A Rate Limiter that distributes permits at a configurable rate. Each {@link #acquire()} blocks if necessary until a @@ -50,6 +51,7 @@ *
  • Faster: RateLimiter is light-weight and faster than Guava-RateLimiter
  • * */ +@Slf4j public class RateLimiter implements AutoCloseable{ private final ScheduledExecutorService executorService; private long rateTime; @@ -175,7 +177,10 @@ public synchronized boolean tryAcquire() { * @return {@code true} if the permits were acquired, {@code false} otherwise */ public synchronized boolean tryAcquire(long acquirePermit) { - checkArgument(!isClosed(), "Rate limiter is already shutdown"); + if (isClosed()) { + log.info("The current rate limiter is already shutdown, acquire permits directly."); + return true; + } // lazy init and start task only once application start using it if (renewTask == null) { renewTask = createTask(); From c9b523c4907dc6f43f5e19d27128684141298d53 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 21 Feb 2024 09:34:29 +0200 Subject: [PATCH 268/699] [fix][broker] Support running docker container with gid != 0 (#22081) (cherry picked from commit 4097ddd5e8c4fae4d95c939222341e5ad5dd6d20) (cherry picked from commit 831f77e9071f136da3165db8c022769152999f96) --- docker/pulsar/Dockerfile | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index a127af166eb52..b465bbf251da8 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -36,10 +36,14 @@ COPY scripts/install-pulsar-client.sh /pulsar/bin # The final image needs to give the root group sufficient permission for Pulsar components # to write to specific directories within /pulsar +# The ownership is changed to uid 10000 to allow using a different root group. This is necessary when running the +# container when gid=0 is prohibited. In that case, the container must be run with uid 10000 with +# any group id != 0 (for example 10001). # The file permissions are preserved when copying files from this builder image to the target image. RUN for SUBDIRECTORY in conf data download logs; do \ [ -d /pulsar/$SUBDIRECTORY ] || mkdir /pulsar/$SUBDIRECTORY; \ - chmod -R g+w /pulsar/$SUBDIRECTORY; \ + chmod -R ug+w /pulsar/$SUBDIRECTORY; \ + chown -R 10000:0 /pulsar/$SUBDIRECTORY; \ done # Trino writes logs to this directory (at least during tests), so we need to give the process permission From 25a0a7f76d336d73f69bf5ce00c03c5213f13af0 Mon Sep 17 00:00:00 2001 From: Masahiro Sakamoto Date: Wed, 21 Feb 2024 18:51:56 +0900 Subject: [PATCH 269/699] [fix][sec] Upgrade commons-compress to 1.26.0 (#22086) (cherry picked from commit 613a77100226628d8685d34260685d2df2b405ae) (cherry picked from commit e89602394480688be3cabe452269b19f2d1f3db5) --- distribution/server/src/assemble/LICENSE.bin.txt | 2 +- distribution/shell/src/assemble/LICENSE.bin.txt | 2 +- pom.xml | 2 +- .../pulsar/functions/instance/JavaInstanceDepsTest.java | 4 ++++ pulsar-sql/presto-distribution/LICENSE | 2 +- 5 files changed, 8 insertions(+), 4 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 4181bee8a4e69..4e018e71d8058 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-lang-commons-lang-2.6.jar - commons-logging-commons-logging-1.1.1.jar - org.apache.commons-commons-collections4-4.4.jar - - org.apache.commons-commons-compress-1.21.jar + - org.apache.commons-commons-compress-1.26.0.jar - org.apache.commons-commons-lang3-3.11.jar - org.apache.commons-commons-text-1.10.0.jar * Netty diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index d4d72973993a3..31e0e884caa16 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -342,7 +342,7 @@ The Apache Software License, Version 2.0 - commons-logging-1.2.jar - commons-lang3-3.11.jar - commons-text-1.10.0.jar - - commons-compress-1.21.jar + - commons-compress-1.26.0.jar * Netty - netty-buffer-4.1.100.Final.jar - netty-codec-4.1.100.Final.jar diff --git a/pom.xml b/pom.xml index 030b65af39204..7f6b1cde4ab99 100644 --- a/pom.xml +++ b/pom.xml @@ -131,7 +131,7 @@ flexible messaging model and an intuitive client API. package - 1.21 + 1.26.0 4.16.4 3.9.1 diff --git a/pulsar-functions/runtime-all/src/test/java/org/apache/pulsar/functions/instance/JavaInstanceDepsTest.java b/pulsar-functions/runtime-all/src/test/java/org/apache/pulsar/functions/instance/JavaInstanceDepsTest.java index 854c146893243..b65bf17f70b44 100644 --- a/pulsar-functions/runtime-all/src/test/java/org/apache/pulsar/functions/instance/JavaInstanceDepsTest.java +++ b/pulsar-functions/runtime-all/src/test/java/org/apache/pulsar/functions/instance/JavaInstanceDepsTest.java @@ -46,6 +46,8 @@ * 8. Apache AVRO * 9. Jackson Mapper and Databind (dependency of AVRO) * 10. Apache Commons Compress (dependency of AVRO) + * 11. Apache Commons Lang (dependency of Apache Commons Compress) + * 12. Apache Commons IO (dependency of Apache Commons Compress) */ public class JavaInstanceDepsTest { @@ -71,6 +73,8 @@ public void testInstanceJarDeps() throws IOException { && !name.startsWith("org/apache/avro") && !name.startsWith("com/fasterxml/jackson") && !name.startsWith("org/apache/commons/compress") + && !name.startsWith("org/apache/commons/lang3") + && !name.startsWith("org/apache/commons/io") && !name.startsWith("com/google") && !name.startsWith("org/checkerframework") && !name.startsWith("javax/annotation") diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 0e0c99f85a117..392d83d9f13bc 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -228,7 +228,7 @@ The Apache Software License, Version 2.0 - guice-5.1.0.jar * Apache Commons - commons-math3-3.6.1.jar - - commons-compress-1.21.jar + - commons-compress-1.26.0.jar - commons-lang3-3.11.jar * Netty - netty-buffer-4.1.100.Final.jar From 4946e6238354019eae2a77a2dac74fe8ca61cb7d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 26 Feb 2024 13:50:10 +0200 Subject: [PATCH 270/699] [improve][fn] Optimize Function Worker startup by lazy loading and direct zip/bytecode access (#22122) (cherry picked from commit bbc62245c5ddba1de4b1e7cee4ab49334bc36277) (cherry picked from commit 3d3606bfdae3398450f63a5274115c54a326eb37) --- conf/functions_worker.yml | 15 +- .../server/src/assemble/LICENSE.bin.txt | 4 + pom.xml | 14 + .../pulsar/common/nar/NarClassLoader.java | 33 +- .../apache/pulsar/common/nar/NarUnpacker.java | 27 +- .../apache/pulsar/functions/LocalRunner.java | 45 +- .../runtime/JavaInstanceStarter.java | 19 +- .../runtime/thread/ThreadRuntime.java | 6 +- .../functions/worker/ConnectorsManager.java | 46 +- .../functions/worker/FunctionsManager.java | 44 +- .../pulsar/functions/worker/WorkerConfig.java | 21 +- pulsar-functions/utils/pom.xml | 11 + .../functions/utils/FunctionCommon.java | 311 +-- .../functions/utils/FunctionConfigUtils.java | 155 +- .../functions/utils/FunctionFilePackage.java | 179 ++ .../utils/FunctionRuntimeCommon.java | 170 ++ .../utils/LoadedFunctionPackage.java | 89 + .../functions/utils/SinkConfigUtils.java | 125 +- .../functions/utils/SourceConfigUtils.java | 82 +- .../utils/ValidatableFunctionPackage.java | 59 + .../functions/utils/ValidatorUtils.java | 207 +- .../utils/functions/FunctionArchive.java | 52 +- .../utils/functions/FunctionUtils.java | 74 +- .../pulsar/functions/utils/io/Connector.java | 76 +- .../functions/utils/io/ConnectorUtils.java | 153 +- .../functions/utils/FunctionCommonTest.java | 81 +- .../utils/FunctionConfigUtilsTest.java | 59 +- .../functions/utils/SinkConfigUtilsTest.java | 67 +- .../functions/worker/FunctionActioner.java | 13 +- .../functions/worker/PulsarWorkerService.java | 8 + .../worker/rest/api/ComponentImpl.java | 11 +- .../worker/rest/api/FunctionsImpl.java | 40 +- .../functions/worker/rest/api/SinksImpl.java | 59 +- .../worker/rest/api/SourcesImpl.java | 33 +- .../worker/rest/api/FunctionsImplTest.java | 2 +- .../api/v2/FunctionApiV2ResourceTest.java | 1446 +----------- .../v3/AbstractFunctionApiResourceTest.java | 1367 +++++++++++ .../api/v3/AbstractFunctionsResourceTest.java | 323 +++ .../api/v3/FunctionApiV3ResourceTest.java | 2079 +++-------------- .../rest/api/v3/SinkApiV3ResourceTest.java | 451 +--- .../rest/api/v3/SourceApiV3ResourceTest.java | 354 +-- .../conf/functions_worker.conf | 2 +- .../integration/topologies/PulsarCluster.java | 39 +- 43 files changed, 3642 insertions(+), 4809 deletions(-) create mode 100644 pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionFilePackage.java create mode 100644 pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionRuntimeCommon.java create mode 100644 pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/LoadedFunctionPackage.java create mode 100644 pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/ValidatableFunctionPackage.java create mode 100644 pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionApiResourceTest.java create mode 100644 pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionsResourceTest.java diff --git a/conf/functions_worker.yml b/conf/functions_worker.yml index 4c5b6aab1b7f4..8c62536971990 100644 --- a/conf/functions_worker.yml +++ b/conf/functions_worker.yml @@ -43,6 +43,16 @@ metadataStoreOperationTimeoutSeconds: 30 # Metadata store cache expiry time in seconds metadataStoreCacheExpirySeconds: 300 +# Specifies if the function worker should use classloading for validating submissions for built-in +# connectors and functions. This is required for validateConnectorConfig to take effect. +# Default is false. +enableClassloadingOfBuiltinFiles: false + +# Specifies if the function worker should use classloading for validating submissions for external +# connectors and functions. This is required for validateConnectorConfig to take effect. +# Default is false. +enableClassloadingOfExternalFiles: false + ################################ # Function package management ################################ @@ -400,7 +410,10 @@ saslJaasServerRoleTokenSignerSecretPath: connectorsDirectory: ./connectors functionsDirectory: ./functions -# Should connector config be validated during submission +# Enables extended validation for connector config with fine-grain annotation based validation +# during submission. Classloading with either enableClassloadingOfExternalFiles or +# enableClassloadingOfBuiltinFiles must be enabled on the worker for this to take effect. +# Default is false. validateConnectorConfig: false # Whether to initialize distributed log metadata by runtime. diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 4e018e71d8058..9b7074a6299a0 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -445,6 +445,10 @@ The Apache Software License, Version 2.0 * Jodah - net.jodah-typetools-0.5.0.jar - net.jodah-failsafe-2.4.4.jar + * Byte Buddy + - net.bytebuddy-byte-buddy-1.14.12.jar + * 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 diff --git a/pom.xml b/pom.xml index 7f6b1cde4ab99..f721c7542286b 100644 --- a/pom.xml +++ b/pom.xml @@ -162,6 +162,8 @@ flexible messaging model and an intuitive client API. 0.43.3 true 0.5.0 + 1.14.12 + 1.17 3.19.6 ${protobuf3.version} 1.55.3 @@ -1063,6 +1065,18 @@ flexible messaging model and an intuitive client API. ${typetools.version}
    + + net.bytebuddy + byte-buddy + ${byte-buddy.version} + + + + org.zeroturnaround + zt-zip + ${zt-zip.version} + + io.grpc grpc-bom diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarClassLoader.java b/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarClassLoader.java index 620e1156d3555..9736d8b47ef71 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarClassLoader.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarClassLoader.java @@ -154,6 +154,11 @@ public NarClassLoader run() { }); } + public static List getClasspathFromArchive(File narPath, String narExtractionDirectory) throws IOException { + File unpacked = NarUnpacker.unpackNar(narPath, getNarExtractionDirectory(narExtractionDirectory)); + return getClassPathEntries(unpacked); + } + private static File getNarExtractionDirectory(String configuredDirectory) { return new File(configuredDirectory + "/" + TMP_DIR_PREFIX); } @@ -164,16 +169,11 @@ private static File getNarExtractionDirectory(String configuredDirectory) { * @param narWorkingDirectory * directory to explode nar contents to * @param parent - * @throws IllegalArgumentException - * if the NAR is missing the Java Services API file for FlowFileProcessor implementations. - * @throws ClassNotFoundException - * if any of the FlowFileProcessor implementations defined by the Java Services API cannot be - * loaded. * @throws IOException * if an error occurs while loading the NAR. */ private NarClassLoader(final File narWorkingDirectory, Set additionalJars, ClassLoader parent) - throws ClassNotFoundException, IOException { + throws IOException { super(new URL[0], parent); this.narWorkingDirectory = narWorkingDirectory; @@ -238,22 +238,31 @@ public List getServiceImplementation(String serviceName) throws IOExcept * if the URL list could not be updated. */ private void updateClasspath(File root) throws IOException { - addURL(root.toURI().toURL()); // for compiled classes, META-INF/, etc. + getClassPathEntries(root).forEach(f -> { + try { + addURL(f.toURI().toURL()); + } catch (IOException e) { + log.error("Failed to add entry to classpath: {}", f, e); + } + }); + } + static List getClassPathEntries(File root) { + List classPathEntries = new ArrayList<>(); + classPathEntries.add(root); File dependencies = new File(root, "META-INF/bundled-dependencies"); if (!dependencies.isDirectory()) { - log.warn("{} does not contain META-INF/bundled-dependencies!", narWorkingDirectory); + log.warn("{} does not contain META-INF/bundled-dependencies!", root); } - addURL(dependencies.toURI().toURL()); + classPathEntries.add(dependencies); if (dependencies.isDirectory()) { final File[] jarFiles = dependencies.listFiles(JAR_FILTER); if (jarFiles != null) { Arrays.sort(jarFiles, Comparator.comparing(File::getName)); - for (File libJar : jarFiles) { - addURL(libJar.toURI().toURL()); - } + classPathEntries.addAll(Arrays.asList(jarFiles)); } } + return classPathEntries; } @Override diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarUnpacker.java b/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarUnpacker.java index 9bd5bc48df819..1e34c3e4fe706 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarUnpacker.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarUnpacker.java @@ -32,13 +32,14 @@ import java.io.RandomAccessFile; import java.nio.channels.FileChannel; import java.nio.channels.FileLock; +import java.nio.file.Path; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; import java.util.Base64; import java.util.Enumeration; import java.util.concurrent.ConcurrentHashMap; -import java.util.jar.JarEntry; -import java.util.jar.JarFile; +import java.util.zip.ZipEntry; +import java.util.zip.ZipFile; import lombok.extern.slf4j.Slf4j; /** @@ -113,18 +114,24 @@ static File doUnpackNar(final File nar, final File baseWorkingDirectory, Runnabl * if the NAR could not be unpacked. */ private static void unpack(final File nar, final File workingDirectory) throws IOException { - try (JarFile jarFile = new JarFile(nar)) { - Enumeration jarEntries = jarFile.entries(); - while (jarEntries.hasMoreElements()) { - JarEntry jarEntry = jarEntries.nextElement(); - String name = jarEntry.getName(); - File f = new File(workingDirectory, name); - if (jarEntry.isDirectory()) { + Path workingDirectoryPath = workingDirectory.toPath().normalize(); + try (ZipFile zipFile = new ZipFile(nar)) { + Enumeration zipEntries = zipFile.entries(); + while (zipEntries.hasMoreElements()) { + ZipEntry zipEntry = zipEntries.nextElement(); + String name = zipEntry.getName(); + Path targetFilePath = workingDirectoryPath.resolve(name).normalize(); + if (!targetFilePath.startsWith(workingDirectoryPath)) { + log.error("Invalid zip file with entry '{}'", name); + throw new IOException("Invalid zip file. Aborting unpacking."); + } + File f = targetFilePath.toFile(); + if (zipEntry.isDirectory()) { FileUtils.ensureDirectoryExistAndCanReadAndWrite(f); } else { // The directory entry might appear after the file entry FileUtils.ensureDirectoryExistAndCanReadAndWrite(f.getParentFile()); - makeFile(jarFile.getInputStream(jarEntry), f); + makeFile(zipFile.getInputStream(zipEntry), f); } } } 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 ed9b0af3b43d8..711fa33edb2a2 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 @@ -52,7 +52,9 @@ import lombok.Value; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.common.functions.FunctionConfig; +import org.apache.pulsar.common.functions.FunctionDefinition; import org.apache.pulsar.common.functions.Utils; +import org.apache.pulsar.common.io.ConnectorDefinition; import org.apache.pulsar.common.io.SinkConfig; import org.apache.pulsar.common.io.SourceConfig; import org.apache.pulsar.common.nar.FileUtils; @@ -75,8 +77,11 @@ import org.apache.pulsar.functions.secretsproviderconfigurator.SecretsProviderConfigurator; import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.utils.FunctionConfigUtils; +import org.apache.pulsar.functions.utils.FunctionRuntimeCommon; +import org.apache.pulsar.functions.utils.LoadedFunctionPackage; import org.apache.pulsar.functions.utils.SinkConfigUtils; import org.apache.pulsar.functions.utils.SourceConfigUtils; +import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; import org.apache.pulsar.functions.utils.functioncache.FunctionCacheEntry; import org.apache.pulsar.functions.utils.functions.FunctionArchive; import org.apache.pulsar.functions.utils.functions.FunctionUtils; @@ -357,9 +362,12 @@ public void start(boolean blocking) throws Exception { userCodeFile = functionConfig.getJar(); userCodeClassLoader = extractClassLoader( userCodeFile, ComponentType.FUNCTION, functionConfig.getClassName()); + ValidatableFunctionPackage validatableFunctionPackage = + new LoadedFunctionPackage(getCurrentOrUserCodeClassLoader(), + FunctionDefinition.class); functionDetails = FunctionConfigUtils.convert( functionConfig, - FunctionConfigUtils.validateJavaFunction(functionConfig, getCurrentOrUserCodeClassLoader())); + FunctionConfigUtils.validateJavaFunction(functionConfig, validatableFunctionPackage)); } else if (functionConfig.getRuntime() == FunctionConfig.Runtime.GO) { userCodeFile = functionConfig.getGo(); } else if (functionConfig.getRuntime() == FunctionConfig.Runtime.PYTHON) { @@ -369,7 +377,10 @@ public void start(boolean blocking) throws Exception { } if (functionDetails == null) { - functionDetails = FunctionConfigUtils.convert(functionConfig, getCurrentOrUserCodeClassLoader()); + ValidatableFunctionPackage validatableFunctionPackage = + new LoadedFunctionPackage(getCurrentOrUserCodeClassLoader(), + FunctionDefinition.class); + functionDetails = FunctionConfigUtils.convert(functionConfig, validatableFunctionPackage); } } else if (sourceConfig != null) { inferMissingArguments(sourceConfig); @@ -377,9 +388,10 @@ public void start(boolean blocking) throws Exception { parallelism = sourceConfig.getParallelism(); userCodeClassLoader = extractClassLoader( userCodeFile, ComponentType.SOURCE, sourceConfig.getClassName()); - functionDetails = SourceConfigUtils.convert( - sourceConfig, - SourceConfigUtils.validateAndExtractDetails(sourceConfig, getCurrentOrUserCodeClassLoader(), true)); + ValidatableFunctionPackage validatableFunctionPackage = + new LoadedFunctionPackage(getCurrentOrUserCodeClassLoader(), ConnectorDefinition.class); + functionDetails = SourceConfigUtils.convert(sourceConfig, + SourceConfigUtils.validateAndExtractDetails(sourceConfig, validatableFunctionPackage, true)); } else if (sinkConfig != null) { inferMissingArguments(sinkConfig); userCodeFile = sinkConfig.getArchive(); @@ -387,6 +399,8 @@ public void start(boolean blocking) throws Exception { parallelism = sinkConfig.getParallelism(); userCodeClassLoader = extractClassLoader( userCodeFile, ComponentType.SINK, sinkConfig.getClassName()); + ValidatableFunctionPackage validatableFunctionPackage = + new LoadedFunctionPackage(getCurrentOrUserCodeClassLoader(), ConnectorDefinition.class); if (isNotEmpty(sinkConfig.getTransformFunction())) { transformFunctionCodeClassLoader = extractClassLoader( sinkConfig.getTransformFunction(), @@ -395,16 +409,19 @@ public void start(boolean blocking) throws Exception { } ClassLoader functionClassLoader = null; + ValidatableFunctionPackage validatableTransformFunction = null; if (transformFunctionCodeClassLoader != null) { functionClassLoader = transformFunctionCodeClassLoader.getClassLoader() == null ? Thread.currentThread().getContextClassLoader() : transformFunctionCodeClassLoader.getClassLoader(); + validatableTransformFunction = + new LoadedFunctionPackage(functionClassLoader, FunctionDefinition.class); } functionDetails = SinkConfigUtils.convert( sinkConfig, - SinkConfigUtils.validateAndExtractDetails(sinkConfig, getCurrentOrUserCodeClassLoader(), - functionClassLoader, true)); + SinkConfigUtils.validateAndExtractDetails(sinkConfig, validatableFunctionPackage, + validatableTransformFunction, true)); } else { throw new IllegalArgumentException("Must specify Function, Source or Sink config"); } @@ -472,7 +489,7 @@ private UserCodeClassLoader extractClassLoader(String userCodeFile, ComponentTyp if (classLoader == null) { if (userCodeFile != null && Utils.isFunctionPackageUrlSupported(userCodeFile)) { File file = FunctionCommon.extractFileFromPkgURL(userCodeFile); - classLoader = FunctionCommon.getClassLoaderFromPackage( + classLoader = FunctionRuntimeCommon.getClassLoaderFromPackage( componentType, className, file, narExtractionDirectory); classLoaderCreated = true; } else if (userCodeFile != null) { @@ -494,7 +511,7 @@ private UserCodeClassLoader extractClassLoader(String userCodeFile, ComponentTyp } throw new RuntimeException(errorMsg + " (" + userCodeFile + ") does not exist"); } - classLoader = FunctionCommon.getClassLoaderFromPackage( + classLoader = FunctionRuntimeCommon.getClassLoaderFromPackage( componentType, className, file, narExtractionDirectory); classLoaderCreated = true; } else { @@ -713,7 +730,7 @@ private ClassLoader isBuiltInFunction(String functionType) throws IOException { FunctionArchive function = functions.get(functionName); if (function != null && function.getFunctionDefinition().getFunctionClass() != null) { // Function type is a valid built-in type. - return function.getClassLoader(); + return function.getFunctionPackage().getClassLoader(); } else { return null; } @@ -727,7 +744,7 @@ private ClassLoader isBuiltInSource(String sourceType) throws IOException { Connector connector = connectors.get(source); if (connector != null && connector.getConnectorDefinition().getSourceClass() != null) { // Source type is a valid built-in connector type. - return connector.getClassLoader(); + return connector.getConnectorFunctionPackage().getClassLoader(); } else { return null; } @@ -741,18 +758,18 @@ private ClassLoader isBuiltInSink(String sinkType) throws IOException { Connector connector = connectors.get(sink); if (connector != null && connector.getConnectorDefinition().getSinkClass() != null) { // Sink type is a valid built-in connector type - return connector.getClassLoader(); + return connector.getConnectorFunctionPackage().getClassLoader(); } else { return null; } } private TreeMap getFunctions() throws IOException { - return FunctionUtils.searchForFunctions(functionsDir); + return FunctionUtils.searchForFunctions(functionsDir, narExtractionDirectory, true); } private TreeMap getConnectors() throws IOException { - return ConnectorUtils.searchForConnectors(connectorsDir, narExtractionDirectory); + return ConnectorUtils.searchForConnectors(connectorsDir, narExtractionDirectory, true); } private SecretsProviderConfigurator getSecretsProviderConfigurator() { diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/JavaInstanceStarter.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/JavaInstanceStarter.java index 89281a2f550e2..e23838cb34396 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/JavaInstanceStarter.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/JavaInstanceStarter.java @@ -38,6 +38,9 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; +import net.bytebuddy.description.type.TypeDefinition; +import net.bytebuddy.dynamic.ClassFileLocator; +import net.bytebuddy.pool.TypePool; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.common.functions.WindowConfig; import org.apache.pulsar.common.nar.NarClassLoader; @@ -325,7 +328,8 @@ public void close() { } private void inferringMissingTypeClassName(Function.FunctionDetails.Builder functionDetailsBuilder, - ClassLoader classLoader) throws ClassNotFoundException { + ClassLoader classLoader) { + TypePool typePool = TypePool.Default.of(ClassFileLocator.ForClassLoader.of(classLoader)); switch (functionDetailsBuilder.getComponentType()) { case FUNCTION: if ((functionDetailsBuilder.hasSource() @@ -344,14 +348,13 @@ private void inferringMissingTypeClassName(Function.FunctionDetails.Builder func WindowConfig.class); className = windowConfig.getActualWindowFunctionClassName(); } - - Class[] typeArgs = FunctionCommon.getFunctionTypes(classLoader.loadClass(className), + TypeDefinition[] typeArgs = FunctionCommon.getFunctionTypes(typePool.describe(className).resolve(), isWindowConfigPresent); if (functionDetailsBuilder.hasSource() && functionDetailsBuilder.getSource().getTypeClassName().isEmpty() && typeArgs[0] != null) { Function.SourceSpec.Builder sourceBuilder = functionDetailsBuilder.getSource().toBuilder(); - sourceBuilder.setTypeClassName(typeArgs[0].getName()); + sourceBuilder.setTypeClassName(typeArgs[0].asErasure().getTypeName()); functionDetailsBuilder.setSource(sourceBuilder.build()); } @@ -359,7 +362,7 @@ private void inferringMissingTypeClassName(Function.FunctionDetails.Builder func && functionDetailsBuilder.getSink().getTypeClassName().isEmpty() && typeArgs[1] != null) { Function.SinkSpec.Builder sinkBuilder = functionDetailsBuilder.getSink().toBuilder(); - sinkBuilder.setTypeClassName(typeArgs[1].getName()); + sinkBuilder.setTypeClassName(typeArgs[1].asErasure().getTypeName()); functionDetailsBuilder.setSink(sinkBuilder.build()); } } @@ -368,7 +371,8 @@ private void inferringMissingTypeClassName(Function.FunctionDetails.Builder func if ((functionDetailsBuilder.hasSink() && functionDetailsBuilder.getSink().getTypeClassName().isEmpty())) { String typeArg = - getSinkType(functionDetailsBuilder.getSink().getClassName(), classLoader).getName(); + getSinkType(functionDetailsBuilder.getSink().getClassName(), typePool).asErasure() + .getTypeName(); Function.SinkSpec.Builder sinkBuilder = Function.SinkSpec.newBuilder(functionDetailsBuilder.getSink()); @@ -387,7 +391,8 @@ private void inferringMissingTypeClassName(Function.FunctionDetails.Builder func if ((functionDetailsBuilder.hasSource() && functionDetailsBuilder.getSource().getTypeClassName().isEmpty())) { String typeArg = - getSourceType(functionDetailsBuilder.getSource().getClassName(), classLoader).getName(); + getSourceType(functionDetailsBuilder.getSource().getClassName(), typePool).asErasure() + .getTypeName(); Function.SourceSpec.Builder sourceBuilder = Function.SourceSpec.newBuilder(functionDetailsBuilder.getSource()); diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/thread/ThreadRuntime.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/thread/ThreadRuntime.java index ed128568bcf50..9dca4015d5ef5 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/thread/ThreadRuntime.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/thread/ThreadRuntime.java @@ -124,17 +124,17 @@ private static ClassLoader getFunctionClassLoader(InstanceConfig instanceConfig, if (componentType == Function.FunctionDetails.ComponentType.FUNCTION && functionsManager.isPresent()) { return functionsManager.get() .getFunction(instanceConfig.getFunctionDetails().getBuiltin()) - .getClassLoader(); + .getFunctionPackage().getClassLoader(); } if (componentType == Function.FunctionDetails.ComponentType.SOURCE && connectorsManager.isPresent()) { return connectorsManager.get() .getConnector(instanceConfig.getFunctionDetails().getSource().getBuiltin()) - .getClassLoader(); + .getConnectorFunctionPackage().getClassLoader(); } if (componentType == Function.FunctionDetails.ComponentType.SINK && connectorsManager.isPresent()) { return connectorsManager.get() .getConnector(instanceConfig.getFunctionDetails().getSink().getBuiltin()) - .getClassLoader(); + .getConnectorFunctionPackage().getClassLoader(); } } return loadJars(jarFile, instanceConfig, functionId, instanceConfig.getFunctionDetails().getName(), diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/ConnectorsManager.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/ConnectorsManager.java index e1770b8b64415..19d31d0f63b1d 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/ConnectorsManager.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/ConnectorsManager.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.functions.worker; +import com.google.common.annotations.VisibleForTesting; import java.io.IOException; import java.nio.file.Path; import java.util.List; @@ -27,18 +28,35 @@ import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.common.io.ConfigFieldDefinition; import org.apache.pulsar.common.io.ConnectorDefinition; +import org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactory; import org.apache.pulsar.functions.utils.io.Connector; import org.apache.pulsar.functions.utils.io.ConnectorUtils; @Slf4j -public class ConnectorsManager { +public class ConnectorsManager implements AutoCloseable { @Getter private volatile TreeMap connectors; + @VisibleForTesting + public ConnectorsManager() { + this.connectors = new TreeMap<>(); + } + public ConnectorsManager(WorkerConfig workerConfig) throws IOException { - this.connectors = ConnectorUtils - .searchForConnectors(workerConfig.getConnectorsDirectory(), workerConfig.getNarExtractionDirectory()); + this.connectors = createConnectors(workerConfig); + } + + private static TreeMap createConnectors(WorkerConfig workerConfig) throws IOException { + boolean enableClassloading = workerConfig.getEnableClassloadingOfBuiltinFiles() + || ThreadRuntimeFactory.class.getName().equals(workerConfig.getFunctionRuntimeFactoryClassName()); + return ConnectorUtils.searchForConnectors(workerConfig.getConnectorsDirectory(), + workerConfig.getNarExtractionDirectory(), enableClassloading); + } + + @VisibleForTesting + public void addConnector(String connectorType, Connector connector) { + connectors.put(connectorType, connector); } public Connector getConnector(String connectorType) { @@ -71,7 +89,25 @@ public Path getSinkArchive(String sinkType) { } public void reloadConnectors(WorkerConfig workerConfig) throws IOException { - connectors = ConnectorUtils - .searchForConnectors(workerConfig.getConnectorsDirectory(), workerConfig.getNarExtractionDirectory()); + TreeMap oldConnectors = connectors; + this.connectors = createConnectors(workerConfig); + closeConnectors(oldConnectors); } + + @Override + public void close() { + closeConnectors(connectors); + } + + private void closeConnectors(TreeMap connectorMap) { + connectorMap.values().forEach(connector -> { + try { + connector.close(); + } catch (Exception e) { + log.warn("Failed to close connector", e); + } + }); + connectorMap.clear(); + } + } diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/FunctionsManager.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/FunctionsManager.java index 9199d568cad03..5ab7ff7221abb 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/FunctionsManager.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/FunctionsManager.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.functions.worker; +import com.google.common.annotations.VisibleForTesting; import java.io.IOException; import java.nio.file.Path; import java.util.List; @@ -25,16 +26,25 @@ import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.common.functions.FunctionDefinition; +import org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactory; import org.apache.pulsar.functions.utils.functions.FunctionArchive; import org.apache.pulsar.functions.utils.functions.FunctionUtils; @Slf4j -public class FunctionsManager { - +public class FunctionsManager implements AutoCloseable { private TreeMap functions; + @VisibleForTesting + public FunctionsManager() { + this.functions = new TreeMap<>(); + } + public FunctionsManager(WorkerConfig workerConfig) throws IOException { - this.functions = FunctionUtils.searchForFunctions(workerConfig.getFunctionsDirectory()); + this.functions = createFunctions(workerConfig); + } + + public void addFunction(String functionType, FunctionArchive functionArchive) { + functions.put(functionType, functionArchive); } public FunctionArchive getFunction(String functionType) { @@ -51,6 +61,32 @@ public List getFunctionDefinitions() { } public void reloadFunctions(WorkerConfig workerConfig) throws IOException { - this.functions = FunctionUtils.searchForFunctions(workerConfig.getFunctionsDirectory()); + TreeMap oldFunctions = functions; + this.functions = createFunctions(workerConfig); + closeFunctions(oldFunctions); + } + + private static TreeMap createFunctions(WorkerConfig workerConfig) throws IOException { + boolean enableClassloading = workerConfig.getEnableClassloadingOfBuiltinFiles() + || ThreadRuntimeFactory.class.getName().equals(workerConfig.getFunctionRuntimeFactoryClassName()); + return FunctionUtils.searchForFunctions(workerConfig.getFunctionsDirectory(), + workerConfig.getNarExtractionDirectory(), + enableClassloading); + } + + @Override + public void close() { + closeFunctions(functions); + } + + private void closeFunctions(TreeMap functionMap) { + functionMap.values().forEach(functionArchive -> { + try { + functionArchive.close(); + } catch (Exception e) { + log.warn("Failed to close function archive", e); + } + }); + functionMap.clear(); } } diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java index 0ed73953d7aa7..2d9698103fa0f 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java @@ -238,6 +238,22 @@ public class WorkerConfig implements Serializable, PulsarConfiguration { ) private boolean zooKeeperAllowReadOnlyOperations; + @FieldContext( + category = CATEGORY_WORKER, + doc = "Specifies if the function worker should use classloading for validating submissions for built-in " + + "connectors and functions. This is required for validateConnectorConfig to take effect. " + + "Default is false." + ) + private Boolean enableClassloadingOfBuiltinFiles = false; + + @FieldContext( + category = CATEGORY_WORKER, + doc = "Specifies if the function worker should use classloading for validating submissions for external " + + "connectors and functions. This is required for validateConnectorConfig to take effect. " + + "Default is false." + ) + private Boolean enableClassloadingOfExternalFiles = false; + @FieldContext( category = CATEGORY_CONNECTORS, doc = "The path to the location to locate builtin connectors" @@ -250,7 +266,10 @@ public class WorkerConfig implements Serializable, PulsarConfiguration { private String narExtractionDirectory = NarClassLoader.DEFAULT_NAR_EXTRACTION_DIR; @FieldContext( category = CATEGORY_CONNECTORS, - doc = "Should we validate connector config during submission" + doc = "Enables extended validation for connector config with fine-grain annotation based validation " + + "during submission. Classloading with either enableClassloadingOfExternalFiles or " + + "enableClassloadingOfBuiltinFiles must be enabled on the worker for this to take effect. " + + "Default is false." ) private Boolean validateConnectorConfig = false; @FieldContext( diff --git a/pulsar-functions/utils/pom.xml b/pulsar-functions/utils/pom.xml index 2e2b06828c523..2ac10417c54da 100644 --- a/pulsar-functions/utils/pom.xml +++ b/pulsar-functions/utils/pom.xml @@ -87,6 +87,17 @@ typetools + + net.bytebuddy + byte-buddy + + + + org.zeroturnaround + zt-zip + 1.17 + + ${project.groupId} pulsar-client-original diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionCommon.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionCommon.java index 7df173da0f195..6a3d2f6ad7ddb 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionCommon.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionCommon.java @@ -22,16 +22,9 @@ import com.google.protobuf.AbstractMessage.Builder; import com.google.protobuf.MessageOrBuilder; import com.google.protobuf.util.JsonFormat; -import java.io.ByteArrayOutputStream; import java.io.File; import java.io.IOException; import java.io.InputStream; -import java.io.ObjectOutputStream; -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.ParameterizedType; -import java.lang.reflect.Type; -import java.net.MalformedURLException; import java.net.ServerSocket; import java.net.URISyntaxException; import java.net.URL; @@ -41,10 +34,14 @@ import java.util.Collection; import java.util.Map; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import lombok.AccessLevel; import lombok.NoArgsConstructor; import lombok.extern.slf4j.Slf4j; -import net.jodah.typetools.TypeResolver; +import net.bytebuddy.description.type.TypeDefinition; +import net.bytebuddy.description.type.TypeDescription; +import net.bytebuddy.description.type.TypeList; +import net.bytebuddy.pool.TypePool; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.api.CompressionType; import org.apache.pulsar.client.api.MessageId; @@ -54,16 +51,11 @@ import org.apache.pulsar.client.impl.auth.AuthenticationDataBasic; import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.functions.Utils; -import org.apache.pulsar.common.nar.NarClassLoader; -import org.apache.pulsar.common.nar.NarClassLoaderBuilder; -import org.apache.pulsar.common.util.ClassLoaderUtils; import org.apache.pulsar.functions.api.Function; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.functions.api.WindowFunction; import org.apache.pulsar.functions.proto.Function.FunctionDetails.ComponentType; import org.apache.pulsar.functions.proto.Function.FunctionDetails.Runtime; -import org.apache.pulsar.functions.utils.functions.FunctionUtils; -import org.apache.pulsar.functions.utils.io.ConnectorUtils; import org.apache.pulsar.io.core.BatchSource; import org.apache.pulsar.io.core.Sink; import org.apache.pulsar.io.core.Source; @@ -97,50 +89,74 @@ public static int findAvailablePort() { } } - public static Class[] getFunctionTypes(FunctionConfig functionConfig, ClassLoader classLoader) + public static TypeDefinition[] getFunctionTypes(FunctionConfig functionConfig, TypePool typePool) throws ClassNotFoundException { - return getFunctionTypes(functionConfig, classLoader.loadClass(functionConfig.getClassName())); + return getFunctionTypes(functionConfig, typePool.describe(functionConfig.getClassName()).resolve()); } - public static Class[] getFunctionTypes(FunctionConfig functionConfig, Class functionClass) - throws ClassNotFoundException { + public static TypeDefinition[] getFunctionTypes(FunctionConfig functionConfig, TypeDefinition functionClass) { boolean isWindowConfigPresent = functionConfig.getWindowConfig() != null; return getFunctionTypes(functionClass, isWindowConfigPresent); } - public static Class[] getFunctionTypes(Class userClass, boolean isWindowConfigPresent) { + public static TypeDefinition[] getFunctionTypes(TypeDefinition userClass, boolean isWindowConfigPresent) { Class classParent = getFunctionClassParent(userClass, isWindowConfigPresent); - Class[] typeArgs = TypeResolver.resolveRawArguments(classParent, userClass); + TypeList.Generic typeArgsList = resolveInterfaceTypeArguments(userClass, classParent); + TypeDescription.Generic[] typeArgs = new TypeDescription.Generic[2]; + typeArgs[0] = typeArgsList.get(0); + typeArgs[1] = typeArgsList.get(1); // if window function if (isWindowConfigPresent) { if (classParent.equals(java.util.function.Function.class)) { - if (!typeArgs[0].equals(Collection.class)) { + if (!typeArgs[0].asErasure().isAssignableTo(Collection.class)) { throw new IllegalArgumentException("Window function must take a collection as input"); } - typeArgs[0] = (Class) unwrapType(classParent, userClass, 0); + typeArgs[0] = typeArgs[0].getTypeArguments().get(0); } } - if (typeArgs[1].equals(Record.class)) { - typeArgs[1] = (Class) unwrapType(classParent, userClass, 1); + if (typeArgs[1].asErasure().isAssignableTo(Record.class)) { + typeArgs[1] = typeArgs[1].getTypeArguments().get(0); + } + if (typeArgs[1].asErasure().isAssignableTo(CompletableFuture.class)) { + typeArgs[1] = typeArgs[1].getTypeArguments().get(0); } - return typeArgs; } - public static Class[] getRawFunctionTypes(Class userClass, boolean isWindowConfigPresent) { + private static TypeList.Generic resolveInterfaceTypeArguments(TypeDefinition userClass, Class interfaceClass) { + if (!interfaceClass.isInterface()) { + throw new IllegalArgumentException("interfaceClass must be an interface"); + } + for (TypeDescription.Generic interfaze : userClass.getInterfaces()) { + if (interfaze.asErasure().isAssignableTo(interfaceClass)) { + return interfaze.getTypeArguments(); + } + } + if (userClass.getSuperClass() != null) { + return resolveInterfaceTypeArguments(userClass.getSuperClass(), interfaceClass); + } + return null; + } + + public static TypeDescription.Generic[] getRawFunctionTypes(TypeDefinition userClass, + boolean isWindowConfigPresent) { Class classParent = getFunctionClassParent(userClass, isWindowConfigPresent); - return TypeResolver.resolveRawArguments(classParent, userClass); + TypeList.Generic typeArgsList = resolveInterfaceTypeArguments(userClass, classParent); + TypeDescription.Generic[] typeArgs = new TypeDescription.Generic[2]; + typeArgs[0] = typeArgsList.get(0); + typeArgs[1] = typeArgsList.get(1); + return typeArgs; } - public static Class getFunctionClassParent(Class userClass, boolean isWindowConfigPresent) { + public static Class getFunctionClassParent(TypeDefinition userClass, boolean isWindowConfigPresent) { if (isWindowConfigPresent) { - if (WindowFunction.class.isAssignableFrom(userClass)) { + if (userClass.asErasure().isAssignableTo(WindowFunction.class)) { return WindowFunction.class; } else { return java.util.function.Function.class; } } else { - if (Function.class.isAssignableFrom(userClass)) { + if (userClass.asErasure().isAssignableTo(Function.class)) { return Function.class; } else { return java.util.function.Function.class; @@ -148,41 +164,6 @@ public static Class getFunctionClassParent(Class userClass, boolean isWind } } - private static Type unwrapType(Class type, Class subType, int position) { - Type genericType = TypeResolver.resolveGenericType(type, subType); - Type argType = ((ParameterizedType) genericType).getActualTypeArguments()[position]; - return ((ParameterizedType) argType).getActualTypeArguments()[0]; - } - - public static Object createInstance(String userClassName, ClassLoader classLoader) { - Class theCls; - try { - theCls = Class.forName(userClassName); - } catch (ClassNotFoundException | NoClassDefFoundError cnfe) { - try { - theCls = Class.forName(userClassName, true, classLoader); - } catch (ClassNotFoundException | NoClassDefFoundError e) { - throw new RuntimeException("User class must be in class path", cnfe); - } - } - Object result; - try { - Constructor meth = theCls.getDeclaredConstructor(); - meth.setAccessible(true); - result = meth.newInstance(); - } catch (InstantiationException ie) { - throw new RuntimeException("User class must be concrete", ie); - } catch (NoSuchMethodException e) { - throw new RuntimeException("User class doesn't have such method", e); - } catch (IllegalAccessException e) { - throw new RuntimeException("User class must have a no-arg constructor", e); - } catch (InvocationTargetException e) { - throw new RuntimeException("User class constructor throws exception", e); - } - return result; - - } - public static Runtime convertRuntime(FunctionConfig.Runtime runtime) { for (Runtime type : Runtime.values()) { if (type.name().equals(runtime.name())) { @@ -223,29 +204,34 @@ public static FunctionConfig.ProcessingGuarantees convertProcessingGuarantee( throw new RuntimeException("Unrecognized processing guarantee: " + processingGuarantees.name()); } - public static Class getSourceType(String className, ClassLoader classLoader) throws ClassNotFoundException { - return getSourceType(classLoader.loadClass(className)); + public static TypeDefinition getSourceType(String className, TypePool typePool) { + return getSourceType(typePool.describe(className).resolve()); } - public static Class getSourceType(Class sourceClass) { - - if (Source.class.isAssignableFrom(sourceClass)) { - return TypeResolver.resolveRawArgument(Source.class, sourceClass); - } else if (BatchSource.class.isAssignableFrom(sourceClass)) { - return TypeResolver.resolveRawArgument(BatchSource.class, sourceClass); + public static TypeDefinition getSourceType(TypeDefinition sourceClass) { + if (sourceClass.asErasure().isAssignableTo(Source.class)) { + return resolveInterfaceTypeArguments(sourceClass, Source.class).get(0); + } else if (sourceClass.asErasure().isAssignableTo(BatchSource.class)) { + return resolveInterfaceTypeArguments(sourceClass, BatchSource.class).get(0); } else { throw new IllegalArgumentException( String.format("Source class %s does not implement the correct interface", - sourceClass.getName())); + sourceClass.getActualName())); } } - public static Class getSinkType(String className, ClassLoader classLoader) throws ClassNotFoundException { - return getSinkType(classLoader.loadClass(className)); + public static TypeDefinition getSinkType(String className, TypePool typePool) { + return getSinkType(typePool.describe(className).resolve()); } - public static Class getSinkType(Class sinkClass) { - return TypeResolver.resolveRawArgument(Sink.class, sinkClass); + public static TypeDefinition getSinkType(TypeDefinition sinkClass) { + if (sinkClass.asErasure().isAssignableTo(Sink.class)) { + return resolveInterfaceTypeArguments(sinkClass, Sink.class).get(0); + } else { + throw new IllegalArgumentException( + String.format("Sink class %s does not implement the correct interface", + sinkClass.getActualName())); + } } public static void downloadFromHttpUrl(String destPkgUrl, File targetFile) throws IOException { @@ -264,16 +250,6 @@ public static void downloadFromHttpUrl(String destPkgUrl, File targetFile) throw log.info("Downloading function package from {} to {} completed!", destPkgUrl, targetFile.getAbsoluteFile()); } - public static ClassLoader extractClassLoader(String destPkgUrl) throws IOException, URISyntaxException { - File file = extractFileFromPkgURL(destPkgUrl); - try { - return ClassLoaderUtils.loadJar(file); - } catch (MalformedURLException e) { - throw new IllegalArgumentException( - "Corrupt User PackageFile " + file + " with error " + e.getMessage()); - } - } - public static File createPkgTempFile() throws IOException { return File.createTempFile("functions", ".tmp"); } @@ -297,21 +273,6 @@ public static File extractFileFromPkgURL(String destPkgUrl) throws IOException, } } - public static NarClassLoader extractNarClassLoader(File packageFile, - String narExtractionDirectory) { - if (packageFile != null) { - try { - return NarClassLoaderBuilder.builder() - .narFile(packageFile) - .extractionDirectory(narExtractionDirectory) - .build(); - } catch (IOException e) { - throw new IllegalArgumentException(e.getMessage()); - } - } - return null; - } - public static String getFullyQualifiedInstanceId(org.apache.pulsar.functions.proto.Function.Instance instance) { return getFullyQualifiedInstanceId( instance.getFunctionMetaData().getFunctionDetails().getTenant(), @@ -345,17 +306,6 @@ public static final MessageId getMessageId(long sequenceId) { return new MessageIdImpl(ledgerId, entryId, -1); } - public static byte[] toByteArray(Object obj) throws IOException { - byte[] bytes = null; - try (ByteArrayOutputStream bos = new ByteArrayOutputStream(); - ObjectOutputStream oos = new ObjectOutputStream(bos)) { - oos.writeObject(obj); - oos.flush(); - bytes = bos.toByteArray(); - } - return bytes; - } - public static String getUniquePackageName(String packageName) { return String.format("%s-%s", UUID.randomUUID().toString(), packageName); } @@ -403,146 +353,11 @@ private static String extractFromFullyQualifiedName(String fqfn, int index) { throw new RuntimeException("Invalid Fully Qualified Function Name " + fqfn); } - public static Class getTypeArg(String className, Class funClass, ClassLoader classLoader) - throws ClassNotFoundException { - Class loadedClass = classLoader.loadClass(className); - if (!funClass.isAssignableFrom(loadedClass)) { - throw new IllegalArgumentException( - String.format("class %s is not type of %s", className, funClass.getName())); - } - return TypeResolver.resolveRawArgument(funClass, loadedClass); - } - public static double roundDecimal(double value, int places) { double scale = Math.pow(10, places); return Math.round(value * scale) / scale; } - public static ClassLoader getClassLoaderFromPackage( - ComponentType componentType, - String className, - File packageFile, - String narExtractionDirectory) { - String connectorClassName = className; - ClassLoader jarClassLoader = null; - boolean keepJarClassLoader = false; - ClassLoader narClassLoader = null; - boolean keepNarClassLoader = false; - - Exception jarClassLoaderException = null; - Exception narClassLoaderException = null; - - try { - try { - jarClassLoader = ClassLoaderUtils.extractClassLoader(packageFile); - } catch (Exception e) { - jarClassLoaderException = e; - } - try { - narClassLoader = FunctionCommon.extractNarClassLoader(packageFile, narExtractionDirectory); - } catch (Exception e) { - narClassLoaderException = e; - } - - // if connector class name is not provided, we can only try to load archive as a NAR - if (isEmpty(connectorClassName)) { - if (narClassLoader == null) { - throw new IllegalArgumentException(String.format("%s package does not have the correct format. " - + "Pulsar cannot determine if the package is a NAR package or JAR package. " - + "%s classname is not provided and attempts to load it as a NAR package produced " - + "the following error.", - capFirstLetter(componentType), capFirstLetter(componentType)), - narClassLoaderException); - } - try { - if (componentType == ComponentType.FUNCTION) { - connectorClassName = FunctionUtils.getFunctionClass(narClassLoader); - } else if (componentType == ComponentType.SOURCE) { - connectorClassName = ConnectorUtils.getIOSourceClass((NarClassLoader) narClassLoader); - } else { - connectorClassName = ConnectorUtils.getIOSinkClass((NarClassLoader) narClassLoader); - } - } catch (IOException e) { - throw new IllegalArgumentException(String.format("Failed to extract %s class from archive", - componentType.toString().toLowerCase()), e); - } - - try { - narClassLoader.loadClass(connectorClassName); - keepNarClassLoader = true; - return narClassLoader; - } catch (ClassNotFoundException | NoClassDefFoundError e) { - throw new IllegalArgumentException( - String.format("%s class %s must be in class path", capFirstLetter(componentType), - connectorClassName), e); - } - - } else { - // if connector class name is provided, we need to try to load it as a JAR and as a NAR. - if (jarClassLoader != null) { - try { - jarClassLoader.loadClass(connectorClassName); - keepJarClassLoader = true; - return jarClassLoader; - } catch (ClassNotFoundException | NoClassDefFoundError e) { - // class not found in JAR try loading as a NAR and searching for the class - if (narClassLoader != null) { - - try { - narClassLoader.loadClass(connectorClassName); - keepNarClassLoader = true; - return narClassLoader; - } catch (ClassNotFoundException | NoClassDefFoundError e1) { - throw new IllegalArgumentException( - String.format("%s class %s must be in class path", - capFirstLetter(componentType), connectorClassName), e1); - } - } else { - throw new IllegalArgumentException( - String.format("%s class %s must be in class path", capFirstLetter(componentType), - connectorClassName), e); - } - } - } else if (narClassLoader != null) { - try { - narClassLoader.loadClass(connectorClassName); - keepNarClassLoader = true; - return narClassLoader; - } catch (ClassNotFoundException | NoClassDefFoundError e1) { - throw new IllegalArgumentException( - String.format("%s class %s must be in class path", - capFirstLetter(componentType), connectorClassName), e1); - } - } else { - StringBuilder errorMsg = new StringBuilder(capFirstLetter(componentType) - + " package does not have the correct format." - + " Pulsar cannot determine if the package is a NAR package or JAR package."); - - if (jarClassLoaderException != null) { - errorMsg.append( - " Attempts to load it as a JAR package produced error: " + jarClassLoaderException - .getMessage()); - } - - if (narClassLoaderException != null) { - errorMsg.append( - " Attempts to load it as a NAR package produced error: " + narClassLoaderException - .getMessage()); - } - - throw new IllegalArgumentException(errorMsg.toString()); - } - } - } finally { - if (!keepJarClassLoader) { - ClassLoaderUtils.closeClassLoader(jarClassLoader); - } - if (!keepNarClassLoader) { - ClassLoaderUtils.closeClassLoader(narClassLoader); - } - } - } - public static String capFirstLetter(Enum en) { return StringUtils.capitalize(en.toString().toLowerCase()); } diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java index e4609672a3d0d..ee59317daf755 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java @@ -18,12 +18,11 @@ */ package org.apache.pulsar.functions.utils; -import static org.apache.commons.lang.StringUtils.isBlank; -import static org.apache.commons.lang.StringUtils.isNotBlank; -import static org.apache.commons.lang.StringUtils.isNotEmpty; +import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.commons.lang3.StringUtils.isEmpty; +import static org.apache.commons.lang3.StringUtils.isNotBlank; +import static org.apache.commons.lang3.StringUtils.isNotEmpty; import static org.apache.pulsar.common.functions.Utils.BUILTIN; -import static org.apache.pulsar.common.util.ClassLoaderUtils.loadJar; import static org.apache.pulsar.functions.utils.FunctionCommon.convertFromCompressionType; import static org.apache.pulsar.functions.utils.FunctionCommon.convertFromFunctionDetailsCompressionType; import static org.apache.pulsar.functions.utils.FunctionCommon.convertFromFunctionDetailsSubscriptionPosition; @@ -32,9 +31,7 @@ import com.google.gson.Gson; import com.google.gson.reflect.TypeToken; import java.io.File; -import java.io.IOException; import java.lang.reflect.Type; -import java.net.MalformedURLException; import java.util.Collection; import java.util.HashMap; import java.util.LinkedList; @@ -44,10 +41,13 @@ import lombok.Getter; import lombok.Setter; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang.StringUtils; +import net.bytebuddy.description.type.TypeDefinition; +import net.bytebuddy.pool.TypePool; +import org.apache.commons.lang3.StringUtils; 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.FunctionDefinition; import org.apache.pulsar.common.functions.ProducerConfig; import org.apache.pulsar.common.functions.Resources; import org.apache.pulsar.common.functions.WindowConfig; @@ -55,7 +55,6 @@ import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.proto.Function.FunctionDetails; -import org.apache.pulsar.functions.utils.functions.FunctionUtils; @Slf4j public class FunctionConfigUtils { @@ -74,26 +73,21 @@ public static class ExtractedFunctionDetails { private static final ObjectMapper OBJECT_MAPPER = ObjectMapperFactory.create(); - public static FunctionDetails convert(FunctionConfig functionConfig, ClassLoader classLoader) - throws IllegalArgumentException { + public static FunctionDetails convert(FunctionConfig functionConfig) { + return convert(functionConfig, (ValidatableFunctionPackage) null); + } - if (functionConfig.getRuntime() == FunctionConfig.Runtime.JAVA) { - if (classLoader != null) { - try { - Class[] typeArgs = FunctionCommon.getFunctionTypes(functionConfig, classLoader); - return convert( - functionConfig, - new ExtractedFunctionDetails( - functionConfig.getClassName(), - typeArgs[0].getName(), - typeArgs[1].getName())); - } catch (ClassNotFoundException | NoClassDefFoundError e) { - throw new IllegalArgumentException( - String.format("Function class %s must be in class path", functionConfig.getClassName()), e); - } - } + public static FunctionDetails convert(FunctionConfig functionConfig, + ValidatableFunctionPackage validatableFunctionPackage) + throws IllegalArgumentException { + if (functionConfig == null) { + throw new IllegalArgumentException("Function config is not provided"); + } + if (functionConfig.getRuntime() == FunctionConfig.Runtime.JAVA && validatableFunctionPackage != null) { + return convert(functionConfig, doJavaChecks(functionConfig, validatableFunctionPackage)); + } else { + return convert(functionConfig, new ExtractedFunctionDetails(functionConfig.getClassName(), null, null)); } - return convert(functionConfig, new ExtractedFunctionDetails(functionConfig.getClassName(), null, null)); } public static FunctionDetails convert(FunctionConfig functionConfig, ExtractedFunctionDetails extractedDetails) @@ -593,48 +587,49 @@ public static void inferMissingArguments(FunctionConfig functionConfig, } } - public static ExtractedFunctionDetails doJavaChecks(FunctionConfig functionConfig, ClassLoader clsLoader) { + public static ExtractedFunctionDetails doJavaChecks(FunctionConfig functionConfig, + ValidatableFunctionPackage validatableFunctionPackage) { - String functionClassName = functionConfig.getClassName(); - Class functionClass; + String functionClassName = StringUtils.trimToNull(functionConfig.getClassName()); + TypeDefinition functionClass; try { // if class name in function config is not set, this should be a built-in function // thus we should try to find its class name in the NAR service definition if (functionClassName == null) { - try { - functionClassName = FunctionUtils.getFunctionClass(clsLoader); - } catch (IOException e) { - throw new IllegalArgumentException("Failed to extract source class from archive", e); + FunctionDefinition functionDefinition = + validatableFunctionPackage.getFunctionMetaData(FunctionDefinition.class); + if (functionDefinition == null) { + throw new IllegalArgumentException("Function class name is not provided."); + } + functionClassName = functionDefinition.getFunctionClass(); + if (functionClassName == null) { + throw new IllegalArgumentException("Function class name is not provided."); } } - functionClass = clsLoader.loadClass(functionClassName); + functionClass = validatableFunctionPackage.resolveType(functionClassName); - if (!org.apache.pulsar.functions.api.Function.class.isAssignableFrom(functionClass) - && !java.util.function.Function.class.isAssignableFrom(functionClass) - && !org.apache.pulsar.functions.api.WindowFunction.class.isAssignableFrom(functionClass)) { + if (!functionClass.asErasure().isAssignableTo(org.apache.pulsar.functions.api.Function.class) + && !functionClass.asErasure().isAssignableTo(java.util.function.Function.class) + && !functionClass.asErasure() + .isAssignableTo(org.apache.pulsar.functions.api.WindowFunction.class)) { throw new IllegalArgumentException( String.format("Function class %s does not implement the correct interface", - functionClass.getName())); + functionClassName)); } - } catch (ClassNotFoundException | NoClassDefFoundError e) { + } catch (TypePool.Resolution.NoSuchTypeException e) { throw new IllegalArgumentException( - String.format("Function class %s must be in class path", functionConfig.getClassName()), e); + String.format("Function class %s must be in class path", functionClassName), e); } - Class[] typeArgs; - try { - typeArgs = FunctionCommon.getFunctionTypes(functionConfig, functionClass); - } catch (ClassNotFoundException | NoClassDefFoundError e) { - throw new IllegalArgumentException( - String.format("Function class %s must be in class path", functionConfig.getClassName()), e); - } + TypeDefinition[] typeArgs = FunctionCommon.getFunctionTypes(functionConfig, functionClass); // inputs use default schema, so there is no check needed there // Check if the Input serialization/deserialization class exists in jar or already loaded and that it // implements SerDe class if (functionConfig.getCustomSerdeInputs() != null) { functionConfig.getCustomSerdeInputs().forEach((topicName, inputSerializer) -> { - ValidatorUtils.validateSerde(inputSerializer, typeArgs[0], clsLoader, true); + ValidatorUtils.validateSerde(inputSerializer, typeArgs[0], validatableFunctionPackage.getTypePool(), + true); }); } @@ -649,8 +644,8 @@ public static ExtractedFunctionDetails doJavaChecks(FunctionConfig functionConfi throw new IllegalArgumentException( String.format("Topic %s has an incorrect schema Info", topicName)); } - ValidatorUtils.validateSchema(consumerConfig.getSchemaType(), typeArgs[0], clsLoader, true); - + ValidatorUtils.validateSchema(consumerConfig.getSchemaType(), typeArgs[0], + validatableFunctionPackage.getTypePool(), true); }); } @@ -665,13 +660,16 @@ public static ExtractedFunctionDetails doJavaChecks(FunctionConfig functionConfi "Only one of schemaType or serdeClassName should be set in inputSpec"); } if (!isEmpty(conf.getSerdeClassName())) { - ValidatorUtils.validateSerde(conf.getSerdeClassName(), typeArgs[0], clsLoader, true); + ValidatorUtils.validateSerde(conf.getSerdeClassName(), typeArgs[0], + validatableFunctionPackage.getTypePool(), true); } if (!isEmpty(conf.getSchemaType())) { - ValidatorUtils.validateSchema(conf.getSchemaType(), typeArgs[0], clsLoader, true); + ValidatorUtils.validateSchema(conf.getSchemaType(), typeArgs[0], + validatableFunctionPackage.getTypePool(), true); } if (conf.getCryptoConfig() != null) { - ValidatorUtils.validateCryptoKeyReader(conf.getCryptoConfig(), clsLoader, false); + ValidatorUtils.validateCryptoKeyReader(conf.getCryptoConfig(), + validatableFunctionPackage.getTypePool(), false); } }); } @@ -679,8 +677,8 @@ public static ExtractedFunctionDetails doJavaChecks(FunctionConfig functionConfi if (Void.class.equals(typeArgs[1])) { return new FunctionConfigUtils.ExtractedFunctionDetails( functionClassName, - typeArgs[0].getName(), - typeArgs[1].getName()); + typeArgs[0].asErasure().getTypeName(), + typeArgs[1].asErasure().getTypeName()); } // One and only one of outputSchemaType and outputSerdeClassName should be set @@ -690,22 +688,25 @@ public static ExtractedFunctionDetails doJavaChecks(FunctionConfig functionConfi } if (!isEmpty(functionConfig.getOutputSchemaType())) { - ValidatorUtils.validateSchema(functionConfig.getOutputSchemaType(), typeArgs[1], clsLoader, false); + ValidatorUtils.validateSchema(functionConfig.getOutputSchemaType(), typeArgs[1], + validatableFunctionPackage.getTypePool(), false); } if (!isEmpty(functionConfig.getOutputSerdeClassName())) { - ValidatorUtils.validateSerde(functionConfig.getOutputSerdeClassName(), typeArgs[1], clsLoader, false); + ValidatorUtils.validateSerde(functionConfig.getOutputSerdeClassName(), typeArgs[1], + validatableFunctionPackage.getTypePool(), false); } if (functionConfig.getProducerConfig() != null && functionConfig.getProducerConfig().getCryptoConfig() != null) { ValidatorUtils - .validateCryptoKeyReader(functionConfig.getProducerConfig().getCryptoConfig(), clsLoader, true); + .validateCryptoKeyReader(functionConfig.getProducerConfig().getCryptoConfig(), + validatableFunctionPackage.getTypePool(), true); } return new FunctionConfigUtils.ExtractedFunctionDetails( functionClassName, - typeArgs[0].getName(), - typeArgs[1].getName()); + typeArgs[0].asErasure().getTypeName(), + typeArgs[1].asErasure().getTypeName()); } private static void doPythonChecks(FunctionConfig functionConfig) { @@ -912,47 +913,21 @@ public static Collection collectAllInputTopics(FunctionConfig functionCo return retval; } - public static ClassLoader validate(FunctionConfig functionConfig, File functionPackageFile) { + public static void validateNonJavaFunction(FunctionConfig functionConfig) { doCommonChecks(functionConfig); - if (functionConfig.getRuntime() == FunctionConfig.Runtime.JAVA) { - ClassLoader classLoader; - if (functionPackageFile != null) { - try { - classLoader = loadJar(functionPackageFile); - } catch (MalformedURLException e) { - throw new IllegalArgumentException("Corrupted Jar File", e); - } - } else if (!isEmpty(functionConfig.getJar())) { - File jarFile = new File(functionConfig.getJar()); - if (!jarFile.exists()) { - throw new IllegalArgumentException("Jar file does not exist"); - } - try { - classLoader = loadJar(jarFile); - } catch (Exception e) { - throw new IllegalArgumentException("Corrupted Jar File", e); - } - } else { - throw new IllegalArgumentException("Function Package is not provided"); - } - - doJavaChecks(functionConfig, classLoader); - return classLoader; - } else if (functionConfig.getRuntime() == FunctionConfig.Runtime.GO) { + if (functionConfig.getRuntime() == FunctionConfig.Runtime.GO) { doGolangChecks(functionConfig); - return null; } else if (functionConfig.getRuntime() == FunctionConfig.Runtime.PYTHON) { doPythonChecks(functionConfig); - return null; } else { throw new IllegalArgumentException("Function language runtime is either not set or cannot be determined"); } } public static ExtractedFunctionDetails validateJavaFunction(FunctionConfig functionConfig, - ClassLoader classLoader) { + ValidatableFunctionPackage validatableFunctionPackage) { doCommonChecks(functionConfig); - return doJavaChecks(functionConfig, classLoader); + return doJavaChecks(functionConfig, validatableFunctionPackage); } public static FunctionConfig validateUpdate(FunctionConfig existingConfig, FunctionConfig newConfig) { diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionFilePackage.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionFilePackage.java new file mode 100644 index 0000000000000..8224de32521fb --- /dev/null +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionFilePackage.java @@ -0,0 +1,179 @@ +/* + * 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.utils; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.net.MalformedURLException; +import java.net.URLClassLoader; +import java.util.ArrayList; +import java.util.List; +import net.bytebuddy.description.type.TypeDescription; +import net.bytebuddy.dynamic.ClassFileLocator; +import net.bytebuddy.pool.TypePool; +import org.apache.pulsar.common.nar.NarClassLoader; +import org.apache.pulsar.common.nar.NarClassLoaderBuilder; +import org.apache.pulsar.functions.utils.functions.FunctionUtils; +import org.zeroturnaround.zip.ZipUtil; + +/** + * FunctionFilePackage is a class that represents a function package and + * implements the ValidatableFunctionPackage interface which decouples the + * function package from classloading. + */ +public class FunctionFilePackage implements AutoCloseable, ValidatableFunctionPackage { + private final File file; + private final ClassFileLocator.Compound classFileLocator; + private final TypePool typePool; + private final boolean isNar; + private final String narExtractionDirectory; + private final boolean enableClassloading; + + private ClassLoader classLoader; + + private final Object configMetadata; + + public FunctionFilePackage(File file, String narExtractionDirectory, boolean enableClassloading, + Class configClass) { + this.file = file; + boolean nonZeroFile = file.isFile() && file.length() > 0; + this.isNar = nonZeroFile ? ZipUtil.containsAnyEntry(file, + new String[] {"META-INF/services/pulsar-io.yaml", "META-INF/bundled-dependencies"}) : false; + this.narExtractionDirectory = narExtractionDirectory; + this.enableClassloading = enableClassloading; + if (isNar) { + List classpathFromArchive = null; + try { + classpathFromArchive = NarClassLoader.getClasspathFromArchive(file, narExtractionDirectory); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + List classFileLocators = new ArrayList<>(); + classFileLocators.add(ClassFileLocator.ForClassLoader.ofSystemLoader()); + for (File classpath : classpathFromArchive) { + if (classpath.exists()) { + try { + ClassFileLocator locator; + if (classpath.isDirectory()) { + locator = new ClassFileLocator.ForFolder(classpath); + } else { + locator = ClassFileLocator.ForJarFile.of(classpath); + } + classFileLocators.add(locator); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + } + this.classFileLocator = new ClassFileLocator.Compound(classFileLocators); + this.typePool = TypePool.Default.of(classFileLocator); + try { + this.configMetadata = FunctionUtils.getPulsarIOServiceConfig(file, configClass); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } else { + try { + this.classFileLocator = nonZeroFile + ? new ClassFileLocator.Compound(ClassFileLocator.ForClassLoader.ofSystemLoader(), + ClassFileLocator.ForJarFile.of(file)) : + new ClassFileLocator.Compound(ClassFileLocator.ForClassLoader.ofSystemLoader()); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + this.typePool = + TypePool.Default.of(classFileLocator); + this.configMetadata = null; + } + } + + public TypeDescription resolveType(String className) { + return typePool.describe(className).resolve(); + } + + public boolean isNar() { + return isNar; + } + + public File getFile() { + return file; + } + + public TypePool getTypePool() { + return typePool; + } + + @Override + public T getFunctionMetaData(Class clazz) { + return configMetadata != null ? clazz.cast(configMetadata) : null; + } + + @Override + public synchronized void close() throws IOException { + classFileLocator.close(); + if (classLoader instanceof Closeable) { + ((Closeable) classLoader).close(); + } + } + + public boolean isEnableClassloading() { + return enableClassloading; + } + + public synchronized ClassLoader getClassLoader() { + if (classLoader == null) { + classLoader = createClassLoader(); + } + return classLoader; + } + + private ClassLoader createClassLoader() { + if (enableClassloading) { + if (isNar) { + try { + return NarClassLoaderBuilder.builder() + .narFile(file) + .extractionDirectory(narExtractionDirectory) + .build(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } else { + try { + return new URLClassLoader(new java.net.URL[] {file.toURI().toURL()}, + NarClassLoader.class.getClassLoader()); + } catch (MalformedURLException e) { + throw new UncheckedIOException(e); + } + } + } else { + throw new IllegalStateException("Classloading is not enabled"); + } + } + + @Override + public String toString() { + return "FunctionFilePackage{" + + "file=" + file + + ", isNar=" + isNar + + '}'; + } +} diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionRuntimeCommon.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionRuntimeCommon.java new file mode 100644 index 0000000000000..ed17478dd00ed --- /dev/null +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionRuntimeCommon.java @@ -0,0 +1,170 @@ +/* + * 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.utils; + +import static org.apache.commons.lang3.StringUtils.isEmpty; +import java.io.File; +import java.io.IOException; +import org.apache.pulsar.common.nar.NarClassLoader; +import org.apache.pulsar.common.nar.NarClassLoaderBuilder; +import org.apache.pulsar.common.util.ClassLoaderUtils; +import org.apache.pulsar.functions.proto.Function; +import org.apache.pulsar.functions.utils.functions.FunctionUtils; +import org.apache.pulsar.functions.utils.io.ConnectorUtils; + +public class FunctionRuntimeCommon { + public static NarClassLoader extractNarClassLoader(File packageFile, + String narExtractionDirectory) { + if (packageFile != null) { + try { + return NarClassLoaderBuilder.builder() + .narFile(packageFile) + .extractionDirectory(narExtractionDirectory) + .build(); + } catch (IOException e) { + throw new IllegalArgumentException(e.getMessage()); + } + } + return null; + } + + public static ClassLoader getClassLoaderFromPackage( + Function.FunctionDetails.ComponentType componentType, + String className, + File packageFile, + String narExtractionDirectory) { + String connectorClassName = className; + ClassLoader jarClassLoader = null; + boolean keepJarClassLoader = false; + NarClassLoader narClassLoader = null; + boolean keepNarClassLoader = false; + + Exception jarClassLoaderException = null; + Exception narClassLoaderException = null; + + try { + try { + jarClassLoader = ClassLoaderUtils.extractClassLoader(packageFile); + } catch (Exception e) { + jarClassLoaderException = e; + } + try { + narClassLoader = extractNarClassLoader(packageFile, narExtractionDirectory); + } catch (Exception e) { + narClassLoaderException = e; + } + + // if connector class name is not provided, we can only try to load archive as a NAR + if (isEmpty(connectorClassName)) { + if (narClassLoader == null) { + throw new IllegalArgumentException(String.format("%s package does not have the correct format. " + + "Pulsar cannot determine if the package is a NAR package or JAR package. " + + "%s classname is not provided and attempts to load it as a NAR package produced " + + "the following error.", + FunctionCommon.capFirstLetter(componentType), FunctionCommon.capFirstLetter(componentType)), + narClassLoaderException); + } + try { + if (componentType == Function.FunctionDetails.ComponentType.FUNCTION) { + connectorClassName = FunctionUtils.getFunctionClass(narClassLoader); + } else if (componentType == Function.FunctionDetails.ComponentType.SOURCE) { + connectorClassName = ConnectorUtils.getIOSourceClass(narClassLoader); + } else { + connectorClassName = ConnectorUtils.getIOSinkClass(narClassLoader); + } + } catch (IOException e) { + throw new IllegalArgumentException(String.format("Failed to extract %s class from archive", + componentType.toString().toLowerCase()), e); + } + + try { + narClassLoader.loadClass(connectorClassName); + keepNarClassLoader = true; + return narClassLoader; + } catch (ClassNotFoundException | NoClassDefFoundError e) { + throw new IllegalArgumentException(String.format("%s class %s must be in class path", + FunctionCommon.capFirstLetter(componentType), connectorClassName), e); + } + + } else { + // if connector class name is provided, we need to try to load it as a JAR and as a NAR. + if (jarClassLoader != null) { + try { + jarClassLoader.loadClass(connectorClassName); + keepJarClassLoader = true; + return jarClassLoader; + } catch (ClassNotFoundException | NoClassDefFoundError e) { + // class not found in JAR try loading as a NAR and searching for the class + if (narClassLoader != null) { + + try { + narClassLoader.loadClass(connectorClassName); + keepNarClassLoader = true; + return narClassLoader; + } catch (ClassNotFoundException | NoClassDefFoundError e1) { + throw new IllegalArgumentException( + String.format("%s class %s must be in class path", + FunctionCommon.capFirstLetter(componentType), connectorClassName), e1); + } + } else { + throw new IllegalArgumentException(String.format("%s class %s must be in class path", + FunctionCommon.capFirstLetter(componentType), connectorClassName), e); + } + } + } else if (narClassLoader != null) { + try { + narClassLoader.loadClass(connectorClassName); + keepNarClassLoader = true; + return narClassLoader; + } catch (ClassNotFoundException | NoClassDefFoundError e1) { + throw new IllegalArgumentException( + String.format("%s class %s must be in class path", + FunctionCommon.capFirstLetter(componentType), connectorClassName), e1); + } + } else { + StringBuilder errorMsg = new StringBuilder(FunctionCommon.capFirstLetter(componentType) + + " package does not have the correct format." + + " Pulsar cannot determine if the package is a NAR package or JAR package."); + + if (jarClassLoaderException != null) { + errorMsg.append( + " Attempts to load it as a JAR package produced error: " + jarClassLoaderException + .getMessage()); + } + + if (narClassLoaderException != null) { + errorMsg.append( + " Attempts to load it as a NAR package produced error: " + narClassLoaderException + .getMessage()); + } + + throw new IllegalArgumentException(errorMsg.toString()); + } + } + } finally { + if (!keepJarClassLoader) { + ClassLoaderUtils.closeClassLoader(jarClassLoader); + } + if (!keepNarClassLoader) { + ClassLoaderUtils.closeClassLoader(narClassLoader); + } + } + } + +} diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/LoadedFunctionPackage.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/LoadedFunctionPackage.java new file mode 100644 index 0000000000000..e27ed0eca1973 --- /dev/null +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/LoadedFunctionPackage.java @@ -0,0 +1,89 @@ +/* + * 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.utils; + +import java.io.IOException; +import java.io.UncheckedIOException; +import net.bytebuddy.description.type.TypeDescription; +import net.bytebuddy.dynamic.ClassFileLocator; +import net.bytebuddy.pool.TypePool; +import org.apache.pulsar.common.nar.NarClassLoader; +import org.apache.pulsar.functions.utils.functions.FunctionUtils; + +/** + * LoadedFunctionPackage is a class that represents a function package and + * implements the ValidatableFunctionPackage interface which decouples the + * function package from classloading. This implementation is backed by + * a ClassLoader, and it is used when the function package is already loaded + * by a ClassLoader. This is the case in the LocalRunner and in some of + * the unit tests. + */ +public class LoadedFunctionPackage implements ValidatableFunctionPackage { + private final ClassLoader classLoader; + private final Object configMetadata; + private final TypePool typePool; + + public LoadedFunctionPackage(ClassLoader classLoader, Class configMetadataClass, T configMetadata) { + this.classLoader = classLoader; + this.configMetadata = configMetadata; + typePool = TypePool.Default.of( + ClassFileLocator.ForClassLoader.of(classLoader)); + } + + public LoadedFunctionPackage(ClassLoader classLoader, Class configMetadataClass) { + this.classLoader = classLoader; + if (classLoader instanceof NarClassLoader) { + try { + configMetadata = FunctionUtils.getPulsarIOServiceConfig((NarClassLoader) classLoader, + configMetadataClass); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } else { + configMetadata = null; + } + typePool = TypePool.Default.of( + ClassFileLocator.ForClassLoader.of(classLoader)); + } + + @Override + public TypeDescription resolveType(String className) { + return typePool.describe(className).resolve(); + } + + @Override + public TypePool getTypePool() { + return typePool; + } + + @Override + public T getFunctionMetaData(Class clazz) { + return configMetadata != null ? clazz.cast(configMetadata) : null; + } + + @Override + public boolean isEnableClassloading() { + return true; + } + + @Override + public ClassLoader getClassLoader() { + return classLoader; + } +} 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 7919d69712600..d93676a106d9a 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 @@ -41,23 +41,23 @@ import lombok.Setter; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; +import net.bytebuddy.description.type.TypeDefinition; +import net.bytebuddy.pool.TypePool; import org.apache.commons.lang.StringUtils; 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.FunctionDefinition; import org.apache.pulsar.common.functions.Resources; import org.apache.pulsar.common.io.ConnectorDefinition; import org.apache.pulsar.common.io.SinkConfig; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.nar.NarClassLoader; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.config.validation.ConfigValidation; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.functions.api.utils.IdentityFunction; import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.proto.Function.FunctionDetails; -import org.apache.pulsar.functions.utils.functions.FunctionUtils; -import org.apache.pulsar.functions.utils.io.ConnectorUtils; @Slf4j public class SinkConfigUtils { @@ -402,8 +402,8 @@ public static SinkConfig convertFromDetails(FunctionDetails functionDetails) { } public static ExtractedSinkDetails validateAndExtractDetails(SinkConfig sinkConfig, - ClassLoader sinkClassLoader, - ClassLoader functionClassLoader, + ValidatableFunctionPackage sinkFunction, + ValidatableFunctionPackage transformFunction, boolean validateConnectorConfig) { if (isEmpty(sinkConfig.getTenant())) { throw new IllegalArgumentException("Sink tenant cannot be null"); @@ -443,63 +443,72 @@ public static ExtractedSinkDetails validateAndExtractDetails(SinkConfig sinkConf // if class name in sink config is not set, this should be a built-in sink // thus we should try to find it class name in the NAR service definition if (sinkClassName == null) { - try { - sinkClassName = ConnectorUtils.getIOSinkClass((NarClassLoader) sinkClassLoader); - } catch (IOException e) { - throw new IllegalArgumentException("Failed to extract sink class from archive", e); + ConnectorDefinition connectorDefinition = sinkFunction.getFunctionMetaData(ConnectorDefinition.class); + if (connectorDefinition == null) { + throw new IllegalArgumentException( + "Sink package doesn't contain the META-INF/services/pulsar-io.yaml file."); + } + sinkClassName = connectorDefinition.getSinkClass(); + if (sinkClassName == null) { + throw new IllegalArgumentException("Failed to extract sink class from archive"); } } // check if sink implements the correct interfaces - Class sinkClass; + TypeDefinition sinkClass; try { - sinkClass = sinkClassLoader.loadClass(sinkClassName); - } catch (ClassNotFoundException e) { + sinkClass = sinkFunction.resolveType(sinkClassName); + } catch (TypePool.Resolution.NoSuchTypeException e) { throw new IllegalArgumentException( - String.format("Sink class %s not found in class loader", sinkClassName), e); + String.format("Sink class %s not found", sinkClassName), e); } String functionClassName = sinkConfig.getTransformFunctionClassName(); - Class typeArg; - ClassLoader inputClassLoader; - if (functionClassLoader != null) { + TypeDefinition typeArg; + ValidatableFunctionPackage inputFunction; + if (transformFunction != null) { // if function class name in sink config is not set, this should be a built-in function // thus we should try to find it class name in the NAR service definition if (functionClassName == null) { - try { - functionClassName = FunctionUtils.getFunctionClass(functionClassLoader); - } catch (IOException e) { - throw new IllegalArgumentException("Failed to extract function class from archive", e); + FunctionDefinition functionDefinition = + transformFunction.getFunctionMetaData(FunctionDefinition.class); + if (functionDefinition == null) { + throw new IllegalArgumentException( + "Function package doesn't contain the META-INF/services/pulsar-io.yaml file."); + } + functionClassName = functionDefinition.getFunctionClass(); + if (functionClassName == null) { + throw new IllegalArgumentException("Transform function class name must be set"); } } - Class functionClass; + TypeDefinition functionClass; try { - functionClass = functionClassLoader.loadClass(functionClassName); - } catch (ClassNotFoundException e) { + functionClass = transformFunction.resolveType(functionClassName); + } catch (TypePool.Resolution.NoSuchTypeException e) { throw new IllegalArgumentException( - String.format("Function class %s not found in class loader", functionClassName), e); + String.format("Function class %s not found", functionClassName), e); } // extract type from transform function class - if (!getRawFunctionTypes(functionClass, false)[1].equals(Record.class)) { + if (!getRawFunctionTypes(functionClass, false)[1].asErasure().isAssignableTo(Record.class)) { throw new IllegalArgumentException("Sink transform function output must be of type Record"); } typeArg = getFunctionTypes(functionClass, false)[0]; - inputClassLoader = functionClassLoader; + inputFunction = transformFunction; } else { // extract type from sink class typeArg = getSinkType(sinkClass); - inputClassLoader = sinkClassLoader; + inputFunction = sinkFunction; } if (sinkConfig.getTopicToSerdeClassName() != null) { for (String serdeClassName : sinkConfig.getTopicToSerdeClassName().values()) { - ValidatorUtils.validateSerde(serdeClassName, typeArg, inputClassLoader, true); + ValidatorUtils.validateSerde(serdeClassName, typeArg, inputFunction.getTypePool(), true); } } if (sinkConfig.getTopicToSchemaType() != null) { for (String schemaType : sinkConfig.getTopicToSchemaType().values()) { - ValidatorUtils.validateSchema(schemaType, typeArg, inputClassLoader, true); + ValidatorUtils.validateSchema(schemaType, typeArg, inputFunction.getTypePool(), true); } } @@ -512,23 +521,43 @@ public static ExtractedSinkDetails validateAndExtractDetails(SinkConfig sinkConf throw new IllegalArgumentException("Only one of serdeClassName or schemaType should be set"); } if (!isEmpty(consumerSpec.getSerdeClassName())) { - ValidatorUtils.validateSerde(consumerSpec.getSerdeClassName(), typeArg, inputClassLoader, true); + ValidatorUtils.validateSerde(consumerSpec.getSerdeClassName(), typeArg, + inputFunction.getTypePool(), true); } if (!isEmpty(consumerSpec.getSchemaType())) { - ValidatorUtils.validateSchema(consumerSpec.getSchemaType(), typeArg, inputClassLoader, true); + ValidatorUtils.validateSchema(consumerSpec.getSchemaType(), typeArg, + inputFunction.getTypePool(), true); } if (consumerSpec.getCryptoConfig() != null) { - ValidatorUtils.validateCryptoKeyReader(consumerSpec.getCryptoConfig(), inputClassLoader, false); + ValidatorUtils.validateCryptoKeyReader(consumerSpec.getCryptoConfig(), + inputFunction.getTypePool(), false); } } } - // validate user defined config if enabled and sink is loaded from NAR - if (validateConnectorConfig && sinkClassLoader instanceof NarClassLoader) { - validateSinkConfig(sinkConfig, (NarClassLoader) sinkClassLoader); + if (sinkConfig.getRetainKeyOrdering() != null + && sinkConfig.getRetainKeyOrdering() + && sinkConfig.getProcessingGuarantees() != null + && sinkConfig.getProcessingGuarantees() == FunctionConfig.ProcessingGuarantees.EFFECTIVELY_ONCE) { + throw new IllegalArgumentException( + "When effectively once processing guarantee is specified, retain Key ordering cannot be set"); + } + + if (sinkConfig.getRetainKeyOrdering() != null && sinkConfig.getRetainKeyOrdering() + && sinkConfig.getRetainOrdering() != null && sinkConfig.getRetainOrdering()) { + throw new IllegalArgumentException("Only one of retain ordering or retain key ordering can be set"); } - return new ExtractedSinkDetails(sinkClassName, typeArg.getName(), functionClassName); + // validate user defined config if enabled and classloading is enabled + if (validateConnectorConfig) { + if (sinkFunction.isEnableClassloading()) { + validateSinkConfig(sinkConfig, sinkFunction); + } else { + log.warn("Skipping annotation based validation of sink config as classloading is disabled"); + } + } + + return new ExtractedSinkDetails(sinkClassName, typeArg.asErasure().getTypeName(), functionClassName); } public static Collection collectAllInputTopics(SinkConfig sinkConfig) { @@ -684,29 +713,13 @@ public static SinkConfig validateUpdate(SinkConfig existingConfig, SinkConfig ne return mergedConfig; } - public static void validateSinkConfig(SinkConfig sinkConfig, NarClassLoader narClassLoader) { - - if (sinkConfig.getRetainKeyOrdering() != null - && sinkConfig.getRetainKeyOrdering() - && sinkConfig.getProcessingGuarantees() != null - && sinkConfig.getProcessingGuarantees() == FunctionConfig.ProcessingGuarantees.EFFECTIVELY_ONCE) { - throw new IllegalArgumentException( - "When effectively once processing guarantee is specified, retain Key ordering cannot be set"); - } - - if (sinkConfig.getRetainKeyOrdering() != null && sinkConfig.getRetainKeyOrdering() - && sinkConfig.getRetainOrdering() != null && sinkConfig.getRetainOrdering()) { - throw new IllegalArgumentException("Only one of retain ordering or retain key ordering can be set"); - } - + public static void validateSinkConfig(SinkConfig sinkConfig, ValidatableFunctionPackage sinkFunction) { try { - ConnectorDefinition defn = ConnectorUtils.getConnectorDefinition(narClassLoader); - if (defn.getSinkConfigClass() != null) { - Class configClass = Class.forName(defn.getSinkConfigClass(), true, narClassLoader); + ConnectorDefinition defn = sinkFunction.getFunctionMetaData(ConnectorDefinition.class); + if (defn != null && defn.getSinkConfigClass() != null) { + Class configClass = Class.forName(defn.getSinkConfigClass(), true, sinkFunction.getClassLoader()); validateSinkConfig(sinkConfig, configClass); } - } catch (IOException e) { - throw new IllegalArgumentException("Error validating sink config", e); } catch (ClassNotFoundException e) { throw new IllegalArgumentException("Could not find sink config class", e); } diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SourceConfigUtils.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SourceConfigUtils.java index f3be015d73754..a6430bbea4585 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SourceConfigUtils.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SourceConfigUtils.java @@ -35,7 +35,9 @@ import lombok.Setter; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; -import net.jodah.typetools.TypeResolver; +import net.bytebuddy.description.type.TypeDefinition; +import net.bytebuddy.description.type.TypeDescription; +import net.bytebuddy.pool.TypePool; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.common.functions.ProducerConfig; import org.apache.pulsar.common.functions.Resources; @@ -44,13 +46,11 @@ import org.apache.pulsar.common.io.SourceConfig; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.nar.NarClassLoader; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.config.validation.ConfigValidation; import org.apache.pulsar.functions.api.utils.IdentityFunction; import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.proto.Function.FunctionDetails; -import org.apache.pulsar.functions.utils.io.ConnectorUtils; import org.apache.pulsar.io.core.BatchSource; import org.apache.pulsar.io.core.Source; @@ -294,7 +294,7 @@ public static SourceConfig convertFromDetails(FunctionDetails functionDetails) { } public static ExtractedSourceDetails validateAndExtractDetails(SourceConfig sourceConfig, - ClassLoader sourceClassLoader, + ValidatableFunctionPackage sourceFunction, boolean validateConnectorConfig) { if (isEmpty(sourceConfig.getTenant())) { throw new IllegalArgumentException("Source tenant cannot be null"); @@ -319,29 +319,34 @@ public static ExtractedSourceDetails validateAndExtractDetails(SourceConfig sour // if class name in source config is not set, this should be a built-in source // thus we should try to find it class name in the NAR service definition if (sourceClassName == null) { - try { - sourceClassName = ConnectorUtils.getIOSourceClass((NarClassLoader) sourceClassLoader); - } catch (IOException e) { - throw new IllegalArgumentException("Failed to extract source class from archive", e); + ConnectorDefinition connectorDefinition = sourceFunction.getFunctionMetaData(ConnectorDefinition.class); + if (connectorDefinition == null) { + throw new IllegalArgumentException( + "Source package doesn't contain the META-INF/services/pulsar-io.yaml file."); + } + sourceClassName = connectorDefinition.getSourceClass(); + if (sourceClassName == null) { + throw new IllegalArgumentException("Failed to extract source class from archive"); } } // check if source implements the correct interfaces - Class sourceClass; + TypeDescription sourceClass; try { - sourceClass = sourceClassLoader.loadClass(sourceClassName); - } catch (ClassNotFoundException e) { + sourceClass = sourceFunction.resolveType(sourceClassName); + } catch (TypePool.Resolution.NoSuchTypeException e) { throw new IllegalArgumentException( String.format("Source class %s not found in class loader", sourceClassName), e); } - if (!Source.class.isAssignableFrom(sourceClass) && !BatchSource.class.isAssignableFrom(sourceClass)) { + if (!(sourceClass.asErasure().isAssignableTo(Source.class) || sourceClass.asErasure() + .isAssignableTo(BatchSource.class))) { throw new IllegalArgumentException( - String.format("Source class %s does not implement the correct interface", - sourceClass.getName())); + String.format("Source class %s does not implement the correct interface", + sourceClass.getName())); } - if (BatchSource.class.isAssignableFrom(sourceClass)) { + if (sourceClass.asErasure().isAssignableTo(BatchSource.class)) { if (sourceConfig.getBatchSourceConfig() != null) { validateBatchSourceConfig(sourceConfig.getBatchSourceConfig()); } else { @@ -352,7 +357,14 @@ public static ExtractedSourceDetails validateAndExtractDetails(SourceConfig sour } // extract type from source class - Class typeArg = getSourceType(sourceClass); + TypeDefinition typeArg; + + try { + typeArg = getSourceType(sourceClass); + } catch (Exception e) { + throw new IllegalArgumentException( + String.format("Failed to resolve type for Source class %s", sourceClassName), e); + } // Only one of serdeClassName or schemaType should be set if (!StringUtils.isEmpty(sourceConfig.getSerdeClassName()) && !StringUtils @@ -361,29 +373,30 @@ public static ExtractedSourceDetails validateAndExtractDetails(SourceConfig sour } if (!StringUtils.isEmpty(sourceConfig.getSerdeClassName())) { - ValidatorUtils.validateSerde(sourceConfig.getSerdeClassName(), typeArg, sourceClassLoader, false); + ValidatorUtils.validateSerde(sourceConfig.getSerdeClassName(), typeArg, sourceFunction.getTypePool(), + false); } if (!StringUtils.isEmpty(sourceConfig.getSchemaType())) { - ValidatorUtils.validateSchema(sourceConfig.getSchemaType(), typeArg, sourceClassLoader, false); + ValidatorUtils.validateSchema(sourceConfig.getSchemaType(), typeArg, sourceFunction.getTypePool(), + false); } if (sourceConfig.getProducerConfig() != null && sourceConfig.getProducerConfig().getCryptoConfig() != null) { ValidatorUtils - .validateCryptoKeyReader(sourceConfig.getProducerConfig().getCryptoConfig(), sourceClassLoader, - true); + .validateCryptoKeyReader(sourceConfig.getProducerConfig().getCryptoConfig(), + sourceFunction.getTypePool(), true); } - if (typeArg.equals(TypeResolver.Unknown.class)) { - throw new IllegalArgumentException( - String.format("Failed to resolve type for Source class %s", sourceClassName)); - } - - // validate user defined config if enabled and source is loaded from NAR - if (validateConnectorConfig && sourceClassLoader instanceof NarClassLoader) { - validateSourceConfig(sourceConfig, (NarClassLoader) sourceClassLoader); + // validate user defined config if enabled and classloading is enabled + if (validateConnectorConfig) { + if (sourceFunction.isEnableClassloading()) { + validateSourceConfig(sourceConfig, sourceFunction); + } else { + log.warn("Skipping annotation based validation of sink config as classloading is disabled"); + } } - return new ExtractedSourceDetails(sourceClassName, typeArg.getName()); + return new ExtractedSourceDetails(sourceClassName, typeArg.asErasure().getTypeName()); } @SneakyThrows @@ -524,15 +537,14 @@ public static void validateBatchSourceConfigUpdate(BatchSourceConfig existingCon } } - public static void validateSourceConfig(SourceConfig sourceConfig, NarClassLoader narClassLoader) { + public static void validateSourceConfig(SourceConfig sourceConfig, ValidatableFunctionPackage sourceFunction) { try { - ConnectorDefinition defn = ConnectorUtils.getConnectorDefinition(narClassLoader); - if (defn.getSourceConfigClass() != null) { - Class configClass = Class.forName(defn.getSourceConfigClass(), true, narClassLoader); + ConnectorDefinition defn = sourceFunction.getFunctionMetaData(ConnectorDefinition.class); + if (defn != null && defn.getSourceConfigClass() != null) { + Class configClass = + Class.forName(defn.getSourceConfigClass(), true, sourceFunction.getClassLoader()); validateSourceConfig(sourceConfig, configClass); } - } catch (IOException e) { - throw new IllegalArgumentException("Error validating source config", e); } catch (ClassNotFoundException e) { throw new IllegalArgumentException("Could not find source config class"); } diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/ValidatableFunctionPackage.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/ValidatableFunctionPackage.java new file mode 100644 index 0000000000000..8d5aefb6f6785 --- /dev/null +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/ValidatableFunctionPackage.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.functions.utils; + +import net.bytebuddy.description.type.TypeDescription; +import net.bytebuddy.pool.TypePool; + +/** + * This abstraction separates the function and connector definition from classloading, + * enabling validation without the need for classloading. It utilizes Byte Buddy for + * type and annotation resolution. + * + * The function or connector definition is directly extracted from the archive file, + * eliminating the need for classloader initialization. + * + * The getClassLoader method should only be invoked when classloading is enabled. + * Classloading is required in the LocalRunner and in the Functions worker when the + * worker is configured with the 'validateConnectorConfig' set to true. + */ +public interface ValidatableFunctionPackage { + /** + * Resolves the type description for the given class name within the function package. + */ + TypeDescription resolveType(String className); + /** + * Returns the Byte Buddy TypePool instance for the function package. + */ + TypePool getTypePool(); + /** + * Returns the function or connector definition metadata. + * Supports FunctionDefinition and ConnectorDefinition as the metadata type. + */ + T getFunctionMetaData(Class clazz); + /** + * Returns if classloading is enabled for the function package. + */ + boolean isEnableClassloading(); + /** + * Returns the classloader for the function package. The classloader is + * lazily initialized when classloading is enabled. + */ + ClassLoader getClassLoader(); +} diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/ValidatorUtils.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/ValidatorUtils.java index 390671c5606af..8df6a3f261a6e 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/ValidatorUtils.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/ValidatorUtils.java @@ -18,35 +18,40 @@ */ package org.apache.pulsar.functions.utils; -import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.commons.lang3.StringUtils.isEmpty; -import static org.apache.commons.lang3.StringUtils.isNotBlank; import java.util.Map; import lombok.extern.slf4j.Slf4j; -import net.jodah.typetools.TypeResolver; +import net.bytebuddy.description.type.TypeDefinition; +import net.bytebuddy.description.type.TypeDescription; +import net.bytebuddy.pool.TypePool; import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.functions.CryptoConfig; import org.apache.pulsar.common.schema.SchemaType; -import org.apache.pulsar.common.util.ClassLoaderUtils; -import org.apache.pulsar.common.util.Reflections; import org.apache.pulsar.functions.api.SerDe; -import org.apache.pulsar.functions.proto.Function; -import org.apache.pulsar.io.core.Sink; -import org.apache.pulsar.io.core.Source; @Slf4j public class ValidatorUtils { private static final String DEFAULT_SERDE = "org.apache.pulsar.functions.api.utils.DefaultSerDe"; - public static void validateSchema(String schemaType, Class typeArg, ClassLoader clsLoader, + public static void validateSchema(String schemaType, TypeDefinition typeArg, TypePool typePool, boolean input) { if (isEmpty(schemaType) || getBuiltinSchemaType(schemaType) != null) { // If it's empty, we use the default schema and no need to validate // If it's built-in, no need to validate } else { - ClassLoaderUtils.implementsClass(schemaType, Schema.class, clsLoader); - validateSchemaType(schemaType, typeArg, clsLoader, input); + TypeDescription schemaClass = null; + try { + schemaClass = typePool.describe(schemaType).resolve(); + } catch (TypePool.Resolution.NoSuchTypeException e) { + throw new IllegalArgumentException( + String.format("The schema class %s does not exist", schemaType)); + } + if (!schemaClass.asErasure().isAssignableTo(Schema.class)) { + throw new IllegalArgumentException( + String.format("%s does not implement %s", schemaType, Schema.class.getName())); + } + validateSchemaType(schemaClass, typeArg, typePool, input); } } @@ -60,29 +65,32 @@ private static SchemaType getBuiltinSchemaType(String schemaTypeOrClassName) { } - public static void validateCryptoKeyReader(CryptoConfig conf, ClassLoader classLoader, boolean isProducer) { + public static void validateCryptoKeyReader(CryptoConfig conf, TypePool typePool, boolean isProducer) { if (isEmpty(conf.getCryptoKeyReaderClassName())) { return; } - Class cryptoClass; + String cryptoClassName = conf.getCryptoKeyReaderClassName(); + TypeDescription cryptoClass = null; try { - cryptoClass = ClassLoaderUtils.loadClass(conf.getCryptoKeyReaderClassName(), classLoader); - } catch (ClassNotFoundException | NoClassDefFoundError e) { + cryptoClass = typePool.describe(cryptoClassName).resolve(); + } catch (TypePool.Resolution.NoSuchTypeException e) { throw new IllegalArgumentException( - String.format("The crypto key reader class %s does not exist", conf.getCryptoKeyReaderClassName())); + String.format("The crypto key reader class %s does not exist", cryptoClassName)); + } + if (!cryptoClass.asErasure().isAssignableTo(CryptoKeyReader.class)) { + throw new IllegalArgumentException( + String.format("%s does not implement %s", cryptoClassName, CryptoKeyReader.class.getName())); } - ClassLoaderUtils.implementsClass(conf.getCryptoKeyReaderClassName(), CryptoKeyReader.class, classLoader); - try { - cryptoClass.getConstructor(Map.class); - } catch (NoSuchMethodException ex) { + boolean hasConstructor = cryptoClass.getDeclaredMethods().stream() + .anyMatch(method -> method.isConstructor() && method.getParameters().size() == 1 + && method.getParameters().get(0).getType().asErasure().represents(Map.class)); + + if (!hasConstructor) { throw new IllegalArgumentException( String.format("The crypto key reader class %s does not implement the desired constructor.", conf.getCryptoKeyReaderClassName())); - - } catch (SecurityException e) { - throw new IllegalArgumentException("Failed to access crypto key reader class", e); } if (isProducer && (conf.getEncryptionKeys() == null || conf.getEncryptionKeys().length == 0)) { @@ -90,7 +98,7 @@ public static void validateCryptoKeyReader(CryptoConfig conf, ClassLoader classL } } - public static void validateSerde(String inputSerializer, Class typeArg, ClassLoader clsLoader, + public static void validateSerde(String inputSerializer, TypeDefinition typeArg, TypePool typePool, boolean deser) { if (isEmpty(inputSerializer)) { return; @@ -98,154 +106,53 @@ public static void validateSerde(String inputSerializer, Class typeArg, Class if (inputSerializer.equals(DEFAULT_SERDE)) { return; } + TypeDescription serdeClass; try { - Class serdeClass = ClassLoaderUtils.loadClass(inputSerializer, clsLoader); - } catch (ClassNotFoundException | NoClassDefFoundError e) { + serdeClass = typePool.describe(inputSerializer).resolve(); + } catch (TypePool.Resolution.NoSuchTypeException e) { throw new IllegalArgumentException( String.format("The input serialization/deserialization class %s does not exist", inputSerializer)); } - ClassLoaderUtils.implementsClass(inputSerializer, SerDe.class, clsLoader); - - SerDe serDe = (SerDe) Reflections.createInstance(inputSerializer, clsLoader); - if (serDe == null) { - throw new IllegalArgumentException(String.format("The SerDe class %s does not exist", - inputSerializer)); - } - Class[] serDeTypes = TypeResolver.resolveRawArguments(SerDe.class, serDe.getClass()); - - // type inheritance information seems to be lost in generic type - // load the actual type class for verification - Class fnInputClass; - Class serdeInputClass; - try { - fnInputClass = Class.forName(typeArg.getName(), true, clsLoader); - serdeInputClass = Class.forName(serDeTypes[0].getName(), true, clsLoader); - } catch (ClassNotFoundException | NoClassDefFoundError e) { - throw new IllegalArgumentException("Failed to load type class", e); - } + TypeDescription.Generic serDeTypeArg = serdeClass.getInterfaces().stream() + .filter(i -> i.asErasure().isAssignableTo(SerDe.class)) + .findFirst() + .map(i -> i.getTypeArguments().get(0)) + .orElseThrow(() -> new IllegalArgumentException( + String.format("%s does not implement %s", inputSerializer, SerDe.class.getName()))); if (deser) { - if (!fnInputClass.isAssignableFrom(serdeInputClass)) { - throw new IllegalArgumentException("Serializer type mismatch " + typeArg + " vs " + serDeTypes[0]); + if (!serDeTypeArg.asErasure().isAssignableTo(typeArg.asErasure())) { + throw new IllegalArgumentException("Serializer type mismatch " + typeArg.getActualName() + " vs " + + serDeTypeArg.getActualName()); } } else { - if (!serdeInputClass.isAssignableFrom(fnInputClass)) { - throw new IllegalArgumentException("Serializer type mismatch " + typeArg + " vs " + serDeTypes[0]); + if (!serDeTypeArg.asErasure().isAssignableFrom(typeArg.asErasure())) { + throw new IllegalArgumentException("Serializer type mismatch " + typeArg.getActualName() + " vs " + + serDeTypeArg.getActualName()); } } } - private static void validateSchemaType(String schemaClassName, Class typeArg, ClassLoader clsLoader, + private static void validateSchemaType(TypeDefinition schema, TypeDefinition typeArg, TypePool typePool, boolean input) { - Schema schema = (Schema) Reflections.createInstance(schemaClassName, clsLoader); - if (schema == null) { - throw new IllegalArgumentException(String.format("The Schema class %s does not exist", - schemaClassName)); - } - Class[] schemaTypes = TypeResolver.resolveRawArguments(Schema.class, schema.getClass()); - // type inheritance information seems to be lost in generic type - // load the actual type class for verification - Class fnInputClass; - Class schemaInputClass; - try { - fnInputClass = Class.forName(typeArg.getName(), true, clsLoader); - schemaInputClass = Class.forName(schemaTypes[0].getName(), true, clsLoader); - } catch (ClassNotFoundException | NoClassDefFoundError e) { - throw new IllegalArgumentException("Failed to load type class", e); - } + TypeDescription.Generic schemaTypeArg = schema.getInterfaces().stream() + .filter(i -> i.asErasure().isAssignableTo(Schema.class)) + .findFirst() + .map(i -> i.getTypeArguments().get(0)) + .orElse(null); if (input) { - if (!fnInputClass.isAssignableFrom(schemaInputClass)) { + if (!schemaTypeArg.asErasure().isAssignableTo(typeArg.asErasure())) { throw new IllegalArgumentException( - "Schema type mismatch " + typeArg + " vs " + schemaTypes[0]); + "Schema type mismatch " + typeArg.getActualName() + " vs " + schemaTypeArg.getActualName()); } } else { - if (!schemaInputClass.isAssignableFrom(fnInputClass)) { + if (!schemaTypeArg.asErasure().isAssignableFrom(typeArg.asErasure())) { throw new IllegalArgumentException( - "Schema type mismatch " + typeArg + " vs " + schemaTypes[0]); - } - } - } - - - public static void validateFunctionClassTypes(ClassLoader classLoader, - Function.FunctionDetails.Builder functionDetailsBuilder) { - - // validate only if classLoader is provided - if (classLoader == null) { - return; - } - - if (isBlank(functionDetailsBuilder.getClassName())) { - throw new IllegalArgumentException("Function class-name can't be empty"); - } - - // validate function class-type - Class functionClass; - try { - functionClass = classLoader.loadClass(functionDetailsBuilder.getClassName()); - } catch (ClassNotFoundException | NoClassDefFoundError e) { - throw new IllegalArgumentException( - String.format("Function class %s must be in class path", functionDetailsBuilder.getClassName()), e); - } - Class[] typeArgs = FunctionCommon.getFunctionTypes(functionClass, false); - - if (!(org.apache.pulsar.functions.api.Function.class.isAssignableFrom(functionClass)) - && !(java.util.function.Function.class.isAssignableFrom(functionClass))) { - throw new RuntimeException("User class must either be Function or java.util.Function"); - } - - if (functionDetailsBuilder.hasSource() && functionDetailsBuilder.getSource() != null - && isNotBlank(functionDetailsBuilder.getSource().getClassName())) { - try { - String sourceClassName = functionDetailsBuilder.getSource().getClassName(); - String argClassName = FunctionCommon.getTypeArg(sourceClassName, Source.class, classLoader).getName(); - functionDetailsBuilder - .setSource(functionDetailsBuilder.getSourceBuilder().setTypeClassName(argClassName)); - - // if sink-class not present then set same arg as source - if (!functionDetailsBuilder.hasSink() || isBlank(functionDetailsBuilder.getSink().getClassName())) { - functionDetailsBuilder - .setSink(functionDetailsBuilder.getSinkBuilder().setTypeClassName(argClassName)); - } - - } catch (IllegalArgumentException ie) { - throw ie; - } catch (Exception e) { - log.error("Failed to validate source class", e); - throw new IllegalArgumentException("Failed to validate source class-name", e); - } - } else if (isBlank(functionDetailsBuilder.getSourceBuilder().getTypeClassName())) { - // if function-src-class is not present then set function-src type-class according to function class - functionDetailsBuilder - .setSource(functionDetailsBuilder.getSourceBuilder().setTypeClassName(typeArgs[0].getName())); - } - - if (functionDetailsBuilder.hasSink() && functionDetailsBuilder.getSink() != null - && isNotBlank(functionDetailsBuilder.getSink().getClassName())) { - try { - String sinkClassName = functionDetailsBuilder.getSink().getClassName(); - String argClassName = FunctionCommon.getTypeArg(sinkClassName, Sink.class, classLoader).getName(); - functionDetailsBuilder.setSink(functionDetailsBuilder.getSinkBuilder().setTypeClassName(argClassName)); - - // if source-class not present then set same arg as sink - if (!functionDetailsBuilder.hasSource() || isBlank(functionDetailsBuilder.getSource().getClassName())) { - functionDetailsBuilder - .setSource(functionDetailsBuilder.getSourceBuilder().setTypeClassName(argClassName)); - } - - } catch (IllegalArgumentException ie) { - throw ie; - } catch (Exception e) { - log.error("Failed to validate sink class", e); - throw new IllegalArgumentException("Failed to validate sink class-name", e); + "Schema type mismatch " + typeArg.getActualName() + " vs " + schemaTypeArg.getActualName()); } - } else if (isBlank(functionDetailsBuilder.getSinkBuilder().getTypeClassName())) { - // if function-sink-class is not present then set function-sink type-class according to function class - functionDetailsBuilder - .setSink(functionDetailsBuilder.getSinkBuilder().setTypeClassName(typeArgs[1].getName())); } } } diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/functions/FunctionArchive.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/functions/FunctionArchive.java index 028b57d69c86b..cfb213f34ed72 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/functions/FunctionArchive.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/functions/FunctionArchive.java @@ -19,14 +19,50 @@ package org.apache.pulsar.functions.utils.functions; import java.nio.file.Path; -import lombok.Builder; -import lombok.Data; import org.apache.pulsar.common.functions.FunctionDefinition; +import org.apache.pulsar.functions.utils.FunctionFilePackage; +import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; -@Builder -@Data -public class FunctionArchive { - private Path archivePath; - private ClassLoader classLoader; - private FunctionDefinition functionDefinition; +public class FunctionArchive implements AutoCloseable { + private final Path archivePath; + private final FunctionDefinition functionDefinition; + private final String narExtractionDirectory; + private final boolean enableClassloading; + private ValidatableFunctionPackage functionPackage; + private boolean closed; + + public FunctionArchive(Path archivePath, FunctionDefinition functionDefinition, String narExtractionDirectory, + boolean enableClassloading) { + this.archivePath = archivePath; + this.functionDefinition = functionDefinition; + this.narExtractionDirectory = narExtractionDirectory; + this.enableClassloading = enableClassloading; + } + + public Path getArchivePath() { + return archivePath; + } + + public synchronized ValidatableFunctionPackage getFunctionPackage() { + if (closed) { + throw new IllegalStateException("FunctionArchive is already closed"); + } + if (functionPackage == null) { + functionPackage = new FunctionFilePackage(archivePath.toFile(), narExtractionDirectory, enableClassloading, + FunctionDefinition.class); + } + return functionPackage; + } + + public FunctionDefinition getFunctionDefinition() { + return functionDefinition; + } + + @Override + public synchronized void close() throws Exception { + closed = true; + if (functionPackage instanceof AutoCloseable) { + ((AutoCloseable) functionPackage).close(); + } + } } 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 941df573e495e..31a5540e0bfaf 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 @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.pulsar.functions.utils.functions; import java.io.File; @@ -30,10 +31,8 @@ import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.common.functions.FunctionDefinition; import org.apache.pulsar.common.nar.NarClassLoader; -import org.apache.pulsar.common.nar.NarClassLoaderBuilder; import org.apache.pulsar.common.util.ObjectMapperFactory; -import org.apache.pulsar.functions.api.Function; -import org.apache.pulsar.functions.utils.Exceptions; +import org.zeroturnaround.zip.ZipUtil; @UtilityClass @@ -45,43 +44,40 @@ public class FunctionUtils { /** * Extract the Pulsar Function class from a function or archive. */ - public static String getFunctionClass(ClassLoader classLoader) throws IOException { - NarClassLoader ncl = (NarClassLoader) classLoader; - String configStr = ncl.getServiceDefinition(PULSAR_IO_SERVICE_NAME); - - FunctionDefinition conf = ObjectMapperFactory.getYamlMapper().reader().readValue(configStr, - FunctionDefinition.class); - if (StringUtils.isEmpty(conf.getFunctionClass())) { - throw new IOException( - String.format("The '%s' functionctor does not provide a function implementation", conf.getName())); - } + public static String getFunctionClass(File narFile) throws IOException { + return getFunctionDefinition(narFile).getFunctionClass(); + } - try { - // Try to load source class and check it implements Function interface - Class functionClass = ncl.loadClass(conf.getFunctionClass()); - if (!(Function.class.isAssignableFrom(functionClass))) { - throw new IOException( - "Class " + conf.getFunctionClass() + " does not implement interface " + Function.class - .getName()); - } - } catch (Throwable t) { - Exceptions.rethrowIOException(t); + public static FunctionDefinition getFunctionDefinition(File narFile) throws IOException { + return getPulsarIOServiceConfig(narFile, FunctionDefinition.class); + } + + public static T getPulsarIOServiceConfig(File narFile, Class valueType) throws IOException { + String filename = "META-INF/services/" + PULSAR_IO_SERVICE_NAME; + byte[] configEntry = ZipUtil.unpackEntry(narFile, filename); + if (configEntry != null) { + return ObjectMapperFactory.getYamlMapper().reader().readValue(configEntry, valueType); + } else { + return null; } + } - return conf.getFunctionClass(); + public static String getFunctionClass(NarClassLoader narClassLoader) throws IOException { + return getFunctionDefinition(narClassLoader).getFunctionClass(); } public static FunctionDefinition getFunctionDefinition(NarClassLoader narClassLoader) throws IOException { - String configStr = narClassLoader.getServiceDefinition(PULSAR_IO_SERVICE_NAME); - return ObjectMapperFactory.getYamlMapper().reader().readValue(configStr, FunctionDefinition.class); + return getPulsarIOServiceConfig(narClassLoader, FunctionDefinition.class); } - public static TreeMap searchForFunctions(String functionsDirectory) throws IOException { - return searchForFunctions(functionsDirectory, false); + public static T getPulsarIOServiceConfig(NarClassLoader narClassLoader, Class valueType) throws IOException { + return ObjectMapperFactory.getYamlMapper().reader() + .readValue(narClassLoader.getServiceDefinition(PULSAR_IO_SERVICE_NAME), valueType); } public static TreeMap searchForFunctions(String functionsDirectory, - boolean alwaysPopulatePath) throws IOException { + String narExtractionDirectory, + boolean enableClassloading) throws IOException { Path path = Paths.get(functionsDirectory).toAbsolutePath(); log.info("Searching for functions in {}", path); @@ -95,22 +91,12 @@ public static TreeMap searchForFunctions(String functio try (DirectoryStream stream = Files.newDirectoryStream(path, "*.nar")) { for (Path archive : stream) { try { - - NarClassLoader ncl = NarClassLoaderBuilder.builder() - .narFile(new File(archive.toString())) - .build(); - - FunctionArchive.FunctionArchiveBuilder functionArchiveBuilder = FunctionArchive.builder(); - FunctionDefinition cntDef = FunctionUtils.getFunctionDefinition(ncl); + FunctionDefinition cntDef = FunctionUtils.getFunctionDefinition(archive.toFile()); log.info("Found function {} from {}", cntDef, archive); - - functionArchiveBuilder.archivePath(archive); - - functionArchiveBuilder.classLoader(ncl); - functionArchiveBuilder.functionDefinition(cntDef); - - if (alwaysPopulatePath || !StringUtils.isEmpty(cntDef.getFunctionClass())) { - functions.put(cntDef.getName(), functionArchiveBuilder.build()); + if (!StringUtils.isEmpty(cntDef.getFunctionClass())) { + FunctionArchive functionArchive = + new FunctionArchive(archive, cntDef, narExtractionDirectory, enableClassloading); + functions.put(cntDef.getName(), functionArchive); } } catch (Throwable t) { log.warn("Failed to load function from {}", archive, t); diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/io/Connector.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/io/Connector.java index f1a03f4424ec6..5fcc22747c516 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/io/Connector.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/io/Connector.java @@ -20,17 +20,79 @@ import java.nio.file.Path; import java.util.List; -import lombok.Builder; -import lombok.Data; +import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.common.io.ConfigFieldDefinition; import org.apache.pulsar.common.io.ConnectorDefinition; +import org.apache.pulsar.functions.utils.FunctionFilePackage; +import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; -@Builder -@Data -public class Connector { - private Path archivePath; +public class Connector implements AutoCloseable { + private final Path archivePath; + private final String narExtractionDirectory; + private final boolean enableClassloading; + private ValidatableFunctionPackage connectorFunctionPackage; private List sourceConfigFieldDefinitions; private List sinkConfigFieldDefinitions; - private ClassLoader classLoader; private ConnectorDefinition connectorDefinition; + private boolean closed; + + public Connector(Path archivePath, ConnectorDefinition connectorDefinition, String narExtractionDirectory, + boolean enableClassloading) { + this.archivePath = archivePath; + this.connectorDefinition = connectorDefinition; + this.narExtractionDirectory = narExtractionDirectory; + this.enableClassloading = enableClassloading; + } + + public Path getArchivePath() { + return archivePath; + } + + public synchronized ValidatableFunctionPackage getConnectorFunctionPackage() { + checkState(); + if (connectorFunctionPackage == null) { + connectorFunctionPackage = + new FunctionFilePackage(archivePath.toFile(), narExtractionDirectory, enableClassloading, + ConnectorDefinition.class); + } + return connectorFunctionPackage; + } + + private void checkState() { + if (closed) { + throw new IllegalStateException("Connector is already closed"); + } + } + + public synchronized List getSourceConfigFieldDefinitions() { + checkState(); + if (sourceConfigFieldDefinitions == null && !StringUtils.isEmpty(connectorDefinition.getSourceClass()) + && !StringUtils.isEmpty(connectorDefinition.getSourceConfigClass())) { + sourceConfigFieldDefinitions = ConnectorUtils.getConnectorConfigDefinition(getConnectorFunctionPackage(), + connectorDefinition.getSourceConfigClass()); + } + return sourceConfigFieldDefinitions; + } + + public synchronized List getSinkConfigFieldDefinitions() { + checkState(); + if (sinkConfigFieldDefinitions == null && !StringUtils.isEmpty(connectorDefinition.getSinkClass()) + && !StringUtils.isEmpty(connectorDefinition.getSinkConfigClass())) { + sinkConfigFieldDefinitions = ConnectorUtils.getConnectorConfigDefinition(getConnectorFunctionPackage(), + connectorDefinition.getSinkConfigClass()); + } + return sinkConfigFieldDefinitions; + } + + public ConnectorDefinition getConnectorDefinition() { + return connectorDefinition; + } + + @Override + public synchronized void close() throws Exception { + closed = true; + if (connectorFunctionPackage instanceof AutoCloseable) { + ((AutoCloseable) connectorFunctionPackage).close(); + } + } } 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 a814bf35548f3..df1310965f392 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 @@ -18,38 +18,31 @@ */ package org.apache.pulsar.functions.utils.io; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import java.io.File; import java.io.IOException; -import java.lang.annotation.Annotation; -import java.lang.reflect.Field; -import java.lang.reflect.Method; import java.nio.file.DirectoryStream; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; -import java.util.AbstractMap; -import java.util.ArrayList; import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.TreeMap; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.stream.Collectors; import lombok.experimental.UtilityClass; import lombok.extern.slf4j.Slf4j; +import net.bytebuddy.description.annotation.AnnotationDescription; +import net.bytebuddy.description.annotation.AnnotationValue; +import net.bytebuddy.description.field.FieldDescription; +import net.bytebuddy.description.method.MethodDescription; +import net.bytebuddy.description.type.TypeDefinition; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.common.io.ConfigFieldDefinition; import org.apache.pulsar.common.io.ConnectorDefinition; import org.apache.pulsar.common.nar.NarClassLoader; -import org.apache.pulsar.common.nar.NarClassLoaderBuilder; -import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.ObjectMapperFactory; -import org.apache.pulsar.common.util.Reflections; import org.apache.pulsar.functions.utils.Exceptions; +import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; +import org.apache.pulsar.functions.utils.functions.FunctionUtils; import org.apache.pulsar.io.core.BatchSource; import org.apache.pulsar.io.core.Sink; import org.apache.pulsar.io.core.Source; @@ -76,7 +69,7 @@ public static String getIOSourceClass(NarClassLoader narClassLoader) throws IOEx Class sourceClass = narClassLoader.loadClass(conf.getSourceClass()); if (!(Source.class.isAssignableFrom(sourceClass) || BatchSource.class.isAssignableFrom(sourceClass))) { throw new IOException(String.format("Class %s does not implement interface %s or %s", - conf.getSourceClass(), Source.class.getName(), BatchSource.class.getName())); + conf.getSourceClass(), Source.class.getName(), BatchSource.class.getName())); } } catch (Throwable t) { Exceptions.rethrowIOException(t); @@ -109,32 +102,36 @@ public static String getIOSinkClass(NarClassLoader narClassLoader) throws IOExce return conf.getSinkClass(); } - public static ConnectorDefinition getConnectorDefinition(NarClassLoader narClassLoader) throws IOException { - String configStr = narClassLoader.getServiceDefinition(PULSAR_IO_SERVICE_NAME); + public static ConnectorDefinition getConnectorDefinition(File narFile) throws IOException { + return FunctionUtils.getPulsarIOServiceConfig(narFile, ConnectorDefinition.class); + } - return ObjectMapperFactory.getYamlMapper().reader().readValue(configStr, ConnectorDefinition.class); + public static ConnectorDefinition getConnectorDefinition(NarClassLoader narClassLoader) throws IOException { + return FunctionUtils.getPulsarIOServiceConfig(narClassLoader, ConnectorDefinition.class); } - public static List getConnectorConfigDefinition(ClassLoader classLoader, - String configClassName) throws Exception { + public static List getConnectorConfigDefinition( + ValidatableFunctionPackage connectorFunctionPackage, + String configClassName) { List retval = new LinkedList<>(); - Class configClass = classLoader.loadClass(configClassName); - for (Field field : Reflections.getAllFields(configClass)) { - if (java.lang.reflect.Modifier.isStatic(field.getModifiers())) { - // We dont want static fields + TypeDefinition configClass = connectorFunctionPackage.resolveType(configClassName); + + for (FieldDescription field : getAllFields(configClass)) { + if (field.isStatic()) { + // We don't want static fields continue; } - field.setAccessible(true); ConfigFieldDefinition configFieldDefinition = new ConfigFieldDefinition(); configFieldDefinition.setFieldName(field.getName()); - configFieldDefinition.setTypeName(field.getType().getName()); + configFieldDefinition.setTypeName(field.getType().getActualName()); Map attributes = new HashMap<>(); - for (Annotation annotation : field.getAnnotations()) { - if (annotation.annotationType().equals(FieldDoc.class)) { - FieldDoc fieldDoc = (FieldDoc) annotation; - for (Method method : FieldDoc.class.getDeclaredMethods()) { - Object value = method.invoke(fieldDoc); - attributes.put(method.getName(), value == null ? "" : value.toString()); + for (AnnotationDescription annotation : field.getDeclaredAnnotations()) { + if (annotation.getAnnotationType().represents(FieldDoc.class)) { + for (MethodDescription.InDefinedShape method : annotation.getAnnotationType() + .getDeclaredMethods()) { + AnnotationValue value = annotation.getValue(method.getName()); + attributes.put(method.getName(), + value == null || value.resolve() == null ? "" : value.resolve().toString()); } } } @@ -145,86 +142,42 @@ public static List getConnectorConfigDefinition(ClassLoad return retval; } + private static List getAllFields(TypeDefinition type) { + List fields = new LinkedList<>(); + fields.addAll(type.getDeclaredFields()); + + if (type.getSuperClass() != null) { + fields.addAll(getAllFields(type.getSuperClass())); + } + + return fields; + } + public static TreeMap searchForConnectors(String connectorsDirectory, - String narExtractionDirectory) throws IOException { + String narExtractionDirectory, + boolean enableClassloading) throws IOException { Path path = Paths.get(connectorsDirectory).toAbsolutePath(); log.info("Searching for connectors in {}", path); + TreeMap connectors = new TreeMap<>(); + if (!path.toFile().exists()) { log.warn("Connectors archive directory not found"); - return new TreeMap<>(); + return connectors; } - List archives = new ArrayList<>(); try (DirectoryStream stream = Files.newDirectoryStream(path, "*.nar")) { for (Path archive : stream) { - archives.add(archive); - } - } - if (archives.isEmpty()) { - return new TreeMap<>(); - } - - ExecutorService oneTimeExecutor = null; - try { - int nThreads = Math.min(Runtime.getRuntime().availableProcessors(), archives.size()); - log.info("Loading {} connector definitions with a thread pool of size {}", archives.size(), nThreads); - oneTimeExecutor = Executors.newFixedThreadPool(nThreads, - new ThreadFactoryBuilder().setNameFormat("connector-extraction-executor-%d").build()); - List>> futures = new ArrayList<>(); - for (Path archive : archives) { - CompletableFuture> future = CompletableFuture.supplyAsync(() -> - getConnectorDefinitionEntry(archive, narExtractionDirectory), oneTimeExecutor); - futures.add(future); - } - - FutureUtil.waitForAll(futures).join(); - return futures.stream() - .map(CompletableFuture::join) - .filter(entry -> entry != null) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue, (a, b) -> a, TreeMap::new)); - } finally { - if (oneTimeExecutor != null) { - oneTimeExecutor.shutdown(); - } - } - } - - private static Map.Entry getConnectorDefinitionEntry(Path archive, - String narExtractionDirectory) { - try { - - NarClassLoader ncl = NarClassLoaderBuilder.builder() - .narFile(new File(archive.toString())) - .extractionDirectory(narExtractionDirectory) - .build(); - - Connector.ConnectorBuilder connectorBuilder = Connector.builder(); - ConnectorDefinition cntDef = ConnectorUtils.getConnectorDefinition(ncl); - log.info("Found connector {} from {}", cntDef, archive); - - connectorBuilder.archivePath(archive); - if (!StringUtils.isEmpty(cntDef.getSourceClass())) { - if (!StringUtils.isEmpty(cntDef.getSourceConfigClass())) { - connectorBuilder.sourceConfigFieldDefinitions( - ConnectorUtils.getConnectorConfigDefinition(ncl, - cntDef.getSourceConfigClass())); + try { + ConnectorDefinition cntDef = ConnectorUtils.getConnectorDefinition(archive.toFile()); + log.info("Found connector {} from {}", cntDef, archive); + Connector connector = new Connector(archive, cntDef, narExtractionDirectory, enableClassloading); + connectors.put(cntDef.getName(), connector); + } catch (Throwable t) { + log.warn("Failed to load connector from {}", archive, t); } } - - if (!StringUtils.isEmpty(cntDef.getSinkClass())) { - if (!StringUtils.isEmpty(cntDef.getSinkConfigClass())) { - connectorBuilder.sinkConfigFieldDefinitions( - ConnectorUtils.getConnectorConfigDefinition(ncl, cntDef.getSinkConfigClass())); - } - } - - connectorBuilder.classLoader(ncl); - connectorBuilder.connectorDefinition(cntDef); - return new AbstractMap.SimpleEntry(cntDef.getName(), connectorBuilder.build()); - } catch (Throwable t) { - log.warn("Failed to load connector from {}", archive, t); - return null; } + return connectors; } } diff --git a/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionCommonTest.java b/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionCommonTest.java index 131f153b08d68..90fdd4da777d3 100644 --- a/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionCommonTest.java +++ b/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionCommonTest.java @@ -30,16 +30,17 @@ import com.github.tomakehurst.wiremock.WireMockServer; import java.io.File; import java.util.Collection; +import java.util.concurrent.CompletableFuture; import lombok.Cleanup; +import net.bytebuddy.description.type.TypeDefinition; +import net.bytebuddy.pool.TypePool; import org.apache.pulsar.client.impl.MessageIdImpl; -import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.functions.api.Context; import org.apache.pulsar.functions.api.Function; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.functions.api.WindowContext; import org.apache.pulsar.functions.api.WindowFunction; import org.assertj.core.util.Files; -import org.testng.Assert; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -47,41 +48,6 @@ * Unit test of {@link Exceptions}. */ public class FunctionCommonTest { - - @Test - public void testValidateLocalFileUrl() throws Exception { - String fileLocation = FutureUtil.class.getProtectionDomain().getCodeSource().getLocation().getPath(); - try { - // eg: fileLocation : /dir/fileName.jar (invalid) - FunctionCommon.extractClassLoader(fileLocation); - Assert.fail("should fail with invalid url: without protocol"); - } catch (IllegalArgumentException ie) { - // Ok.. expected exception - } - String fileLocationWithProtocol = "file://" + fileLocation; - // eg: fileLocation : file:///dir/fileName.jar (valid) - FunctionCommon.extractClassLoader(fileLocationWithProtocol); - // eg: fileLocation : file:/dir/fileName.jar (valid) - fileLocationWithProtocol = "file:" + fileLocation; - FunctionCommon.extractClassLoader(fileLocationWithProtocol); - } - - @Test - public void testValidateHttpFileUrl() throws Exception { - - String jarHttpUrl = "https://repo1.maven.org/maven2/org/apache/pulsar/pulsar-common/2.4.2/pulsar-common-2.4.2.jar"; - FunctionCommon.extractClassLoader(jarHttpUrl); - - jarHttpUrl = "http://_invalidurl_.com"; - try { - // eg: fileLocation : /dir/fileName.jar (invalid) - FunctionCommon.extractClassLoader(jarHttpUrl); - Assert.fail("should fail with invalid url: without protocol"); - } catch (Exception ie) { - // Ok.. expected exception - } - } - @Test public void testDownloadFile() throws Exception { final String jarHttpUrl = "https://repo1.maven.org/maven2/org/apache/pulsar/pulsar-common/2.4.2/pulsar-common-2.4.2.jar"; @@ -150,6 +116,14 @@ public Record process(String input, Context context) throws Exception { } }, false }, + { + new Function>() { + @Override + public CompletableFuture process(String input, Context context) throws Exception { + return null; + } + }, false + }, { new java.util.function.Function() { @Override @@ -166,6 +140,14 @@ public Record apply(String s) { } }, false }, + { + new java.util.function.Function>() { + @Override + public CompletableFuture apply(String s) { + return null; + } + }, false + }, { new WindowFunction() { @Override @@ -182,6 +164,14 @@ public Record process(Collection> input, WindowContext c } }, true }, + { + new WindowFunction>() { + @Override + public CompletableFuture process(Collection> input, WindowContext context) throws Exception { + return null; + } + }, true + }, { new java.util.function.Function, Integer>() { @Override @@ -197,15 +187,26 @@ public Record apply(Collection strings) { return null; } }, true + }, + { + new java.util.function.Function, CompletableFuture>() { + @Override + public CompletableFuture apply(Collection strings) { + return null; + } + }, true } }; } @Test(dataProvider = "function") public void testGetFunctionTypes(Object function, boolean isWindowConfigPresent) { - Class[] types = FunctionCommon.getFunctionTypes(function.getClass(), isWindowConfigPresent); + TypePool typePool = TypePool.Default.of(function.getClass().getClassLoader()); + TypeDefinition[] types = + FunctionCommon.getFunctionTypes(typePool.describe(function.getClass().getName()).resolve(), + isWindowConfigPresent); assertEquals(types.length, 2); - assertEquals(types[0], String.class); - assertEquals(types[1], Integer.class); + assertEquals(types[0].asErasure().getTypeName(), String.class.getName()); + assertEquals(types[1].asErasure().getTypeName(), Integer.class.getName()); } } diff --git a/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java b/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java index 8f46199e8ffd5..954eef44a7366 100644 --- a/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java +++ b/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java @@ -18,13 +18,24 @@ */ package org.apache.pulsar.functions.utils; +import static org.apache.pulsar.common.functions.FunctionConfig.ProcessingGuarantees.EFFECTIVELY_ONCE; +import static org.apache.pulsar.common.functions.FunctionConfig.Runtime.PYTHON; +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 com.google.gson.Gson; - -import org.apache.pulsar.client.api.CompressionType; -import org.apache.pulsar.client.api.SubscriptionInitialPosition; import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.util.JsonFormat; +import java.lang.reflect.Field; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.client.api.CompressionType; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.impl.schema.JSONSchema; import org.apache.pulsar.common.functions.ConsumerConfig; import org.apache.pulsar.common.functions.FunctionConfig; @@ -32,28 +43,29 @@ import org.apache.pulsar.common.functions.Resources; import org.apache.pulsar.common.functions.WindowConfig; import org.apache.pulsar.common.util.Reflections; +import org.apache.pulsar.functions.api.Record; +import org.apache.pulsar.functions.api.WindowContext; +import org.apache.pulsar.functions.api.WindowFunction; import org.apache.pulsar.functions.api.utils.IdentityFunction; import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.proto.Function.FunctionDetails; import org.testng.annotations.Test; -import java.lang.reflect.Field; -import java.util.HashMap; -import java.util.Map; - -import static org.apache.pulsar.common.functions.FunctionConfig.ProcessingGuarantees.EFFECTIVELY_ONCE; -import static org.apache.pulsar.common.functions.FunctionConfig.Runtime.PYTHON; -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; - /** * Unit test of {@link Reflections}. */ @Slf4j public class FunctionConfigUtilsTest { + public static class WordCountWindowFunction implements WindowFunction { + @Override + public Void process(Collection> inputs, WindowContext context) throws Exception { + for (Record input : inputs) { + Arrays.asList(input.getValue().split("\\.")).forEach(word -> context.incrCounter(word, 1)); + } + return null; + } + } + @Test public void testAutoAckConvertFailed() { @@ -63,7 +75,7 @@ public void testAutoAckConvertFailed() { functionConfig.setProcessingGuarantees(FunctionConfig.ProcessingGuarantees.ATMOST_ONCE); assertThrows(IllegalArgumentException.class, () -> { - FunctionConfigUtils.convert(functionConfig, (ClassLoader) null); + FunctionConfigUtils.convert(functionConfig); }); } @@ -99,7 +111,7 @@ public void testConvertBackFidelity() { producerConfig.setBatchBuilder("DEFAULT"); producerConfig.setCompressionType(CompressionType.ZLIB); functionConfig.setProducerConfig(producerConfig); - Function.FunctionDetails functionDetails = FunctionConfigUtils.convert(functionConfig, (ClassLoader) null); + Function.FunctionDetails functionDetails = FunctionConfigUtils.convert(functionConfig); FunctionConfig convertedConfig = FunctionConfigUtils.convertFromDetails(functionDetails); // add default resources @@ -119,7 +131,7 @@ public void testConvertWindow() { functionConfig.setNamespace("test-namespace"); functionConfig.setName("test-function"); functionConfig.setParallelism(1); - functionConfig.setClassName(IdentityFunction.class.getName()); + functionConfig.setClassName(WordCountWindowFunction.class.getName()); Map inputSpecs = new HashMap<>(); inputSpecs.put("test-input", ConsumerConfig.builder().isRegexPattern(true).serdeClassName("test-serde").build()); functionConfig.setInputSpecs(inputSpecs); @@ -141,7 +153,7 @@ public void testConvertWindow() { producerConfig.setBatchBuilder("KEY_BASED"); producerConfig.setCompressionType(CompressionType.SNAPPY); functionConfig.setProducerConfig(producerConfig); - Function.FunctionDetails functionDetails = FunctionConfigUtils.convert(functionConfig, (ClassLoader) null); + Function.FunctionDetails functionDetails = FunctionConfigUtils.convert(functionConfig); FunctionConfig convertedConfig = FunctionConfigUtils.convertFromDetails(functionDetails); // WindowsFunction guarantees convert to FunctionGuarantees. @@ -163,7 +175,7 @@ public void testConvertBatchBuilder() { FunctionConfig functionConfig = createFunctionConfig(); functionConfig.setBatchBuilder("KEY_BASED"); - Function.FunctionDetails functionDetails = FunctionConfigUtils.convert(functionConfig, (ClassLoader) null); + Function.FunctionDetails functionDetails = FunctionConfigUtils.convert(functionConfig); assertEquals(functionDetails.getSink().getProducerSpec().getBatchBuilder(), "KEY_BASED"); FunctionConfig convertedConfig = FunctionConfigUtils.convertFromDetails(functionDetails); @@ -519,7 +531,6 @@ private FunctionConfig createFunctionConfig() { functionConfig.setUserConfig(new HashMap<>()); functionConfig.setAutoAck(true); functionConfig.setTimeoutMs(2000L); - functionConfig.setWindowConfig(new WindowConfig().setWindowLengthCount(10)); functionConfig.setCleanupSubscription(true); functionConfig.setRuntimeFlags("-Dfoo=bar"); return functionConfig; @@ -553,7 +564,7 @@ public void testDisableForwardSourceMessageProperty() throws InvalidProtocolBuff config.setForwardSourceMessageProperty(true); FunctionConfigUtils.inferMissingArguments(config, false); assertNull(config.getForwardSourceMessageProperty()); - FunctionDetails details = FunctionConfigUtils.convert(config, FunctionConfigUtilsTest.class.getClassLoader()); + FunctionDetails details = FunctionConfigUtils.convert(config); assertFalse(details.getSink().getForwardSourceMessageProperty()); String detailsJson = "'" + JsonFormat.printer().omittingInsignificantWhitespace().print(details) + "'"; log.info("Function details : {}", detailsJson); @@ -640,7 +651,7 @@ public void testMergeDifferentOutputSchemaTypes() { @Test public void testPoolMessages() { FunctionConfig functionConfig = createFunctionConfig(); - Function.FunctionDetails functionDetails = FunctionConfigUtils.convert(functionConfig, (ClassLoader) null); + Function.FunctionDetails functionDetails = FunctionConfigUtils.convert(functionConfig); assertFalse(functionDetails.getSource().getInputSpecsMap().get("test-input").getPoolMessages()); FunctionConfig convertedConfig = FunctionConfigUtils.convertFromDetails(functionDetails); assertFalse(convertedConfig.getInputSpecs().get("test-input").isPoolMessages()); @@ -650,7 +661,7 @@ public void testPoolMessages() { .poolMessages(true).build()); functionConfig.setInputSpecs(inputSpecs); - functionDetails = FunctionConfigUtils.convert(functionConfig, (ClassLoader) null); + functionDetails = FunctionConfigUtils.convert(functionConfig); assertTrue(functionDetails.getSource().getInputSpecsMap().get("test-input").getPoolMessages()); convertedConfig = FunctionConfigUtils.convertFromDetails(functionDetails); 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 8ac9b61e3f60f..14cd77f60ff95 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 @@ -18,37 +18,38 @@ */ package org.apache.pulsar.functions.utils; +import static org.apache.pulsar.common.functions.FunctionConfig.ProcessingGuarantees.ATLEAST_ONCE; +import static org.apache.pulsar.common.functions.FunctionConfig.ProcessingGuarantees.ATMOST_ONCE; +import static org.apache.pulsar.common.functions.FunctionConfig.ProcessingGuarantees.EFFECTIVELY_ONCE; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertThrows; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.expectThrows; import com.google.common.collect.Lists; import com.google.gson.Gson; +import java.io.IOException; +import java.lang.reflect.Field; import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import lombok.Data; import lombok.NoArgsConstructor; import lombok.experimental.Accessors; import org.apache.pulsar.common.functions.ConsumerConfig; import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.functions.Resources; +import org.apache.pulsar.common.io.ConnectorDefinition; import org.apache.pulsar.common.io.SinkConfig; import org.apache.pulsar.config.validation.ConfigValidationAnnotations; -import org.apache.pulsar.functions.api.utils.IdentityFunction; +import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.functions.proto.Function; +import org.apache.pulsar.io.core.Sink; +import org.apache.pulsar.io.core.SinkContext; import org.testng.annotations.Test; -import java.io.IOException; -import java.lang.reflect.Field; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.apache.pulsar.common.functions.FunctionConfig.ProcessingGuarantees.ATLEAST_ONCE; -import static org.apache.pulsar.common.functions.FunctionConfig.ProcessingGuarantees.ATMOST_ONCE; -import static org.apache.pulsar.common.functions.FunctionConfig.ProcessingGuarantees.EFFECTIVELY_ONCE; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertThrows; -import static org.testng.Assert.assertTrue; -import static org.testng.Assert.expectThrows; - /** * Unit test of {@link SinkConfigUtilsTest}. */ @@ -62,6 +63,27 @@ public static class TestSinkConfig { private String configParameter; } + + public static class NopSink implements Sink { + + @Override + public void open(Map config, SinkContext sinkContext) throws Exception { + + } + + @Override + public void write(Record record) throws Exception { + + } + + @Override + public void close() throws Exception { + + } + } + + + @Test public void testAutoAckConvertFailed() throws IOException { @@ -521,7 +543,7 @@ private SinkConfig createSinkConfig() { sinkConfig.setNamespace("test-namespace"); sinkConfig.setName("test-sink"); sinkConfig.setParallelism(1); - sinkConfig.setClassName(IdentityFunction.class.getName()); + sinkConfig.setClassName(NopSink.class.getName()); Map inputSpecs = new HashMap<>(); inputSpecs.put("test-input", ConsumerConfig.builder().isRegexPattern(true).serdeClassName("test-serde").build()); sinkConfig.setInputSpecs(inputSpecs); @@ -577,13 +599,16 @@ public void testAllowDisableSinkTimeout() { SinkConfig sinkConfig = createSinkConfig(); sinkConfig.setInputSpecs(null); sinkConfig.setTopicsPattern("my-topic-*"); - SinkConfigUtils.validateAndExtractDetails(sinkConfig, this.getClass().getClassLoader(), null, + LoadedFunctionPackage validatableFunction = + new LoadedFunctionPackage(this.getClass().getClassLoader(), ConnectorDefinition.class); + + SinkConfigUtils.validateAndExtractDetails(sinkConfig, validatableFunction, null, true); sinkConfig.setTimeoutMs(null); - SinkConfigUtils.validateAndExtractDetails(sinkConfig, this.getClass().getClassLoader(), null, + SinkConfigUtils.validateAndExtractDetails(sinkConfig, validatableFunction, null, true); sinkConfig.setTimeoutMs(0L); - SinkConfigUtils.validateAndExtractDetails(sinkConfig, this.getClass().getClassLoader(), null, + SinkConfigUtils.validateAndExtractDetails(sinkConfig, validatableFunction, null, true); } } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java index 03c6eb7921840..250a7cc4c7bd4 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java @@ -70,6 +70,7 @@ import org.apache.pulsar.functions.utils.Actions; import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.utils.SourceConfigUtils; +import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; import org.apache.pulsar.functions.utils.io.Connector; @Data @@ -527,7 +528,7 @@ private File getBuiltinArchive(FunctionDetails.ComponentType componentType, Func builder.setClassName(sourceClass); functionDetails.setSource(builder); - fillSourceTypeClass(functionDetails, connector.getClassLoader(), sourceClass); + fillSourceTypeClass(functionDetails, connector.getConnectorFunctionPackage(), sourceClass); return archive; } } @@ -543,7 +544,7 @@ private File getBuiltinArchive(FunctionDetails.ComponentType componentType, Func builder.setClassName(sinkClass); functionDetails.setSink(builder); - fillSinkTypeClass(functionDetails, connector.getClassLoader(), sinkClass); + fillSinkTypeClass(functionDetails, connector.getConnectorFunctionPackage(), sinkClass); return archive; } } @@ -557,8 +558,8 @@ private File getBuiltinArchive(FunctionDetails.ComponentType componentType, Func } private void fillSourceTypeClass(FunctionDetails.Builder functionDetails, - ClassLoader narClassLoader, String className) throws ClassNotFoundException { - String typeArg = getSourceType(className, narClassLoader).getName(); + ValidatableFunctionPackage functionPackage, String className) { + String typeArg = getSourceType(className, functionPackage.getTypePool()).asErasure().getName(); SourceSpec.Builder sourceBuilder = SourceSpec.newBuilder(functionDetails.getSource()); sourceBuilder.setTypeClassName(typeArg); @@ -573,8 +574,8 @@ private void fillSourceTypeClass(FunctionDetails.Builder functionDetails, } private void fillSinkTypeClass(FunctionDetails.Builder functionDetails, - ClassLoader narClassLoader, String className) throws ClassNotFoundException { - String typeArg = getSinkType(className, narClassLoader).getName(); + ValidatableFunctionPackage functionPackage, String className) { + String typeArg = getSinkType(className, functionPackage.getTypePool()).asErasure().getName(); SinkSpec.Builder sinkBuilder = SinkSpec.newBuilder(functionDetails.getSink()); sinkBuilder.setTypeClassName(typeArg); diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java index 8a9ea22c53015..f9f2738828be7 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java @@ -658,6 +658,14 @@ public void stop() { if (statsUpdater != null) { statsUpdater.shutdownNow(); } + + if (null != functionsManager) { + functionsManager.close(); + } + + if (null != connectorsManager) { + connectorsManager.close(); + } } } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java index 585b54d846169..f80bc31e7f96c 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java @@ -100,6 +100,7 @@ import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.utils.FunctionConfigUtils; import org.apache.pulsar.functions.utils.FunctionMetaDataUtils; +import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; import org.apache.pulsar.functions.utils.functions.FunctionArchive; import org.apache.pulsar.functions.utils.io.Connector; import org.apache.pulsar.functions.worker.FunctionMetaDataManager; @@ -1788,12 +1789,6 @@ private void internalProcessFunctionRequest(final String tenant, final String na } } - protected ClassLoader getClassLoaderFromPackage(String className, - File packageFile, - String narExtractionDirectory) { - return FunctionCommon.getClassLoaderFromPackage(componentType, className, packageFile, narExtractionDirectory); - } - static File downloadPackageFile(PulsarWorkerService worker, String packageName) throws IOException, PulsarAdminException { Path tempDirectory; @@ -1861,7 +1856,7 @@ protected File getPackageFile(String functionPkgUrl) throws IOException, PulsarA } } - protected ClassLoader getBuiltinFunctionClassLoader(String archive) { + protected ValidatableFunctionPackage getBuiltinFunctionPackage(String archive) { if (!StringUtils.isEmpty(archive)) { if (archive.startsWith(org.apache.pulsar.common.functions.Utils.BUILTIN)) { archive = archive.replaceFirst("^builtin://", ""); @@ -1871,7 +1866,7 @@ protected ClassLoader getBuiltinFunctionClassLoader(String archive) { if (function == null) { throw new IllegalArgumentException("Built-in " + componentType + " is not available"); } - return function.getClassLoader(); + return function.getFunctionPackage(); } } return null; diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java index 078c47524f8e9..6b81d2c4918a6 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java @@ -47,7 +47,6 @@ import org.apache.pulsar.common.functions.WorkerInfo; import org.apache.pulsar.common.policies.data.ExceptionInformation; import org.apache.pulsar.common.policies.data.FunctionStatus; -import org.apache.pulsar.common.util.ClassLoaderUtils; import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.functions.auth.FunctionAuthData; import org.apache.pulsar.functions.instance.InstanceUtils; @@ -55,11 +54,14 @@ import org.apache.pulsar.functions.proto.InstanceCommunication; import org.apache.pulsar.functions.utils.ComponentTypeUtils; import org.apache.pulsar.functions.utils.FunctionConfigUtils; +import org.apache.pulsar.functions.utils.FunctionFilePackage; import org.apache.pulsar.functions.utils.FunctionMetaDataUtils; +import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; import org.apache.pulsar.functions.utils.functions.FunctionArchive; import org.apache.pulsar.functions.worker.FunctionMetaDataManager; import org.apache.pulsar.functions.worker.FunctionsManager; import org.apache.pulsar.functions.worker.PulsarWorkerService; +import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.functions.worker.service.api.Functions; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; @@ -732,11 +734,12 @@ private Function.FunctionDetails validateUpdateRequestParams(final String tenant functionConfig.setTenant(tenant); functionConfig.setNamespace(namespace); functionConfig.setName(componentName); + WorkerConfig workerConfig = worker().getWorkerConfig(); FunctionConfigUtils.inferMissingArguments( - functionConfig, worker().getWorkerConfig().isForwardSourceMessageProperty()); + functionConfig, workerConfig.isForwardSourceMessageProperty()); String archive = functionConfig.getJar(); - ClassLoader classLoader = null; + ValidatableFunctionPackage functionPackage = null; // check if function is builtin and extract classloader if (!StringUtils.isEmpty(archive)) { if (archive.startsWith(org.apache.pulsar.common.functions.Utils.BUILTIN)) { @@ -749,35 +752,38 @@ private Function.FunctionDetails validateUpdateRequestParams(final String tenant if (function == null) { throw new IllegalArgumentException(String.format("No Function %s found", archive)); } - classLoader = function.getClassLoader(); + functionPackage = function.getFunctionPackage(); } } - boolean shouldCloseClassLoader = false; + boolean shouldCloseFunctionPackage = false; try { - if (functionConfig.getRuntime() == FunctionConfig.Runtime.JAVA) { // if function is not builtin, attempt to extract classloader from package file if it exists - if (classLoader == null && componentPackageFile != null) { - classLoader = getClassLoaderFromPackage(functionConfig.getClassName(), - componentPackageFile, worker().getWorkerConfig().getNarExtractionDirectory()); - shouldCloseClassLoader = true; + if (functionPackage == null && componentPackageFile != null) { + functionPackage = + new FunctionFilePackage(componentPackageFile, workerConfig.getNarExtractionDirectory(), + workerConfig.getEnableClassloadingOfExternalFiles(), FunctionDefinition.class); + shouldCloseFunctionPackage = true; } - if (classLoader == null) { + if (functionPackage == null) { throw new IllegalArgumentException("Function package is not provided"); } FunctionConfigUtils.ExtractedFunctionDetails functionDetails = FunctionConfigUtils.validateJavaFunction( - functionConfig, classLoader); + functionConfig, functionPackage); return FunctionConfigUtils.convert(functionConfig, functionDetails); } else { - classLoader = FunctionConfigUtils.validate(functionConfig, componentPackageFile); - shouldCloseClassLoader = true; - return FunctionConfigUtils.convert(functionConfig, classLoader); + FunctionConfigUtils.validateNonJavaFunction(functionConfig); + return FunctionConfigUtils.convert(functionConfig); } } finally { - if (shouldCloseClassLoader) { - ClassLoaderUtils.closeClassLoader(classLoader); + if (shouldCloseFunctionPackage && functionPackage instanceof AutoCloseable) { + try { + ((AutoCloseable) functionPackage).close(); + } catch (Exception e) { + log.error("Failed to close function file", e); + } } } } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java index c382ec9e01b35..51d1333a79c36 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java @@ -47,18 +47,20 @@ import org.apache.pulsar.common.io.SinkConfig; import org.apache.pulsar.common.policies.data.ExceptionInformation; import org.apache.pulsar.common.policies.data.SinkStatus; -import org.apache.pulsar.common.util.ClassLoaderUtils; import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.functions.auth.FunctionAuthData; import org.apache.pulsar.functions.instance.InstanceUtils; import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.proto.InstanceCommunication; import org.apache.pulsar.functions.utils.ComponentTypeUtils; +import org.apache.pulsar.functions.utils.FunctionFilePackage; import org.apache.pulsar.functions.utils.FunctionMetaDataUtils; import org.apache.pulsar.functions.utils.SinkConfigUtils; +import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; import org.apache.pulsar.functions.utils.io.Connector; import org.apache.pulsar.functions.worker.FunctionMetaDataManager; import org.apache.pulsar.functions.worker.PulsarWorkerService; +import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.functions.worker.service.api.Sinks; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; @@ -704,7 +706,7 @@ private Function.FunctionDetails validateUpdateRequestParams(final String tenant sinkConfig.setName(sinkName); org.apache.pulsar.common.functions.Utils.inferMissingArguments(sinkConfig); - ClassLoader classLoader = null; + ValidatableFunctionPackage connectorFunctionPackage = null; // check if sink is builtin and extract classloader if (!StringUtils.isEmpty(sinkConfig.getArchive())) { String archive = sinkConfig.getArchive(); @@ -716,45 +718,62 @@ private Function.FunctionDetails validateUpdateRequestParams(final String tenant if (connector == null) { throw new IllegalArgumentException("Built-in sink is not available"); } - classLoader = connector.getClassLoader(); + connectorFunctionPackage = connector.getConnectorFunctionPackage(); } } - boolean shouldCloseClassLoader = false; + boolean shouldCloseFunctionPackage = false; + ValidatableFunctionPackage transformFunctionPackage = null; + boolean shouldCloseTransformFunctionPackage = false; try { // if sink is not builtin, attempt to extract classloader from package file if it exists - if (classLoader == null && sinkPackageFile != null) { - classLoader = getClassLoaderFromPackage(sinkConfig.getClassName(), - sinkPackageFile, worker().getWorkerConfig().getNarExtractionDirectory()); - shouldCloseClassLoader = true; + WorkerConfig workerConfig = worker().getWorkerConfig(); + if (connectorFunctionPackage == null && sinkPackageFile != null) { + connectorFunctionPackage = + new FunctionFilePackage(sinkPackageFile, workerConfig.getNarExtractionDirectory(), + workerConfig.getEnableClassloadingOfExternalFiles(), ConnectorDefinition.class); + shouldCloseFunctionPackage = true; } - if (classLoader == null) { + if (connectorFunctionPackage == null) { throw new IllegalArgumentException("Sink package is not provided"); } - ClassLoader functionClassLoader = null; if (isNotBlank(sinkConfig.getTransformFunction())) { - functionClassLoader = - getBuiltinFunctionClassLoader(sinkConfig.getTransformFunction()); - if (functionClassLoader == null) { + transformFunctionPackage = + getBuiltinFunctionPackage(sinkConfig.getTransformFunction()); + if (transformFunctionPackage == null) { File functionPackageFile = getPackageFile(sinkConfig.getTransformFunction()); - functionClassLoader = getClassLoaderFromPackage(sinkConfig.getTransformFunctionClassName(), - functionPackageFile, worker().getWorkerConfig().getNarExtractionDirectory()); + transformFunctionPackage = + new FunctionFilePackage(functionPackageFile, workerConfig.getNarExtractionDirectory(), + workerConfig.getEnableClassloadingOfExternalFiles(), ConnectorDefinition.class); + shouldCloseTransformFunctionPackage = true; } - if (functionClassLoader == null) { + if (transformFunctionPackage == null) { throw new IllegalArgumentException("Transform Function package not found"); } } - SinkConfigUtils.ExtractedSinkDetails sinkDetails = SinkConfigUtils.validateAndExtractDetails( - sinkConfig, classLoader, functionClassLoader, worker().getWorkerConfig().getValidateConnectorConfig()); + SinkConfigUtils.ExtractedSinkDetails sinkDetails = + SinkConfigUtils.validateAndExtractDetails(sinkConfig, connectorFunctionPackage, + transformFunctionPackage, workerConfig.getValidateConnectorConfig()); return SinkConfigUtils.convert(sinkConfig, sinkDetails); } finally { - if (shouldCloseClassLoader) { - ClassLoaderUtils.closeClassLoader(classLoader); + if (shouldCloseFunctionPackage && connectorFunctionPackage instanceof AutoCloseable) { + try { + ((AutoCloseable) connectorFunctionPackage).close(); + } catch (Exception e) { + log.error("Failed to connector function file", e); + } + } + if (shouldCloseTransformFunctionPackage && transformFunctionPackage instanceof AutoCloseable) { + try { + ((AutoCloseable) transformFunctionPackage).close(); + } catch (Exception e) { + log.error("Failed to close transform function file", e); + } } } } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java index c55ddf48b06b9..dea69698dd28d 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java @@ -46,18 +46,20 @@ import org.apache.pulsar.common.io.SourceConfig; import org.apache.pulsar.common.policies.data.ExceptionInformation; import org.apache.pulsar.common.policies.data.SourceStatus; -import org.apache.pulsar.common.util.ClassLoaderUtils; import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.functions.auth.FunctionAuthData; import org.apache.pulsar.functions.instance.InstanceUtils; import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.proto.InstanceCommunication; import org.apache.pulsar.functions.utils.ComponentTypeUtils; +import org.apache.pulsar.functions.utils.FunctionFilePackage; import org.apache.pulsar.functions.utils.FunctionMetaDataUtils; import org.apache.pulsar.functions.utils.SourceConfigUtils; +import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; import org.apache.pulsar.functions.utils.io.Connector; import org.apache.pulsar.functions.worker.FunctionMetaDataManager; import org.apache.pulsar.functions.worker.PulsarWorkerService; +import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.functions.worker.service.api.Sources; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; @@ -663,7 +665,7 @@ private Function.FunctionDetails validateUpdateRequestParams(final String tenant sourceConfig.setName(sourceName); org.apache.pulsar.common.functions.Utils.inferMissingArguments(sourceConfig); - ClassLoader classLoader = null; + ValidatableFunctionPackage connectorFunctionPackage = null; // check if source is builtin and extract classloader if (!StringUtils.isEmpty(sourceConfig.getArchive())) { String archive = sourceConfig.getArchive(); @@ -675,30 +677,37 @@ private Function.FunctionDetails validateUpdateRequestParams(final String tenant if (connector == null) { throw new IllegalArgumentException("Built-in source is not available"); } - classLoader = connector.getClassLoader(); + connectorFunctionPackage = connector.getConnectorFunctionPackage(); } } - boolean shouldCloseClassLoader = false; + boolean shouldCloseFunctionPackage = false; try { // if source is not builtin, attempt to extract classloader from package file if it exists - if (classLoader == null && sourcePackageFile != null) { - classLoader = getClassLoaderFromPackage(sourceConfig.getClassName(), - sourcePackageFile, worker().getWorkerConfig().getNarExtractionDirectory()); - shouldCloseClassLoader = true; + WorkerConfig workerConfig = worker().getWorkerConfig(); + if (connectorFunctionPackage == null && sourcePackageFile != null) { + connectorFunctionPackage = + new FunctionFilePackage(sourcePackageFile, workerConfig.getNarExtractionDirectory(), + workerConfig.getEnableClassloadingOfExternalFiles(), ConnectorDefinition.class); + shouldCloseFunctionPackage = true; } - if (classLoader == null) { + if (connectorFunctionPackage == null) { throw new IllegalArgumentException("Source package is not provided"); } SourceConfigUtils.ExtractedSourceDetails sourceDetails = SourceConfigUtils.validateAndExtractDetails( - sourceConfig, classLoader, worker().getWorkerConfig().getValidateConnectorConfig()); + sourceConfig, connectorFunctionPackage, + workerConfig.getValidateConnectorConfig()); return SourceConfigUtils.convert(sourceConfig, sourceDetails); } finally { - if (shouldCloseClassLoader) { - ClassLoaderUtils.closeClassLoader(classLoader); + if (shouldCloseFunctionPackage && connectorFunctionPackage instanceof AutoCloseable) { + try { + ((AutoCloseable) connectorFunctionPackage).close(); + } catch (Exception e) { + log.error("Failed to connector function file", e); + } } } } diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplTest.java index cfe087c78406a..73a229893e5d2 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImplTest.java @@ -381,6 +381,6 @@ public static FunctionConfig createDefaultFunctionConfig() { public static Function.FunctionDetails createDefaultFunctionDetails() { FunctionConfig functionConfig = createDefaultFunctionConfig(); - return FunctionConfigUtils.convert(functionConfig, (ClassLoader) null); + return FunctionConfigUtils.convert(functionConfig); } } diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v2/FunctionApiV2ResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v2/FunctionApiV2ResourceTest.java index 32a104c576993..a8415919c119b 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v2/FunctionApiV2ResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v2/FunctionApiV2ResourceTest.java @@ -18,1125 +18,82 @@ */ package org.apache.pulsar.functions.worker.rest.api.v2; - -import static java.nio.file.StandardCopyOption.REPLACE_EXISTING; import static org.apache.pulsar.functions.utils.FunctionCommon.mergeJson; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyBoolean; -import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; -import com.google.common.collect.Lists; import com.google.gson.Gson; -import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.util.JsonFormat; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.net.URL; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.HashMap; -import java.util.LinkedList; import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.function.Consumer; import javax.ws.rs.core.Response; import javax.ws.rs.core.StreamingOutput; -import org.apache.distributedlog.api.namespace.Namespace; -import org.apache.logging.log4j.Level; -import org.apache.logging.log4j.core.config.Configurator; import org.apache.pulsar.broker.authentication.AuthenticationParameters; -import org.apache.pulsar.client.admin.Functions; -import org.apache.pulsar.client.admin.Namespaces; -import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.client.admin.Tenants; import org.apache.pulsar.common.functions.FunctionConfig; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.functions.UpdateOptionsImpl; import org.apache.pulsar.common.util.RestException; -import org.apache.pulsar.functions.api.Context; -import org.apache.pulsar.functions.api.Function; -import org.apache.pulsar.functions.instance.InstanceUtils; -import org.apache.pulsar.functions.proto.Function.FunctionDetails; -import org.apache.pulsar.functions.proto.Function.FunctionMetaData; -import org.apache.pulsar.functions.proto.Function.PackageLocationMetaData; -import org.apache.pulsar.functions.proto.Function.ProcessingGuarantees; -import org.apache.pulsar.functions.proto.Function.SinkSpec; -import org.apache.pulsar.functions.proto.Function.SourceSpec; -import org.apache.pulsar.functions.proto.Function.SubscriptionType; -import org.apache.pulsar.functions.runtime.RuntimeFactory; -import org.apache.pulsar.functions.source.TopicSchema; -import org.apache.pulsar.functions.utils.FunctionCommon; +import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.utils.FunctionConfigUtils; -import org.apache.pulsar.functions.worker.FunctionMetaDataManager; -import org.apache.pulsar.functions.worker.FunctionRuntimeManager; -import org.apache.pulsar.functions.worker.LeaderService; -import org.apache.pulsar.functions.worker.PulsarWorkerService; -import org.apache.pulsar.functions.worker.WorkerConfig; -import org.apache.pulsar.functions.worker.WorkerUtils; -import org.apache.pulsar.functions.worker.rest.api.FunctionsImpl; import org.apache.pulsar.functions.worker.rest.api.FunctionsImplV2; -import org.apache.pulsar.functions.worker.rest.api.PulsarFunctionTestTemporaryDirectory; +import org.apache.pulsar.functions.worker.rest.api.v3.AbstractFunctionApiResourceTest; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; -import org.mockito.MockedStatic; -import org.mockito.Mockito; -import org.testng.Assert; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -/** - * Unit test of {@link FunctionsApiV2Resource}. - */ -public class FunctionApiV2ResourceTest { - - private static final class TestFunction implements Function { - - @Override - public String process(String input, Context context) { - return input; - } - } - - private static final String tenant = "test-tenant"; - private static final String namespace = "test-namespace"; - private static final String function = "test-function"; - private static final String outputTopic = "test-output-topic"; - private static final String outputSerdeClassName = TopicSchema.DEFAULT_SERDE; - private static final String className = TestFunction.class.getName(); - private SubscriptionType subscriptionType = SubscriptionType.FAILOVER; - private static final Map topicsToSerDeClassName = new HashMap<>(); - static { - topicsToSerDeClassName.put("persistent://public/default/test_src", TopicSchema.DEFAULT_SERDE); - } - private static final int parallelism = 1; - - private PulsarWorkerService mockedWorkerService; - private PulsarAdmin mockedPulsarAdmin; - private Tenants mockedTenants; - private Namespaces mockedNamespaces; - private Functions mockedFunctions; - private TenantInfoImpl mockedTenantInfo; - private List namespaceList = new LinkedList<>(); - private FunctionMetaDataManager mockedManager; - private FunctionRuntimeManager mockedFunctionRunTimeManager; - private RuntimeFactory mockedRuntimeFactory; - private Namespace mockedNamespace; +public class FunctionApiV2ResourceTest extends AbstractFunctionApiResourceTest { private FunctionsImplV2 resource; - private InputStream mockedInputStream; - private FormDataContentDisposition mockedFormData; - private FunctionMetaData mockedFunctionMetadata; - private LeaderService mockedLeaderService; - private PulsarFunctionTestTemporaryDirectory tempDirectory; - private static Map mockStaticContexts = new HashMap<>(); - - @BeforeMethod - public void setup() throws Exception { - this.mockedManager = mock(FunctionMetaDataManager.class); - this.mockedFunctionRunTimeManager = mock(FunctionRuntimeManager.class); - this.mockedTenantInfo = mock(TenantInfoImpl.class); - this.mockedRuntimeFactory = mock(RuntimeFactory.class); - this.mockedInputStream = mock(InputStream.class); - this.mockedNamespace = mock(Namespace.class); - this.mockedFormData = mock(FormDataContentDisposition.class); - when(mockedFormData.getFileName()).thenReturn("test"); - this.mockedPulsarAdmin = mock(PulsarAdmin.class); - this.mockedTenants = mock(Tenants.class); - this.mockedNamespaces = mock(Namespaces.class); - this.mockedFunctions = mock(Functions.class); - this.mockedLeaderService = mock(LeaderService.class); - this.mockedFunctionMetadata = FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); - namespaceList.add(tenant + "/" + namespace); - - this.mockedWorkerService = mock(PulsarWorkerService.class); - when(mockedWorkerService.getFunctionMetaDataManager()).thenReturn(mockedManager); - when(mockedWorkerService.getLeaderService()).thenReturn(mockedLeaderService); - when(mockedWorkerService.getFunctionRuntimeManager()).thenReturn(mockedFunctionRunTimeManager); - when(mockedFunctionRunTimeManager.getRuntimeFactory()).thenReturn(mockedRuntimeFactory); - when(mockedWorkerService.getDlogNamespace()).thenReturn(mockedNamespace); - when(mockedWorkerService.isInitialized()).thenReturn(true); - when(mockedWorkerService.getBrokerAdmin()).thenReturn(mockedPulsarAdmin); - when(mockedWorkerService.getFunctionAdmin()).thenReturn(mockedPulsarAdmin); - when(mockedPulsarAdmin.tenants()).thenReturn(mockedTenants); - when(mockedPulsarAdmin.namespaces()).thenReturn(mockedNamespaces); - when(mockedPulsarAdmin.functions()).thenReturn(mockedFunctions); - when(mockedTenants.getTenantInfo(any())).thenReturn(mockedTenantInfo); - when(mockedNamespaces.getNamespaces(any())).thenReturn(namespaceList); - when(mockedLeaderService.isLeader()).thenReturn(true); - when(mockedManager.getFunctionMetaData(any(), any(), any())).thenReturn(mockedFunctionMetadata); - - // worker config - WorkerConfig workerConfig = new WorkerConfig() - .setWorkerId("test") - .setWorkerPort(8080) - .setFunctionMetadataTopicName("pulsar/functions") - .setNumFunctionPackageReplicas(3) - .setPulsarServiceUrl("pulsar://localhost:6650/"); - tempDirectory = PulsarFunctionTestTemporaryDirectory.create(getClass().getSimpleName()); - tempDirectory.useTemporaryDirectoriesForWorkerConfig(workerConfig); - when(mockedWorkerService.getWorkerConfig()).thenReturn(workerConfig); - - FunctionsImpl functions = spy(new FunctionsImpl(() -> mockedWorkerService)); - - this.resource = spy(new FunctionsImplV2(functions)); - - } - - @AfterMethod(alwaysRun = true) - public void cleanup() { - if (tempDirectory != null) { - tempDirectory.delete(); - } - mockStaticContexts.values().forEach(MockedStatic::close); - mockStaticContexts.clear(); - } - - private void mockStatic(Class classStatic, Consumer> consumer) { - final MockedStatic mockedStatic = - mockStaticContexts.computeIfAbsent(classStatic.getName(), name -> Mockito.mockStatic(classStatic)); - consumer.accept(mockedStatic); - } - - private void mockWorkerUtils() { - mockStatic(WorkerUtils.class, ctx -> { - ctx.when(() -> WorkerUtils.dumpToTmpFile(any())).thenCallRealMethod(); - }); - } - - private void mockWorkerUtils(Consumer> consumer) { - mockStatic(WorkerUtils.class, ctx -> { - ctx.when(() -> WorkerUtils.dumpToTmpFile(any())).thenCallRealMethod(); - if (consumer != null) { - consumer.accept(ctx); - } - }); - } - - private void mockInstanceUtils() { - mockStatic(InstanceUtils.class, ctx -> { - ctx.when(() -> InstanceUtils.calculateSubjectType(any())) - .thenReturn(FunctionDetails.ComponentType.FUNCTION); - }); - } - - // - // Register Functions - // - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") - public void testRegisterFunctionMissingTenant() { - try { - testRegisterFunctionMissingArguments( - null, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") - public void testRegisterFunctionMissingNamespace() { - try { - testRegisterFunctionMissingArguments( - tenant, - null, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") - public void testRegisterFunctionMissingFunctionName() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - null, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function package is not provided") - public void testRegisterFunctionMissingPackage() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "No input topic\\(s\\) specified for the function") - public void testRegisterFunctionMissingInputTopics() throws Exception { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - null, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function Package is not provided") - public void testRegisterFunctionMissingPackageDetails() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - null, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function package does not have" - + " the correct format. Pulsar cannot determine if the package is a NAR package or JAR package. Function " - + "classname is not provided and attempts to load it as a NAR package produced the following error.*") - public void testRegisterFunctionMissingClassName() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - null, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function class UnknownClass must be in class path") - public void testRegisterFunctionWrongClassName() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - "UnknownClass", - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function parallelism must be a positive number") - public void testRegisterFunctionWrongParallelism() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - -2, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, - expectedExceptionsMessageRegExp = "Output topic persistent://public/default/test_src is also being used as an input topic \\(topics must be one or the other\\)") - public void testRegisterFunctionSameInputOutput() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - topicsToSerDeClassName.keySet().iterator().next(), - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Output topic " + function + - "-output-topic/test:" + " is invalid") - public void testRegisterFunctionWrongOutputTopic() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - function + "-output-topic/test:", - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Encountered error .*. when getting Function package from .*") - public void testRegisterFunctionHttpUrl() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - null, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "http://localhost:1234/test"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - private void testRegisterFunctionMissingArguments( - String tenant, - String namespace, - String function, - InputStream inputStream, - Map topicsToSerDeClassName, - FormDataContentDisposition details, - String outputTopic, - String outputSerdeClassName, - String className, - Integer parallelism, - String functionPkgUrl) { - FunctionConfig functionConfig = new FunctionConfig(); - if (tenant != null) { - functionConfig.setTenant(tenant); - } - if (namespace != null) { - functionConfig.setNamespace(namespace); - } - if (function != null) { - functionConfig.setName(function); - } - if (topicsToSerDeClassName != null) { - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - } - if (outputTopic != null) { - functionConfig.setOutput(outputTopic); - } - if (outputSerdeClassName != null) { - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - } - if (className != null) { - functionConfig.setClassName(className); - } - if (parallelism != null) { - functionConfig.setParallelism(parallelism); - } - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - - mockWorkerUtils(ctx -> { - ctx.when(() -> WorkerUtils.uploadFileToBookkeeper( - anyString(), - any(File.class), - any(Namespace.class))) - .thenCallRealMethod(); - }); - - try { - resource.registerFunction( - tenant, - namespace, - function, - inputStream, - details, - functionPkgUrl, - JsonFormat.printer().print(FunctionConfigUtils.convert(functionConfig, (ClassLoader) null)), - AuthenticationParameters.builder().build()); - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } - - } - - private void registerDefaultFunction() { - FunctionConfig functionConfig = createDefaultFunctionConfig(); - try { - resource.registerFunction( - tenant, - namespace, - function, - mockedInputStream, - mockedFormData, - null, - JsonFormat.printer().print(FunctionConfigUtils.convert(functionConfig, (ClassLoader) null)), - AuthenticationParameters.builder().build()); - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function already exists") - public void testRegisterExistedFunction() { - try { - Configurator.setRootLevel(Level.DEBUG); - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "upload failure") - public void testRegisterFunctionUploadFailure() throws Exception { - try { - mockWorkerUtils(ctx -> { - ctx.when(() -> WorkerUtils.uploadFileToBookkeeper( - anyString(), - any(File.class), - any(Namespace.class))) - .thenThrow(new IOException("upload failure")); - }); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); - throw re; - } - } - - @Test - public void testRegisterFunctionSuccess() throws Exception { - try { - try (MockedStatic mocked = Mockito.mockStatic(WorkerUtils.class)) { - mocked.when(() -> WorkerUtils.uploadToBookKeeper( - any(Namespace.class), - any(InputStream.class), - anyString())).thenAnswer((i) -> null); - mocked.when(() -> WorkerUtils.dumpToTmpFile(any())).thenAnswer(i -> - { - try { - File tmpFile = FunctionCommon.createPkgTempFile(); - tmpFile.deleteOnExit(); - Files.copy((InputStream) i.getArguments()[0], tmpFile.toPath(), REPLACE_EXISTING); - return tmpFile; - } catch (IOException e) { - throw new RuntimeException("Cannot create a temporary file", e); - } - - } - ); - WorkerUtils.uploadToBookKeeper(null, null, null); - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - - registerDefaultFunction(); - } - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace does not exist") - public void testRegisterFunctionNonExistingNamespace() { - try { - this.namespaceList.clear(); - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant does not exist") - public void testRegisterFunctionNonexistantTenant() throws Exception { - try { - when(mockedTenants.getTenantInfo(any())).thenThrow(PulsarAdminException.NotFoundException.class); - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "function failed to register") - public void testRegisterFunctionFailure() throws Exception { - try { - mockWorkerUtils(); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - doThrow(new IllegalArgumentException("function failed to register")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), anyBoolean()); - - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function registration interrupted") - public void testRegisterFunctionInterrupted() throws Exception { - try { - mockWorkerUtils(); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - - doThrow(new IllegalStateException("Function registration interrupted")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), anyBoolean()); - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); - throw re; - } - } - - // - // Update Functions - // - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") - public void testUpdateFunctionMissingTenant() throws Exception { - try { - testUpdateFunctionMissingArguments( - null, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "Tenant is not provided"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") - public void testUpdateFunctionMissingNamespace() throws Exception { - try { - testUpdateFunctionMissingArguments( - tenant, - null, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "Namespace is not provided"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") - public void testUpdateFunctionMissingFunctionName() throws Exception { - try { - testUpdateFunctionMissingArguments( - tenant, - namespace, - null, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "Function name is not provided"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Update contains no change") - public void testUpdateFunctionMissingPackage() throws Exception { - try { - mockWorkerUtils(); - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "Update contains no change"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Update contains no change") - public void testUpdateFunctionMissingInputTopic() throws Exception { - try { - mockWorkerUtils(); - - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - null, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "Update contains no change"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Update contains no change") - public void testUpdateFunctionMissingClassName() throws Exception { - try { - mockWorkerUtils(); - - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - null, - parallelism, - "Update contains no change"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test - public void testUpdateFunctionChangedParallelism() throws Exception { - try { - mockWorkerUtils(); - - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - null, - parallelism + 1, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } + @Override + protected void doSetup() { + super.doSetup(); + this.resource = spy(new FunctionsImplV2(() -> mockedWorkerService)); } - @Test - public void testUpdateFunctionChangedInputs() throws Exception { - mockWorkerUtils(); - - testUpdateFunctionMissingArguments( + protected void registerFunction(String tenant, String namespace, String function, InputStream inputStream, + FormDataContentDisposition details, String functionPkgUrl, + FunctionConfig functionConfig) throws IOException { + resource.registerFunction( tenant, namespace, function, - null, - topicsToSerDeClassName, - mockedFormData, - "DifferentOutput", - outputSerdeClassName, - null, - parallelism, + inputStream, + details, + functionPkgUrl, + JsonFormat.printer().print(FunctionConfigUtils.convert(functionConfig)), null); } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Input Topics cannot be altered") - public void testUpdateFunctionChangedOutput() throws Exception { - try { - mockWorkerUtils(); - - Map someOtherInput = new HashMap<>(); - someOtherInput.put("DifferentTopic", TopicSchema.DEFAULT_SERDE); - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - someOtherInput, - mockedFormData, - outputTopic, - outputSerdeClassName, - null, - parallelism, - "Input Topics cannot be altered"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - private void testUpdateFunctionMissingArguments( - String tenant, - String namespace, - String function, - InputStream inputStream, - Map topicsToSerDeClassName, - FormDataContentDisposition details, - String outputTopic, - String outputSerdeClassName, - String className, - Integer parallelism, - String expectedError) throws Exception { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - FunctionConfig functionConfig = new FunctionConfig(); - if (tenant != null) { - functionConfig.setTenant(tenant); - } - if (namespace != null) { - functionConfig.setNamespace(namespace); - } - if (function != null) { - functionConfig.setName(function); - } - if (topicsToSerDeClassName != null) { - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - } - if (outputTopic != null) { - functionConfig.setOutput(outputTopic); - } - if (outputSerdeClassName != null) { - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - } - if (className != null) { - functionConfig.setClassName(className); - } - if (parallelism != null) { - functionConfig.setParallelism(parallelism); - } - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - - if (expectedError != null) { - doThrow(new IllegalArgumentException(expectedError)) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), anyBoolean()); - } - - try { - resource.updateFunction( - tenant, - namespace, - function, - inputStream, - details, - null, - JsonFormat.printer().print(FunctionConfigUtils.convert(functionConfig, (ClassLoader) null)), - AuthenticationParameters.builder().build()); - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } - - } - - private void updateDefaultFunction() { - FunctionConfig functionConfig = new FunctionConfig(); - functionConfig.setTenant(tenant); - functionConfig.setNamespace(namespace); - functionConfig.setName(function); - functionConfig.setClassName(className); - functionConfig.setParallelism(parallelism); - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - functionConfig.setOutput(outputTopic); - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - - try { - resource.updateFunction( - tenant, - namespace, - function, - mockedInputStream, - mockedFormData, - null, - JsonFormat.printer().print(FunctionConfigUtils.convert(functionConfig, (ClassLoader) null)), - AuthenticationParameters.builder().build()); - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function doesn't exist") - public void testUpdateNotExistedFunction() { - try { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - updateDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "upload failure") - public void testUpdateFunctionUploadFailure() throws Exception { - try { - mockWorkerUtils(ctx -> { - ctx.when(() -> WorkerUtils.uploadFileToBookkeeper( - anyString(), - any(File.class), - any(Namespace.class))) - .thenThrow(new IOException("upload failure")); - }); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - updateDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); - throw re; - } - } - - @Test - public void testUpdateFunctionSuccess() throws Exception { - mockWorkerUtils(); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - updateDefaultFunction(); - } - - @Test - public void testUpdateFunctionWithUrl() throws Exception { - Configurator.setRootLevel(Level.DEBUG); - - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String fileLocation = file.getAbsolutePath().replace('\\', '/'); - String filePackageUrl = "file:///" + fileLocation; - - FunctionConfig functionConfig = new FunctionConfig(); - functionConfig.setOutput(outputTopic); - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - functionConfig.setTenant(tenant); - functionConfig.setNamespace(namespace); - functionConfig.setName(function); - functionConfig.setClassName(className); - functionConfig.setParallelism(parallelism); - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - try { - resource.updateFunction( - tenant, - namespace, - function, - null, - null, - filePackageUrl, - JsonFormat.printer().print(FunctionConfigUtils.convert(functionConfig, (ClassLoader) null)), - AuthenticationParameters.builder().build()); - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } - - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "function failed to register") - public void testUpdateFunctionFailure() throws Exception { - try { - mockWorkerUtils(); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - doThrow(new IllegalArgumentException("function failed to register")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), anyBoolean()); - - updateDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function registeration interrupted") - public void testUpdateFunctionInterrupted() throws Exception { - try { - mockWorkerUtils(); - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - doThrow(new IllegalStateException("Function registeration interrupted")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), anyBoolean()); - - updateDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); - throw re; - } - } - - // - // deregister function - // - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") - public void testDeregisterFunctionMissingTenant() { - try { - - testDeregisterFunctionMissingArguments( - null, - namespace, - function - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } + protected void updateFunction(String tenant, + String namespace, + String functionName, + InputStream uploadedInputStream, + FormDataContentDisposition fileDetail, + String functionPkgUrl, + FunctionConfig functionConfig, + AuthenticationParameters authParams, + UpdateOptionsImpl updateOptions) throws IOException { + resource.updateFunction(tenant, namespace, functionName, uploadedInputStream, fileDetail, functionPkgUrl, + JsonFormat.printer().print(FunctionConfigUtils.convert(functionConfig)), authParams); } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") - public void testDeregisterFunctionMissingNamespace() { - try { - testDeregisterFunctionMissingArguments( - tenant, - null, - function - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; + protected File downloadFunction(final String path, final AuthenticationParameters authParams) + throws IOException { + Response response = resource.downloadFunction(path, authParams); + StreamingOutput streamingOutput = readEntity(response, StreamingOutput.class); + File pkgFile = File.createTempFile("testpkg", "nar"); + try (OutputStream output = new FileOutputStream(pkgFile)) { + streamingOutput.write(output); } + return pkgFile; } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") - public void testDeregisterFunctionMissingFunctionName() { - try { - testDeregisterFunctionMissingArguments( - tenant, - namespace, - null - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } + private T readEntity(Response response, Class clazz) { + return clazz.cast(response.getEntity()); } - private void testDeregisterFunctionMissingArguments( + protected void testDeregisterFunctionMissingArguments( String tenant, String namespace, String function @@ -1145,112 +102,18 @@ private void testDeregisterFunctionMissingArguments( tenant, namespace, function, - AuthenticationParameters.builder().build()); + null); } - private void deregisterDefaultFunction() { + protected void deregisterDefaultFunction() { resource.deregisterFunction( tenant, namespace, function, - AuthenticationParameters.builder().build()); - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function doesn't exist") - public void testDeregisterNotExistedFunction() { - try { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - deregisterDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.NOT_FOUND); - throw re; - } - } - - @Test - public void testDeregisterFunctionSuccess() { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - deregisterDefaultFunction(); - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "function failed to deregister") - public void testDeregisterFunctionFailure() throws Exception { - try { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - doThrow(new IllegalArgumentException("function failed to deregister")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), anyBoolean()); - - deregisterDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function deregisteration interrupted") - public void testDeregisterFunctionInterrupted() throws Exception { - try { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - doThrow(new IllegalStateException("Function deregisteration interrupted")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), anyBoolean()); - - deregisterDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); - throw re; - } - } - - // - // Get Function Info - // - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") - public void testGetFunctionMissingTenant() throws IOException { - try { - testGetFunctionMissingArguments( - null, - namespace, - function - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") - public void testGetFunctionMissingNamespace() throws IOException { - try { - testGetFunctionMissingArguments( - tenant, - null, - function - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") - public void testGetFunctionMissingFunctionName() throws IOException { - try { - testGetFunctionMissingArguments( - tenant, - namespace, - null - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } + null); } - private void testGetFunctionMissingArguments( + protected void testGetFunctionMissingArguments( String tenant, String namespace, String function @@ -1258,20 +121,36 @@ private void testGetFunctionMissingArguments( resource.getFunctionInfo( tenant, namespace, - function, - AuthenticationParameters.builder().build() + function, null + ); + } + + protected void testListFunctionsMissingArguments( + String tenant, + String namespace + ) { + resource.listFunctions( + tenant, + namespace, null ); } - private FunctionDetails getDefaultFunctionInfo() throws IOException { + protected List listDefaultFunctions() { + return new Gson().fromJson(readEntity(resource.listFunctions( + tenant, + namespace, null + ), String.class), List.class); + } + + private Function.FunctionDetails getDefaultFunctionInfo() throws IOException { String json = (String) resource.getFunctionInfo( tenant, namespace, function, AuthenticationParameters.builder().build() ).getEntity(); - FunctionDetails.Builder functionDetailsBuilder = FunctionDetails.newBuilder(); + Function.FunctionDetails.Builder functionDetailsBuilder = Function.FunctionDetails.newBuilder(); mergeJson(json, functionDetailsBuilder); return functionDetailsBuilder.build(); } @@ -1292,218 +171,31 @@ public void testGetFunctionSuccess() throws IOException { mockInstanceUtils(); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - SinkSpec sinkSpec = SinkSpec.newBuilder() + Function.SinkSpec sinkSpec = Function.SinkSpec.newBuilder() .setTopic(outputTopic) .setSerDeClassName(outputSerdeClassName).build(); - FunctionDetails functionDetails = FunctionDetails.newBuilder() + Function.FunctionDetails functionDetails = Function.FunctionDetails.newBuilder() .setClassName(className) .setSink(sinkSpec) .setName(function) .setNamespace(namespace) - .setProcessingGuarantees(ProcessingGuarantees.ATMOST_ONCE) + .setProcessingGuarantees(Function.ProcessingGuarantees.ATMOST_ONCE) .setAutoAck(true) .setTenant(tenant) .setParallelism(parallelism) - .setSource(SourceSpec.newBuilder().setSubscriptionType(subscriptionType) + .setSource(Function.SourceSpec.newBuilder().setSubscriptionType(subscriptionType) .putAllTopicsToSerDeClassName(topicsToSerDeClassName)).build(); - FunctionMetaData metaData = FunctionMetaData.newBuilder() + Function.FunctionMetaData metaData = Function.FunctionMetaData.newBuilder() .setCreateTime(System.currentTimeMillis()) .setFunctionDetails(functionDetails) - .setPackageLocation(PackageLocationMetaData.newBuilder().setPackagePath("/path/to/package")) + .setPackageLocation(Function.PackageLocationMetaData.newBuilder().setPackagePath("/path/to/package")) .setVersion(1234) .build(); when(mockedManager.getFunctionMetaData(eq(tenant), eq(namespace), eq(function))).thenReturn(metaData); - FunctionDetails actual = getDefaultFunctionInfo(); + Function.FunctionDetails actual = getDefaultFunctionInfo(); assertEquals( functionDetails, actual); } - - // - // List Functions - // - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") - public void testListFunctionsMissingTenant() { - try { - testListFunctionsMissingArguments( - null, - namespace - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") - public void testListFunctionsMissingNamespace() { - try { - testListFunctionsMissingArguments( - tenant, - null - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - private void testListFunctionsMissingArguments( - String tenant, - String namespace - ) { - resource.listFunctions( - tenant, - namespace, - AuthenticationParameters.builder().build() - ); - - } - - private List listDefaultFunctions() { - return new Gson().fromJson((String) resource.listFunctions( - tenant, - namespace, - AuthenticationParameters.builder().build() - ).getEntity(), List.class); - } - - @Test - public void testListFunctionsSuccess() { - mockInstanceUtils(); - final List functions = Lists.newArrayList("test-1", "test-2"); - final List metaDataList = new LinkedList<>(); - FunctionMetaData functionMetaData1 = FunctionMetaData.newBuilder().setFunctionDetails( - FunctionDetails.newBuilder().setName("test-1").build() - ).build(); - FunctionMetaData functionMetaData2 = FunctionMetaData.newBuilder().setFunctionDetails( - FunctionDetails.newBuilder().setName("test-2").build() - ).build(); - metaDataList.add(functionMetaData1); - metaDataList.add(functionMetaData2); - when(mockedManager.listFunctions(eq(tenant), eq(namespace))).thenReturn(metaDataList); - - List functionList = listDefaultFunctions(); - assertEquals(functions, functionList); - } - - @Test - public void testDownloadFunctionHttpUrl() throws Exception { - String jarHttpUrl = - "https://repo1.maven.org/maven2/org/apache/pulsar/pulsar-common/2.4.2/pulsar-common-2.4.2.jar"; - String testDir = FunctionApiV2ResourceTest.class.getProtectionDomain().getCodeSource().getLocation().getPath(); - FunctionsImplV2 function = new FunctionsImplV2(() -> mockedWorkerService); - StreamingOutput streamOutput = (StreamingOutput) function.downloadFunction(jarHttpUrl, - AuthenticationParameters.builder().build()).getEntity(); - File pkgFile = new File(testDir, UUID.randomUUID().toString()); - OutputStream output = new FileOutputStream(pkgFile); - streamOutput.write(output); - Assert.assertTrue(pkgFile.exists()); - if (pkgFile.exists()) { - pkgFile.delete(); - } - } - - @Test - public void testDownloadFunctionFile() throws Exception { - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String fileLocation = file.getAbsolutePath().replace('\\', '/'); - String testDir = FunctionApiV2ResourceTest.class.getProtectionDomain().getCodeSource().getLocation().getPath(); - FunctionsImplV2 function = new FunctionsImplV2(() -> mockedWorkerService); - StreamingOutput streamOutput = (StreamingOutput) function.downloadFunction("file:///" + fileLocation, - AuthenticationParameters.builder().build()).getEntity(); - File pkgFile = new File(testDir, UUID.randomUUID().toString()); - OutputStream output = new FileOutputStream(pkgFile); - streamOutput.write(output); - Assert.assertTrue(pkgFile.exists()); - if (pkgFile.exists()) { - pkgFile.delete(); - } - } - - @Test - public void testRegisterFunctionFileUrlWithValidSinkClass() throws Exception { - Configurator.setRootLevel(Level.DEBUG); - - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String fileLocation = file.getAbsolutePath().replace('\\', '/'); - String filePackageUrl = "file:///" + fileLocation; - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - - FunctionConfig functionConfig = new FunctionConfig(); - functionConfig.setTenant(tenant); - functionConfig.setNamespace(namespace); - functionConfig.setName(function); - functionConfig.setClassName(className); - functionConfig.setParallelism(parallelism); - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - functionConfig.setOutput(outputTopic); - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - try { - resource.registerFunction(tenant, namespace, function, null, null, filePackageUrl, - JsonFormat.printer().print(FunctionConfigUtils.convert(functionConfig, (ClassLoader) null)), - AuthenticationParameters.builder().build()); - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } - - } - - @Test - public void testRegisterFunctionWithConflictingFields() throws Exception { - Configurator.setRootLevel(Level.DEBUG); - String actualTenant = "DIFFERENT_TENANT"; - String actualNamespace = "DIFFERENT_NAMESPACE"; - String actualName = "DIFFERENT_NAME"; - this.namespaceList.add(actualTenant + "/" + actualNamespace); - - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String fileLocation = file.getAbsolutePath().replace('\\', '/'); - String filePackageUrl = "file:///" + fileLocation; - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - when(mockedManager.containsFunction(eq(actualTenant), eq(actualNamespace), eq(actualName))).thenReturn(false); - - FunctionConfig functionConfig = new FunctionConfig(); - functionConfig.setTenant(tenant); - functionConfig.setNamespace(namespace); - functionConfig.setName(function); - functionConfig.setClassName(className); - functionConfig.setParallelism(parallelism); - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - functionConfig.setOutput(outputTopic); - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - try { - resource.registerFunction(actualTenant, actualNamespace, actualName, null, null, filePackageUrl, - JsonFormat.printer().print(FunctionConfigUtils.convert(functionConfig, (ClassLoader) null)), - AuthenticationParameters.builder().build()); - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } - } - - public static FunctionConfig createDefaultFunctionConfig() { - FunctionConfig functionConfig = new FunctionConfig(); - functionConfig.setTenant(tenant); - functionConfig.setNamespace(namespace); - functionConfig.setName(function); - functionConfig.setClassName(className); - functionConfig.setParallelism(parallelism); - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - functionConfig.setOutput(outputTopic); - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - return functionConfig; - } - - public static FunctionDetails createDefaultFunctionDetails() { - FunctionConfig functionConfig = createDefaultFunctionConfig(); - return FunctionConfigUtils.convert(functionConfig, (ClassLoader) null); - } } diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionApiResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionApiResourceTest.java new file mode 100644 index 0000000000000..5845ff3afd9ac --- /dev/null +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionApiResourceTest.java @@ -0,0 +1,1367 @@ +/* + * 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.worker.rest.api.v3; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import com.google.common.collect.Lists; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.function.Consumer; +import javax.ws.rs.core.Response; +import org.apache.distributedlog.api.namespace.Namespace; +import org.apache.logging.log4j.Level; +import org.apache.logging.log4j.core.config.Configurator; +import org.apache.pulsar.broker.authentication.AuthenticationParameters; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.common.functions.FunctionConfig; +import org.apache.pulsar.common.functions.UpdateOptionsImpl; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.RestException; +import org.apache.pulsar.functions.api.Context; +import org.apache.pulsar.functions.api.Function; +import org.apache.pulsar.functions.proto.Function.FunctionDetails; +import org.apache.pulsar.functions.proto.Function.FunctionMetaData; +import org.apache.pulsar.functions.proto.Function.SubscriptionType; +import org.apache.pulsar.functions.source.TopicSchema; +import org.apache.pulsar.functions.utils.FunctionConfigUtils; +import org.apache.pulsar.functions.worker.WorkerConfig; +import org.apache.pulsar.functions.worker.WorkerUtils; +import org.glassfish.jersey.media.multipart.FormDataContentDisposition; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.Test; + +public abstract class AbstractFunctionApiResourceTest extends AbstractFunctionsResourceTest { + + @Test + public void testListFunctionsSuccess() { + mockInstanceUtils(); + final List functions = Lists.newArrayList("test-1", "test-2"); + final List metaDataList = new LinkedList<>(); + FunctionMetaData functionMetaData1 = FunctionMetaData.newBuilder().setFunctionDetails( + FunctionDetails.newBuilder().setName("test-1").build() + ).build(); + FunctionMetaData functionMetaData2 = FunctionMetaData.newBuilder().setFunctionDetails( + FunctionDetails.newBuilder().setName("test-2").build() + ).build(); + metaDataList.add(functionMetaData1); + metaDataList.add(functionMetaData2); + when(mockedManager.listFunctions(eq(tenant), eq(namespace))).thenReturn(metaDataList); + + List functionList = listDefaultFunctions(); + assertEquals(functions, functionList); + } + + @Test + public void testOnlyGetSources() { + List functions = Lists.newArrayList("test-2"); + List functionMetaDataList = new LinkedList<>(); + FunctionMetaData f1 = FunctionMetaData.newBuilder().setFunctionDetails( + FunctionDetails.newBuilder() + .setName("test-1") + .setComponentType(FunctionDetails.ComponentType.SOURCE) + .build()).build(); + functionMetaDataList.add(f1); + FunctionMetaData f2 = FunctionMetaData.newBuilder().setFunctionDetails( + FunctionDetails.newBuilder() + .setName("test-2") + .setComponentType(FunctionDetails.ComponentType.FUNCTION) + .build()).build(); + functionMetaDataList.add(f2); + FunctionMetaData f3 = FunctionMetaData.newBuilder().setFunctionDetails( + FunctionDetails.newBuilder() + .setName("test-3") + .setComponentType(FunctionDetails.ComponentType.SINK) + .build()).build(); + functionMetaDataList.add(f3); + when(mockedManager.listFunctions(eq(tenant), eq(namespace))).thenReturn(functionMetaDataList); + + List functionList = listDefaultFunctions(); + assertEquals(functions, functionList); + } + + private static final class TestFunction implements Function { + + @Override + public String process(String input, Context context) { + return input; + } + } + + private static final class WrongFunction implements Consumer { + @Override + public void accept(String s) { + + } + } + + protected static final String function = "test-function"; + protected static final String outputTopic = "test-output-topic"; + protected static final String outputSerdeClassName = TopicSchema.DEFAULT_SERDE; + protected static final String className = TestFunction.class.getName(); + protected SubscriptionType subscriptionType = SubscriptionType.FAILOVER; + protected FunctionMetaData mockedFunctionMetadata; + + + @Override + protected void doSetup() { + this.mockedFunctionMetadata = + FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); + when(mockedManager.getFunctionMetaData(any(), any(), any())).thenReturn(mockedFunctionMetadata); + } + + @Override + protected FunctionDetails.ComponentType getComponentType() { + return FunctionDetails.ComponentType.FUNCTION; + } + + + abstract protected void registerFunction(String tenant, String namespace, String function, InputStream inputStream, + FormDataContentDisposition details, String functionPkgUrl, FunctionConfig functionConfig) + throws IOException; + abstract protected void updateFunction(String tenant, + String namespace, + String functionName, + InputStream uploadedInputStream, + FormDataContentDisposition fileDetail, + String functionPkgUrl, + FunctionConfig functionConfig, + AuthenticationParameters authParams, + UpdateOptionsImpl updateOptions) throws IOException; + + abstract protected File downloadFunction(final String path, final AuthenticationParameters authParams) + throws IOException; + + abstract protected void testDeregisterFunctionMissingArguments( + String tenant, + String namespace, + String function + ); + + abstract protected void deregisterDefaultFunction(); + + abstract protected void testGetFunctionMissingArguments( + String tenant, + String namespace, + String function + ) throws IOException; + + abstract protected void testListFunctionsMissingArguments( + String tenant, + String namespace + ); + + abstract protected List listDefaultFunctions(); + + // + // Register Functions + // + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") + public void testRegisterFunctionMissingTenant() throws IOException { + try { + testRegisterFunctionMissingArguments( + null, + namespace, + function, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") + public void testRegisterFunctionMissingNamespace() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + null, + function, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") + public void testRegisterFunctionMissingFunctionName() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + null, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function package is not " + + "provided") + public void testRegisterFunctionMissingPackage() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + function, + null, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "No input topic\\(s\\) " + + "specified for the function") + public void testRegisterFunctionMissingInputTopics() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + function, + mockedInputStream, + null, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function Package is not " + + "provided") + public void testRegisterFunctionMissingPackageDetails() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + function, + mockedInputStream, + topicsToSerDeClassName, + null, + outputTopic, + outputSerdeClassName, + className, + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, + expectedExceptionsMessageRegExp = "Function class name is not provided.") + public void testRegisterFunctionMissingClassName() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + function, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + null, + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function class UnknownClass " + + "must be in class path") + public void testRegisterFunctionWrongClassName() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + function, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + "UnknownClass", + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function parallelism must be a" + + " positive number") + public void testRegisterFunctionWrongParallelism() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + function, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + -2, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, + expectedExceptionsMessageRegExp = "Output topic persistent://public/default/test_src is also being used " + + "as an input topic \\(topics must be one or the other\\)") + public void testRegisterFunctionSameInputOutput() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + function, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + topicsToSerDeClassName.keySet().iterator().next(), + outputSerdeClassName, + className, + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Output topic " + function + + "-output-topic/test:" + " is invalid") + public void testRegisterFunctionWrongOutputTopic() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + function, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + function + "-output-topic/test:", + outputSerdeClassName, + className, + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Encountered error .*. when " + + "getting Function package from .*") + public void testRegisterFunctionHttpUrl() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + function, + null, + topicsToSerDeClassName, + null, + outputTopic, + outputSerdeClassName, + className, + parallelism, + "http://localhost:1234/test"); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function class .*. does not " + + "implement the correct interface") + public void testRegisterFunctionImplementWrongInterface() throws IOException { + try { + testRegisterFunctionMissingArguments( + tenant, + namespace, + function, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + WrongFunction.class.getName(), + parallelism, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + private void testRegisterFunctionMissingArguments( + String tenant, + String namespace, + String function, + InputStream inputStream, + Map topicsToSerDeClassName, + FormDataContentDisposition details, + String outputTopic, + String outputSerdeClassName, + String className, + Integer parallelism, + String functionPkgUrl) throws IOException { + FunctionConfig functionConfig = new FunctionConfig(); + if (tenant != null) { + functionConfig.setTenant(tenant); + } + if (namespace != null) { + functionConfig.setNamespace(namespace); + } + if (function != null) { + functionConfig.setName(function); + } + if (topicsToSerDeClassName != null) { + functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); + } + if (outputTopic != null) { + functionConfig.setOutput(outputTopic); + } + if (outputSerdeClassName != null) { + functionConfig.setOutputSerdeClassName(outputSerdeClassName); + } + if (className != null) { + functionConfig.setClassName(className); + } + if (parallelism != null) { + functionConfig.setParallelism(parallelism); + } + functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); + + registerFunction(tenant, namespace, function, inputStream, details, functionPkgUrl, functionConfig); + + } + + @Test(expectedExceptions = Exception.class, expectedExceptionsMessageRegExp = "Function config is not provided") + public void testUpdateMissingFunctionConfig() throws IOException { + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + updateFunction( + tenant, + namespace, + function, + mockedInputStream, + mockedFormData, + null, + null, + null, null); + } + + + private void registerDefaultFunction() throws IOException { + registerDefaultFunctionWithPackageUrl(null); + } + + private void registerDefaultFunctionWithPackageUrl(String packageUrl) throws IOException { + FunctionConfig functionConfig = createDefaultFunctionConfig(); + registerFunction(tenant, namespace, function, mockedInputStream, mockedFormData, packageUrl, functionConfig); + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function already" + + " exists") + public void testRegisterExistedFunction() throws IOException { + try { + Configurator.setRootLevel(Level.DEBUG); + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + registerDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "upload failure") + public void testRegisterFunctionUploadFailure() throws IOException { + try { + mockWorkerUtils(ctx -> { + ctx.when(() -> { + WorkerUtils.uploadFileToBookkeeper( + anyString(), + any(File.class), + any(Namespace.class)); + } + ).thenThrow(new IOException("upload failure")); + }); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); + + registerDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); + throw re; + } + } + + @Test + public void testRegisterFunctionSuccess() throws IOException { + try { + mockWorkerUtils(); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); + + registerDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(timeOut = 20000) + public void testRegisterFunctionSuccessWithPackageName() throws IOException { + registerDefaultFunctionWithPackageUrl("function://public/default/test@v1"); + } + + @Test(timeOut = 20000) + public void testRegisterFunctionFailedWithWrongPackageName() throws PulsarAdminException, IOException { + try { + doThrow(new PulsarAdminException("package name is invalid")) + .when(mockedPackages).download(anyString(), anyString()); + registerDefaultFunctionWithPackageUrl("function://"); + } catch (RestException e) { + // expected exception + assertEquals(e.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace does not exist") + public void testRegisterFunctionNonExistingNamespace() throws IOException { + try { + this.namespaceList.clear(); + registerDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant does not exist") + public void testRegisterFunctionNonexistantTenant() throws Exception { + try { + when(mockedTenants.getTenantInfo(any())).thenThrow(PulsarAdminException.NotFoundException.class); + registerDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "function failed to register") + public void testRegisterFunctionFailure() throws Exception { + try { + mockWorkerUtils(); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); + + doThrow(new IllegalArgumentException("function failed to register")) + .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); + + registerDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function registration " + + "interrupted") + public void testRegisterFunctionInterrupted() throws Exception { + try { + mockWorkerUtils(); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); + + doThrow(new IllegalStateException("Function registration interrupted")) + .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); + + registerDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); + throw re; + } + } + + // + // Update Functions + // + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") + public void testUpdateFunctionMissingTenant() throws Exception { + try { + testUpdateFunctionMissingArguments( + null, + namespace, + function, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + parallelism, + "Tenant is not provided"); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") + public void testUpdateFunctionMissingNamespace() throws Exception { + try { + testUpdateFunctionMissingArguments( + tenant, + null, + function, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + parallelism, + "Namespace is not provided"); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") + public void testUpdateFunctionMissingFunctionName() throws Exception { + try { + testUpdateFunctionMissingArguments( + tenant, + namespace, + null, + mockedInputStream, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + parallelism, + "Function name is not provided"); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Update contains no change") + public void testUpdateFunctionMissingPackage() throws Exception { + try { + mockWorkerUtils(); + testUpdateFunctionMissingArguments( + tenant, + namespace, + function, + null, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + parallelism, + "Update contains no change"); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Update contains no change") + public void testUpdateFunctionMissingInputTopic() throws Exception { + try { + mockWorkerUtils(); + + testUpdateFunctionMissingArguments( + tenant, + namespace, + function, + null, + null, + mockedFormData, + outputTopic, + outputSerdeClassName, + className, + parallelism, + "Update contains no change"); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Update contains no change") + public void testUpdateFunctionMissingClassName() throws Exception { + try { + mockWorkerUtils(); + + testUpdateFunctionMissingArguments( + tenant, + namespace, + function, + null, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + null, + parallelism, + "Update contains no change"); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test + public void testUpdateFunctionChangedParallelism() throws Exception { + try { + mockWorkerUtils(); + + testUpdateFunctionMissingArguments( + tenant, + namespace, + function, + null, + topicsToSerDeClassName, + mockedFormData, + outputTopic, + outputSerdeClassName, + null, + parallelism + 1, + null); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test + public void testUpdateFunctionChangedInputs() throws Exception { + mockWorkerUtils(); + + testUpdateFunctionMissingArguments( + tenant, + namespace, + function, + null, + topicsToSerDeClassName, + mockedFormData, + "DifferentOutput", + outputSerdeClassName, + null, + parallelism, + null); + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Input Topics cannot be altered") + public void testUpdateFunctionChangedOutput() throws Exception { + try { + mockWorkerUtils(); + + Map someOtherInput = new HashMap<>(); + someOtherInput.put("DifferentTopic", TopicSchema.DEFAULT_SERDE); + testUpdateFunctionMissingArguments( + tenant, + namespace, + function, + null, + someOtherInput, + mockedFormData, + outputTopic, + outputSerdeClassName, + null, + parallelism, + "Input Topics cannot be altered"); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + private void testUpdateFunctionMissingArguments( + String tenant, + String namespace, + String function, + InputStream inputStream, + Map topicsToSerDeClassName, + FormDataContentDisposition details, + String outputTopic, + String outputSerdeClassName, + String className, + Integer parallelism, + String expectedError) throws Exception { + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + FunctionConfig functionConfig = new FunctionConfig(); + if (tenant != null) { + functionConfig.setTenant(tenant); + } + if (namespace != null) { + functionConfig.setNamespace(namespace); + } + if (function != null) { + functionConfig.setName(function); + } + if (topicsToSerDeClassName != null) { + functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); + } + if (outputTopic != null) { + functionConfig.setOutput(outputTopic); + } + if (outputSerdeClassName != null) { + functionConfig.setOutputSerdeClassName(outputSerdeClassName); + } + if (className != null) { + functionConfig.setClassName(className); + } + if (parallelism != null) { + functionConfig.setParallelism(parallelism); + } + functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); + + if (expectedError != null) { + doThrow(new IllegalArgumentException(expectedError)) + .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); + } + + updateFunction( + tenant, + namespace, + function, + inputStream, + details, + null, + functionConfig, + null, null); + + } + + private void updateDefaultFunction() throws IOException { + updateDefaultFunctionWithPackageUrl(null); + } + + private void updateDefaultFunctionWithPackageUrl(String packageUrl) throws IOException { + FunctionConfig functionConfig = new FunctionConfig(); + functionConfig.setTenant(tenant); + functionConfig.setNamespace(namespace); + functionConfig.setName(function); + functionConfig.setClassName(className); + functionConfig.setParallelism(parallelism); + functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); + functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); + functionConfig.setOutput(outputTopic); + functionConfig.setOutputSerdeClassName(outputSerdeClassName); + + updateFunction( + tenant, + namespace, + function, + mockedInputStream, + mockedFormData, + packageUrl, + functionConfig, + null, null); + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function doesn't" + + " exist") + public void testUpdateNotExistedFunction() throws IOException { + try { + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); + updateDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "upload failure") + public void testUpdateFunctionUploadFailure() throws Exception { + try { + mockWorkerUtils(ctx -> { + ctx.when(() -> { + WorkerUtils.uploadFileToBookkeeper( + anyString(), + any(File.class), + any(Namespace.class)); + + }).thenThrow(new IOException("upload failure")); + }); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + updateDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); + throw re; + } + } + + @Test + public void testUpdateFunctionSuccess() throws Exception { + mockWorkerUtils(); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + updateDefaultFunction(); + } + + @Test + public void testUpdateFunctionWithUrl() throws IOException { + Configurator.setRootLevel(Level.DEBUG); + + String fileLocation = FutureUtil.class.getProtectionDomain().getCodeSource().getLocation().getPath(); + String filePackageUrl = "file://" + fileLocation; + + FunctionConfig functionConfig = new FunctionConfig(); + functionConfig.setOutput(outputTopic); + functionConfig.setOutputSerdeClassName(outputSerdeClassName); + functionConfig.setTenant(tenant); + functionConfig.setNamespace(namespace); + functionConfig.setName(function); + functionConfig.setClassName(className); + functionConfig.setParallelism(parallelism); + functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); + functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + updateFunction( + tenant, + namespace, + function, + null, + null, + filePackageUrl, + functionConfig, + null, null); + + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "function failed to register") + public void testUpdateFunctionFailure() throws Exception { + try { + mockWorkerUtils(); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + doThrow(new IllegalArgumentException("function failed to register")) + .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); + + updateDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function registeration " + + "interrupted") + public void testUpdateFunctionInterrupted() throws Exception { + try { + mockWorkerUtils(); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + doThrow(new IllegalStateException("Function registeration interrupted")) + .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); + + updateDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); + throw re; + } + } + + + @Test(timeOut = 20000) + public void testUpdateFunctionSuccessWithPackageName() throws IOException { + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + updateDefaultFunctionWithPackageUrl("function://public/default/test@v1"); + } + + @Test(timeOut = 20000) + public void testUpdateFunctionFailedWithWrongPackageName() throws PulsarAdminException, IOException { + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + try { + doThrow(new PulsarAdminException("package name is invalid")) + .when(mockedPackages).download(anyString(), anyString()); + registerDefaultFunctionWithPackageUrl("function://"); + } catch (RestException e) { + // expected exception + assertEquals(e.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + } + } + + // + // deregister function + // + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") + public void testDeregisterFunctionMissingTenant() { + try { + + testDeregisterFunctionMissingArguments( + null, + namespace, + function + ); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") + public void testDeregisterFunctionMissingNamespace() { + try { + testDeregisterFunctionMissingArguments( + tenant, + null, + function + ); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") + public void testDeregisterFunctionMissingFunctionName() { + try { + testDeregisterFunctionMissingArguments( + tenant, + namespace, + null + ); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function doesn't" + + " exist") + public void testDeregisterNotExistedFunction() { + try { + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); + deregisterDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.NOT_FOUND); + throw re; + } + } + + @Test + public void testDeregisterFunctionSuccess() { + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + deregisterDefaultFunction(); + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "function failed to deregister") + public void testDeregisterFunctionFailure() throws Exception { + try { + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + doThrow(new IllegalArgumentException("function failed to deregister")) + .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); + + deregisterDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function deregisteration " + + "interrupted") + public void testDeregisterFunctionInterrupted() throws Exception { + try { + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + doThrow(new IllegalStateException("Function deregisteration interrupted")) + .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); + + deregisterDefaultFunction(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); + throw re; + } + } + + // + // Get Function Info + // + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") + public void testGetFunctionMissingTenant() throws IOException { + try { + testGetFunctionMissingArguments( + null, + namespace, + function + ); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") + public void testGetFunctionMissingNamespace() throws IOException { + try { + testGetFunctionMissingArguments( + tenant, + null, + function + ); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") + public void testGetFunctionMissingFunctionName() throws IOException { + try { + testGetFunctionMissingArguments( + tenant, + namespace, + null + ); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + // + // List Functions + // + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") + public void testListFunctionsMissingTenant() { + try { + testListFunctionsMissingArguments( + null, + namespace + ); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") + public void testListFunctionsMissingNamespace() { + try { + testListFunctionsMissingArguments( + tenant, + null + ); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); + throw re; + } + } + + @Test + public void testDownloadFunctionHttpUrl() throws Exception { + String jarHttpUrl = + "https://repo1.maven.org/maven2/org/apache/pulsar/pulsar-common/2.4.2/pulsar-common-2.4.2.jar"; + File pkgFile = downloadFunction(jarHttpUrl, null); + pkgFile.delete(); + } + + @Test + public void testDownloadFunctionFile() throws Exception { + File file = getPulsarApiExamplesNar(); + File pkgFile = downloadFunction(file.toURI().toString(), null); + Assert.assertTrue(pkgFile.exists()); + Assert.assertEquals(file.length(), pkgFile.length()); + pkgFile.delete(); + } + + @Test + public void testDownloadFunctionBuiltinConnector() throws Exception { + File file = getPulsarApiExamplesNar(); + + WorkerConfig config = new WorkerConfig() + .setUploadBuiltinSinksSources(false); + when(mockedWorkerService.getWorkerConfig()).thenReturn(config); + + registerBuiltinConnector("cassandra", file); + + File pkgFile = downloadFunction("builtin://cassandra", null); + Assert.assertTrue(pkgFile.exists()); + Assert.assertEquals(file.length(), pkgFile.length()); + pkgFile.delete(); + } + + @Test + public void testDownloadFunctionBuiltinFunction() throws Exception { + File file = getPulsarApiExamplesNar(); + + WorkerConfig config = new WorkerConfig() + .setUploadBuiltinSinksSources(false); + when(mockedWorkerService.getWorkerConfig()).thenReturn(config); + + registerBuiltinFunction("exclamation", file); + + File pkgFile = downloadFunction("builtin://exclamation", null); + Assert.assertTrue(pkgFile.exists()); + Assert.assertEquals(file.length(), pkgFile.length()); + pkgFile.delete(); + } + + @Test + public void testRegisterFunctionFileUrlWithValidSinkClass() throws Exception { + Configurator.setRootLevel(Level.DEBUG); + + File file = getPulsarApiExamplesNar(); + String filePackageUrl = file.toURI().toString(); + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); + + FunctionConfig functionConfig = new FunctionConfig(); + functionConfig.setTenant(tenant); + functionConfig.setNamespace(namespace); + functionConfig.setName(function); + functionConfig.setClassName(className); + functionConfig.setParallelism(parallelism); + functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); + functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); + functionConfig.setOutput(outputTopic); + functionConfig.setOutputSerdeClassName(outputSerdeClassName); + registerFunction(tenant, namespace, function, null, null, filePackageUrl, functionConfig); + + } + + @Test + public void testRegisterFunctionWithConflictingFields() throws Exception { + Configurator.setRootLevel(Level.DEBUG); + String actualTenant = "DIFFERENT_TENANT"; + String actualNamespace = "DIFFERENT_NAMESPACE"; + String actualName = "DIFFERENT_NAME"; + this.namespaceList.add(actualTenant + "/" + actualNamespace); + + File file = getPulsarApiExamplesNar(); + String filePackageUrl = file.toURI().toString(); + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + when(mockedManager.containsFunction(eq(actualTenant), eq(actualNamespace), eq(actualName))).thenReturn(false); + + FunctionConfig functionConfig = new FunctionConfig(); + functionConfig.setTenant(tenant); + functionConfig.setNamespace(namespace); + functionConfig.setName(function); + functionConfig.setClassName(className); + functionConfig.setParallelism(parallelism); + functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); + functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); + functionConfig.setOutput(outputTopic); + functionConfig.setOutputSerdeClassName(outputSerdeClassName); + registerFunction(actualTenant, actualNamespace, actualName, null, null, filePackageUrl, functionConfig); + } + + public static FunctionConfig createDefaultFunctionConfig() { + FunctionConfig functionConfig = new FunctionConfig(); + functionConfig.setTenant(tenant); + functionConfig.setNamespace(namespace); + functionConfig.setName(function); + functionConfig.setClassName(className); + functionConfig.setParallelism(parallelism); + functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); + functionConfig.setOutput(outputTopic); + functionConfig.setOutputSerdeClassName(outputSerdeClassName); + functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); + return functionConfig; + } + + public static FunctionDetails createDefaultFunctionDetails() { + FunctionConfig functionConfig = createDefaultFunctionConfig(); + return FunctionConfigUtils.convert(functionConfig); + } +} diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionsResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionsResourceTest.java new file mode 100644 index 0000000000000..4cc4ed0b09819 --- /dev/null +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionsResourceTest.java @@ -0,0 +1,323 @@ +/* + * 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.worker.rest.api.v3; + +import static org.apache.pulsar.functions.source.TopicSchema.DEFAULT_SERDE; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.withSettings; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.UncheckedIOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardCopyOption; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.function.Consumer; +import org.apache.distributedlog.api.namespace.Namespace; +import org.apache.pulsar.client.admin.Functions; +import org.apache.pulsar.client.admin.Namespaces; +import org.apache.pulsar.client.admin.Packages; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.Tenants; +import org.apache.pulsar.common.functions.FunctionDefinition; +import org.apache.pulsar.common.io.ConnectorDefinition; +import org.apache.pulsar.common.nar.NarClassLoader; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.functions.instance.InstanceUtils; +import org.apache.pulsar.functions.proto.Function; +import org.apache.pulsar.functions.runtime.RuntimeFactory; +import org.apache.pulsar.functions.source.TopicSchema; +import org.apache.pulsar.functions.utils.LoadedFunctionPackage; +import org.apache.pulsar.functions.utils.ValidatableFunctionPackage; +import org.apache.pulsar.functions.utils.functions.FunctionArchive; +import org.apache.pulsar.functions.utils.functions.FunctionUtils; +import org.apache.pulsar.functions.utils.io.Connector; +import org.apache.pulsar.functions.utils.io.ConnectorUtils; +import org.apache.pulsar.functions.worker.ConnectorsManager; +import org.apache.pulsar.functions.worker.FunctionMetaDataManager; +import org.apache.pulsar.functions.worker.FunctionRuntimeManager; +import org.apache.pulsar.functions.worker.FunctionsManager; +import org.apache.pulsar.functions.worker.LeaderService; +import org.apache.pulsar.functions.worker.PulsarWorkerService; +import org.apache.pulsar.functions.worker.WorkerConfig; +import org.apache.pulsar.functions.worker.WorkerUtils; +import org.apache.pulsar.functions.worker.rest.api.PulsarFunctionTestTemporaryDirectory; +import org.glassfish.jersey.media.multipart.FormDataContentDisposition; +import org.mockito.Answers; +import org.mockito.MockSettings; +import org.mockito.MockedStatic; +import org.mockito.Mockito; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; + +public abstract class AbstractFunctionsResourceTest { + + protected static final String tenant = "test-tenant"; + protected static final String namespace = "test-namespace"; + protected static final Map topicsToSerDeClassName = new HashMap<>(); + protected static final String subscriptionName = "test-subscription"; + protected static final String CASSANDRA_STRING_SINK = "org.apache.pulsar.io.cassandra.CassandraStringSink"; + protected static final int parallelism = 1; + private static final String SYSTEM_PROPERTY_NAME_CASSANDRA_NAR_FILE_PATH = "pulsar-io-cassandra.nar.path"; + private static final String SYSTEM_PROPERTY_NAME_TWITTER_NAR_FILE_PATH = "pulsar-io-twitter.nar.path"; + private static final String SYSTEM_PROPERTY_NAME_INVALID_NAR_FILE_PATH = "pulsar-io-invalid.nar.path"; + private static final String SYSTEM_PROPERTY_NAME_FUNCTIONS_API_EXAMPLES_NAR_FILE_PATH = + "pulsar-functions-api-examples.nar.path"; + protected static Map mockStaticContexts = new HashMap<>(); + + static { + topicsToSerDeClassName.put("test_src", DEFAULT_SERDE); + topicsToSerDeClassName.put("persistent://public/default/test_src", TopicSchema.DEFAULT_SERDE); + } + + protected PulsarWorkerService mockedWorkerService; + protected PulsarAdmin mockedPulsarAdmin; + protected Tenants mockedTenants; + protected Namespaces mockedNamespaces; + protected Functions mockedFunctions; + protected TenantInfoImpl mockedTenantInfo; + protected List namespaceList = new LinkedList<>(); + protected FunctionMetaDataManager mockedManager; + protected FunctionRuntimeManager mockedFunctionRunTimeManager; + protected RuntimeFactory mockedRuntimeFactory; + protected Namespace mockedNamespace; + protected InputStream mockedInputStream; + protected FormDataContentDisposition mockedFormData; + protected Function.FunctionMetaData mockedFunctionMetaData; + protected LeaderService mockedLeaderService; + protected Packages mockedPackages; + protected PulsarFunctionTestTemporaryDirectory tempDirectory; + protected ConnectorsManager connectorsManager = new ConnectorsManager(); + protected FunctionsManager functionsManager = new FunctionsManager(); + + public static File getPulsarIOCassandraNar() { + return new File(Objects.requireNonNull(System.getProperty(SYSTEM_PROPERTY_NAME_CASSANDRA_NAR_FILE_PATH) + , "pulsar-io-cassandra.nar file location must be specified with " + + SYSTEM_PROPERTY_NAME_CASSANDRA_NAR_FILE_PATH + " system property")); + } + + public static File getPulsarIOTwitterNar() { + return new File(Objects.requireNonNull(System.getProperty(SYSTEM_PROPERTY_NAME_TWITTER_NAR_FILE_PATH) + , "pulsar-io-twitter.nar file location must be specified with " + + SYSTEM_PROPERTY_NAME_TWITTER_NAR_FILE_PATH + " system property")); + } + + public static File getPulsarIOInvalidNar() { + return new File(Objects.requireNonNull(System.getProperty(SYSTEM_PROPERTY_NAME_INVALID_NAR_FILE_PATH) + , "invalid nar file location must be specified with " + + SYSTEM_PROPERTY_NAME_INVALID_NAR_FILE_PATH + " system property")); + } + + public static File getPulsarApiExamplesNar() { + return new File(Objects.requireNonNull( + System.getProperty(SYSTEM_PROPERTY_NAME_FUNCTIONS_API_EXAMPLES_NAR_FILE_PATH) + , "pulsar-functions-api-examples.nar file location must be specified with " + + SYSTEM_PROPERTY_NAME_FUNCTIONS_API_EXAMPLES_NAR_FILE_PATH + " system property")); + } + + @BeforeMethod + public final void setup() throws Exception { + this.mockedManager = mock(FunctionMetaDataManager.class); + this.mockedFunctionRunTimeManager = mock(FunctionRuntimeManager.class); + this.mockedRuntimeFactory = mock(RuntimeFactory.class); + this.mockedInputStream = mock(InputStream.class); + this.mockedNamespace = mock(Namespace.class); + this.mockedFormData = mock(FormDataContentDisposition.class); + when(mockedFormData.getFileName()).thenReturn("test"); + this.mockedTenantInfo = mock(TenantInfoImpl.class); + this.mockedPulsarAdmin = mock(PulsarAdmin.class); + this.mockedTenants = mock(Tenants.class); + this.mockedNamespaces = mock(Namespaces.class); + this.mockedFunctions = mock(Functions.class); + this.mockedLeaderService = mock(LeaderService.class); + this.mockedPackages = mock(Packages.class); + namespaceList.add(tenant + "/" + namespace); + + this.mockedWorkerService = mock(PulsarWorkerService.class); + when(mockedWorkerService.getFunctionMetaDataManager()).thenReturn(mockedManager); + when(mockedWorkerService.getLeaderService()).thenReturn(mockedLeaderService); + when(mockedWorkerService.getFunctionRuntimeManager()).thenReturn(mockedFunctionRunTimeManager); + when(mockedFunctionRunTimeManager.getRuntimeFactory()).thenReturn(mockedRuntimeFactory); + when(mockedWorkerService.getDlogNamespace()).thenReturn(mockedNamespace); + when(mockedWorkerService.isInitialized()).thenReturn(true); + when(mockedWorkerService.getBrokerAdmin()).thenReturn(mockedPulsarAdmin); + when(mockedWorkerService.getFunctionAdmin()).thenReturn(mockedPulsarAdmin); + when(mockedPulsarAdmin.tenants()).thenReturn(mockedTenants); + when(mockedPulsarAdmin.namespaces()).thenReturn(mockedNamespaces); + when(mockedPulsarAdmin.functions()).thenReturn(mockedFunctions); + when(mockedPulsarAdmin.packages()).thenReturn(mockedPackages); + when(mockedTenants.getTenantInfo(any())).thenReturn(mockedTenantInfo); + when(mockedNamespaces.getNamespaces(any())).thenReturn(namespaceList); + when(mockedLeaderService.isLeader()).thenReturn(true); + doAnswer(invocationOnMock -> { + Files.copy(getDefaultNarFile().toPath(), Paths.get(invocationOnMock.getArgument(1, String.class)), + StandardCopyOption.REPLACE_EXISTING); + return null; + }).when(mockedPackages).download(any(), any()); + + // worker config + WorkerConfig workerConfig = new WorkerConfig() + .setWorkerId("test") + .setWorkerPort(8080) + .setFunctionMetadataTopicName("pulsar/functions") + .setNumFunctionPackageReplicas(3) + .setPulsarServiceUrl("pulsar://localhost:6650/"); + tempDirectory = PulsarFunctionTestTemporaryDirectory.create(getClass().getSimpleName()); + tempDirectory.useTemporaryDirectoriesForWorkerConfig(workerConfig); + when(mockedWorkerService.getWorkerConfig()).thenReturn(workerConfig); + when(mockedWorkerService.getFunctionsManager()).thenReturn(functionsManager); + when(mockedWorkerService.getConnectorsManager()).thenReturn(connectorsManager); + + doSetup(); + } + + protected File getDefaultNarFile() { + return getPulsarIOTwitterNar(); + } + + protected void doSetup() throws Exception { + + } + + @AfterMethod(alwaysRun = true) + public void cleanup() { + if (tempDirectory != null) { + tempDirectory.delete(); + } + mockStaticContexts.values().forEach(MockedStatic::close); + mockStaticContexts.clear(); + } + + protected void mockStatic(Class classStatic, Consumer> consumer) { + mockStatic(classStatic, withSettings().defaultAnswer(Mockito.CALLS_REAL_METHODS), consumer); + } + + private void mockStatic(Class classStatic, MockSettings mockSettings, Consumer> consumer) { + final MockedStatic mockedStatic = mockStaticContexts.computeIfAbsent(classStatic.getName(), + name -> Mockito.mockStatic(classStatic, mockSettings)); + consumer.accept(mockedStatic); + } + + protected void mockWorkerUtils() { + mockWorkerUtils(null); + } + + protected void mockWorkerUtils(Consumer> consumer) { + mockStatic(WorkerUtils.class, withSettings(), ctx -> { + // make dumpToTmpFile return the nar file copy + ctx.when(() -> WorkerUtils.dumpToTmpFile(mockedInputStream)) + .thenAnswer(invocation -> { + Path tempFile = Files.createTempFile("test", ".nar"); + Files.copy(getPulsarApiExamplesNar().toPath(), tempFile, + StandardCopyOption.REPLACE_EXISTING); + return tempFile.toFile(); + }); + ctx.when(() -> WorkerUtils.dumpToTmpFile(any())) + .thenAnswer(Answers.CALLS_REAL_METHODS); + if (consumer != null) { + consumer.accept(ctx); + } + }); + } + + protected void mockInstanceUtils() { + mockStatic(InstanceUtils.class, ctx -> { + ctx.when(() -> InstanceUtils.calculateSubjectType(any())) + .thenReturn(getComponentType()); + }); + } + + protected abstract Function.FunctionDetails.ComponentType getComponentType(); + + public static class LoadedConnector extends Connector { + public LoadedConnector(ConnectorDefinition connectorDefinition) { + super(null, connectorDefinition, null, true); + } + + @Override + public ValidatableFunctionPackage getConnectorFunctionPackage() { + return new LoadedFunctionPackage(getClass().getClassLoader(), ConnectorDefinition.class, + getConnectorDefinition()); + } + } + + + protected void registerBuiltinConnector(String connectorType, String className) { + ConnectorDefinition connectorDefinition = null; + if (className != null) { + connectorDefinition = new ConnectorDefinition(); + // set source and sink class to the same to simplify the test + connectorDefinition.setSinkClass(className); + connectorDefinition.setSourceClass(className); + } + connectorsManager.addConnector(connectorType, new LoadedConnector(connectorDefinition)); + } + + protected void registerBuiltinConnector(String connectorType, File packageFile) { + ConnectorDefinition cntDef; + try { + cntDef = ConnectorUtils.getConnectorDefinition(packageFile); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + connectorsManager.addConnector(connectorType, + new Connector(packageFile.toPath(), cntDef, NarClassLoader.DEFAULT_NAR_EXTRACTION_DIR, true)); + } + + public static class LoadedFunctionArchive extends FunctionArchive { + public LoadedFunctionArchive(FunctionDefinition functionDefinition) { + super(null, functionDefinition, null, true); + } + + @Override + public ValidatableFunctionPackage getFunctionPackage() { + return new LoadedFunctionPackage(getClass().getClassLoader(), FunctionDefinition.class, + getFunctionDefinition()); + } + } + + protected void registerBuiltinFunction(String functionType, String className) { + FunctionDefinition functionDefinition = null; + if (className != null) { + functionDefinition = new FunctionDefinition(); + functionDefinition.setFunctionClass(className); + } + functionsManager.addFunction(functionType, new LoadedFunctionArchive(functionDefinition)); + } + + protected void registerBuiltinFunction(String functionType, File packageFile) { + FunctionDefinition cntDef; + try { + cntDef = FunctionUtils.getFunctionDefinition(packageFile); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + functionsManager.addFunction(functionType, + new FunctionArchive(packageFile.toPath(), cntDef, NarClassLoader.DEFAULT_NAR_EXTRACTION_DIR, true)); + } +} diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/FunctionApiV3ResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/FunctionApiV3ResourceTest.java index 0c20083bb89ca..a1a418460be45 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/FunctionApiV3ResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/FunctionApiV3ResourceTest.java @@ -18,1791 +18,168 @@ */ package org.apache.pulsar.functions.worker.rest.api.v3; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; 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 com.google.common.collect.Lists; 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.URL; -import java.nio.file.Paths; -import java.util.HashMap; -import java.util.LinkedList; import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.UUID; -import java.util.function.Consumer; import javax.ws.rs.core.Response; import javax.ws.rs.core.StreamingOutput; import org.apache.distributedlog.api.namespace.Namespace; import org.apache.logging.log4j.Level; import org.apache.logging.log4j.core.config.Configurator; import org.apache.pulsar.broker.authentication.AuthenticationParameters; -import org.apache.pulsar.client.admin.Functions; -import org.apache.pulsar.client.admin.Namespaces; -import org.apache.pulsar.client.admin.Packages; -import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.client.admin.Tenants; import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.functions.UpdateOptionsImpl; -import org.apache.pulsar.common.nar.NarClassLoader; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; -import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.RestException; -import org.apache.pulsar.functions.api.Context; -import org.apache.pulsar.functions.api.Function; -import org.apache.pulsar.functions.instance.InstanceUtils; -import org.apache.pulsar.functions.proto.Function.FunctionDetails; -import org.apache.pulsar.functions.proto.Function.FunctionMetaData; -import org.apache.pulsar.functions.proto.Function.PackageLocationMetaData; -import org.apache.pulsar.functions.proto.Function.ProcessingGuarantees; -import org.apache.pulsar.functions.proto.Function.SinkSpec; -import org.apache.pulsar.functions.proto.Function.SourceSpec; -import org.apache.pulsar.functions.proto.Function.SubscriptionType; -import org.apache.pulsar.functions.runtime.RuntimeFactory; -import org.apache.pulsar.functions.source.TopicSchema; -import org.apache.pulsar.functions.utils.FunctionCommon; +import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.utils.FunctionConfigUtils; -import org.apache.pulsar.functions.utils.functions.FunctionArchive; -import org.apache.pulsar.functions.utils.io.Connector; -import org.apache.pulsar.functions.worker.ConnectorsManager; -import org.apache.pulsar.functions.worker.FunctionMetaDataManager; -import org.apache.pulsar.functions.worker.FunctionRuntimeManager; -import org.apache.pulsar.functions.worker.FunctionsManager; -import org.apache.pulsar.functions.worker.LeaderService; -import org.apache.pulsar.functions.worker.PulsarWorkerService; import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.functions.worker.rest.api.FunctionsImpl; -import org.apache.pulsar.functions.worker.rest.api.PulsarFunctionTestTemporaryDirectory; -import org.apache.pulsar.functions.worker.rest.api.v2.FunctionsApiV2Resource; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; -import org.mockito.MockedStatic; -import org.mockito.Mockito; import org.testng.Assert; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -/** - * Unit test of {@link FunctionsApiV2Resource}. - */ -public class FunctionApiV3ResourceTest { - - private static final class TestFunction implements Function { - - @Override - public String process(String input, Context context) { - return input; - } - } - - private static final class WrongFunction implements Consumer { - @Override - public void accept(String s) { - - } - } - - private static final String tenant = "test-tenant"; - private static final String namespace = "test-namespace"; - private static final String function = "test-function"; - private static final String outputTopic = "test-output-topic"; - private static final String outputSerdeClassName = TopicSchema.DEFAULT_SERDE; - private static final String className = TestFunction.class.getName(); - private SubscriptionType subscriptionType = SubscriptionType.FAILOVER; - private static final Map topicsToSerDeClassName = new HashMap<>(); - static { - topicsToSerDeClassName.put("persistent://public/default/test_src", TopicSchema.DEFAULT_SERDE); - } - private static final int parallelism = 1; - - private PulsarWorkerService mockedWorkerService; - private PulsarAdmin mockedPulsarAdmin; - private Tenants mockedTenants; - private Namespaces mockedNamespaces; - private Functions mockedFunctions; - private TenantInfoImpl mockedTenantInfo; - private List namespaceList = new LinkedList<>(); - private FunctionMetaDataManager mockedManager; - private FunctionRuntimeManager mockedFunctionRunTimeManager; - private RuntimeFactory mockedRuntimeFactory; - private Namespace mockedNamespace; +public class FunctionApiV3ResourceTest extends AbstractFunctionApiResourceTest { private FunctionsImpl resource; - private InputStream mockedInputStream; - private FormDataContentDisposition mockedFormData; - private FunctionMetaData mockedFunctionMetadata; - private LeaderService mockedLeaderService; - private Packages mockedPackages; - private PulsarFunctionTestTemporaryDirectory tempDirectory; - private static Map mockStaticContexts = new HashMap<>(); - - private static final String SYSTEM_PROPERTY_NAME_FUNCTIONS_API_EXAMPLES_NAR_FILE_PATH = - "pulsar-functions-api-examples.nar.path"; - - public static File getPulsarApiExamplesNar() { - return new File(Objects.requireNonNull( - System.getProperty(SYSTEM_PROPERTY_NAME_FUNCTIONS_API_EXAMPLES_NAR_FILE_PATH) - , "pulsar-functions-api-examples.nar file location must be specified with " - + SYSTEM_PROPERTY_NAME_FUNCTIONS_API_EXAMPLES_NAR_FILE_PATH + " system property")); - } - - @BeforeMethod - public void setup() throws Exception { - this.mockedManager = mock(FunctionMetaDataManager.class); - this.mockedFunctionRunTimeManager = mock(FunctionRuntimeManager.class); - this.mockedTenantInfo = mock(TenantInfoImpl.class); - this.mockedRuntimeFactory = mock(RuntimeFactory.class); - this.mockedInputStream = mock(InputStream.class); - this.mockedNamespace = mock(Namespace.class); - this.mockedFormData = mock(FormDataContentDisposition.class); - when(mockedFormData.getFileName()).thenReturn("test"); - this.mockedPulsarAdmin = mock(PulsarAdmin.class); - this.mockedTenants = mock(Tenants.class); - this.mockedNamespaces = mock(Namespaces.class); - this.mockedFunctions = mock(Functions.class); - this.mockedPackages = mock(Packages.class); - this.mockedLeaderService = mock(LeaderService.class); - this.mockedFunctionMetadata = FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); - namespaceList.add(tenant + "/" + namespace); - - this.mockedWorkerService = mock(PulsarWorkerService.class); - when(mockedWorkerService.getFunctionMetaDataManager()).thenReturn(mockedManager); - when(mockedWorkerService.getFunctionRuntimeManager()).thenReturn(mockedFunctionRunTimeManager); - when(mockedWorkerService.getLeaderService()).thenReturn(mockedLeaderService); - when(mockedFunctionRunTimeManager.getRuntimeFactory()).thenReturn(mockedRuntimeFactory); - when(mockedWorkerService.getDlogNamespace()).thenReturn(mockedNamespace); - when(mockedWorkerService.isInitialized()).thenReturn(true); - when(mockedWorkerService.getBrokerAdmin()).thenReturn(mockedPulsarAdmin); - when(mockedWorkerService.getFunctionAdmin()).thenReturn(mockedPulsarAdmin); - when(mockedPulsarAdmin.tenants()).thenReturn(mockedTenants); - when(mockedPulsarAdmin.namespaces()).thenReturn(mockedNamespaces); - when(mockedPulsarAdmin.functions()).thenReturn(mockedFunctions); - when(mockedPulsarAdmin.packages()).thenReturn(mockedPackages); - when(mockedTenants.getTenantInfo(any())).thenReturn(mockedTenantInfo); - when(mockedNamespaces.getNamespaces(any())).thenReturn(namespaceList); - when(mockedLeaderService.isLeader()).thenReturn(true); - when(mockedManager.getFunctionMetaData(any(), any(), any())).thenReturn(mockedFunctionMetadata); - doNothing().when(mockedPackages).download(anyString(), anyString()); - - // worker config - WorkerConfig workerConfig = new WorkerConfig() - .setWorkerId("test") - .setWorkerPort(8080) - .setFunctionMetadataTopicName("pulsar/functions") - .setNumFunctionPackageReplicas(3) - .setPulsarServiceUrl("pulsar://localhost:6650/"); - tempDirectory = PulsarFunctionTestTemporaryDirectory.create(getClass().getSimpleName()); - tempDirectory.useTemporaryDirectoriesForWorkerConfig(workerConfig); - when(mockedWorkerService.getWorkerConfig()).thenReturn(workerConfig); - + @Override + protected void doSetup() { + super.doSetup(); this.resource = spy(new FunctionsImpl(() -> mockedWorkerService)); } - @AfterMethod(alwaysRun = true) - public void cleanup() { - if (tempDirectory != null) { - tempDirectory.delete(); - } - mockStaticContexts.values().forEach(MockedStatic::close); - mockStaticContexts.clear(); - } - - private void mockStatic(Class classStatic, Consumer> consumer) { - final MockedStatic mockedStatic = mockStaticContexts.computeIfAbsent(classStatic.getName(), name -> Mockito.mockStatic(classStatic)); - consumer.accept(mockedStatic); - } - - private void mockWorkerUtils() { - mockWorkerUtils(null); - } - - private void mockWorkerUtils(Consumer> consumer) { - mockStatic(WorkerUtils.class, ctx -> { - ctx.when(() -> WorkerUtils.dumpToTmpFile(any())).thenCallRealMethod(); - if (consumer != null) { - consumer.accept(ctx); - } - }); - } - - private void mockInstanceUtils() { - mockStatic(InstanceUtils.class, ctx -> { - ctx.when(() -> InstanceUtils.calculateSubjectType(any())) - .thenReturn(FunctionDetails.ComponentType.FUNCTION); - }); - } - - // - // Register Functions - // - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") - public void testRegisterFunctionMissingTenant() { - try { - testRegisterFunctionMissingArguments( - null, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") - public void testRegisterFunctionMissingNamespace() { - try { - testRegisterFunctionMissingArguments( - tenant, - null, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re){ - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") - public void testRegisterFunctionMissingFunctionName() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - null, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re){ - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function package is not provided") - public void testRegisterFunctionMissingPackage() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "No input topic\\(s\\) specified for the function") - public void testRegisterFunctionMissingInputTopics() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - null, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function Package is not provided") - public void testRegisterFunctionMissingPackageDetails() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - null, - outputTopic, - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re){ - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function package does not have" - + " the correct format. Pulsar cannot determine if the package is a NAR package or JAR package. Function " - + "classname is not provided and attempts to load it as a NAR package produced the following error.*") - public void testRegisterFunctionMissingClassName() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - null, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function class UnknownClass must be in class path") - public void testRegisterFunctionWrongClassName() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - "UnknownClass", - parallelism, - null); - } catch (RestException re){ - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function parallelism must be a positive number") - public void testRegisterFunctionWrongParallelism() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - -2, - null); - } catch (RestException re){ - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, - expectedExceptionsMessageRegExp = "Output topic persistent://public/default/test_src is also being used as an input topic \\(topics must be one or the other\\)") - public void testRegisterFunctionSameInputOutput() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - topicsToSerDeClassName.keySet().iterator().next(), - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Output topic " + function + "-output-topic/test:" + " is invalid") - public void testRegisterFunctionWrongOutputTopic() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - function + "-output-topic/test:", - outputSerdeClassName, - className, - parallelism, - null); - } catch (RestException re){ - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Encountered error .*. when getting Function package from .*") - public void testRegisterFunctionHttpUrl() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - null, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "http://localhost:1234/test"); - } catch (RestException re){ - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function class .*. does not implement the correct interface") - public void testRegisterFunctionImplementWrongInterface() { - try { - testRegisterFunctionMissingArguments( - tenant, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - WrongFunction.class.getName(), - parallelism, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - private void testRegisterFunctionMissingArguments( - String tenant, - String namespace, - String function, - InputStream inputStream, - Map topicsToSerDeClassName, - FormDataContentDisposition details, - String outputTopic, - String outputSerdeClassName, - String className, - Integer parallelism, - String functionPkgUrl) { - FunctionConfig functionConfig = new FunctionConfig(); - if (tenant != null) { - functionConfig.setTenant(tenant); - } - if (namespace != null) { - functionConfig.setNamespace(namespace); - } - if (function != null) { - functionConfig.setName(function); - } - if (topicsToSerDeClassName != null) { - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - } - if (outputTopic != null) { - functionConfig.setOutput(outputTopic); - } - if (outputSerdeClassName != null) { - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - } - if (className != null) { - functionConfig.setClassName(className); - } - if (parallelism != null) { - functionConfig.setParallelism(parallelism); - } - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - - resource.registerFunction( - tenant, - namespace, - function, - inputStream, - details, - functionPkgUrl, - functionConfig, - null); - - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function config is not provided") - public void testMissingFunctionConfig() { - resource.registerFunction( - tenant, - namespace, - function, - mockedInputStream, - mockedFormData, - null, - null, - null); - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function config is not provided") - public void testUpdateMissingFunctionConfig() { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - resource.updateFunction( - tenant, - namespace, - function, - mockedInputStream, - mockedFormData, - null, - null, - null, null); - } - - @Test - public void testUpdateSourceWithNoChange() throws ClassNotFoundException { - mockWorkerUtils(); - - FunctionDetails functionDetails = createDefaultFunctionDetails(); - NarClassLoader mockedClassLoader = mock(NarClassLoader.class); - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.getFunctionTypes(any(FunctionConfig.class), any(Class.class))).thenReturn(new Class[]{String.class, String.class}); - ctx.when(() -> FunctionCommon.convertRuntime(any(FunctionConfig.Runtime.class))).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.isFunctionCodeBuiltin(any())).thenReturn(true); - ctx.when(() -> FunctionCommon.getClassLoaderFromPackage(any(),any(),any(),any())).thenCallRealMethod(); - ctx.when(FunctionCommon::createPkgTempFile).thenCallRealMethod(); - }); - - doReturn(Function.class).when(mockedClassLoader).loadClass(anyString()); - - FunctionsManager mockedFunctionsManager = mock(FunctionsManager.class); - FunctionArchive functionArchive = FunctionArchive.builder() - .classLoader(mockedClassLoader) - .build(); - when(mockedFunctionsManager.getFunction("exclamation")).thenReturn(functionArchive); - when(mockedFunctionsManager.getFunctionArchive(any())).thenReturn(getPulsarApiExamplesNar().toPath()); - - when(mockedWorkerService.getFunctionsManager()).thenReturn(mockedFunctionsManager); - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - // No change on config, - FunctionConfig funcConfig = createDefaultFunctionConfig(); - mockStatic(FunctionConfigUtils.class, ctx -> { - ctx.when(() -> FunctionConfigUtils.convertFromDetails(any())).thenReturn(funcConfig); - ctx.when(() -> FunctionConfigUtils.validateUpdate(any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionConfigUtils.convert(any(FunctionConfig.class), any(ClassLoader.class))).thenReturn(functionDetails); - ctx.when(() -> FunctionConfigUtils.convert(any(FunctionConfig.class), any(FunctionConfigUtils.ExtractedFunctionDetails.class))).thenReturn(functionDetails); - ctx.when(() -> FunctionConfigUtils.validateJavaFunction(any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionConfigUtils.doCommonChecks(any())).thenCallRealMethod(); - ctx.when(() -> FunctionConfigUtils.collectAllInputTopics(any())).thenCallRealMethod(); - ctx.when(() -> FunctionConfigUtils.doJavaChecks(any(), any())).thenCallRealMethod(); - }); - - // config has not changes and don't update auth, should fail - try { - resource.updateFunction( - funcConfig.getTenant(), - funcConfig.getNamespace(), - funcConfig.getName(), - null, - mockedFormData, - null, - funcConfig, - null, - null); - fail("Update without changes should fail"); - } catch (RestException e) { - assertTrue(e.getMessage().contains("Update contains no change")); - } - - try { - UpdateOptionsImpl updateOptions = new UpdateOptionsImpl(); - updateOptions.setUpdateAuthData(false); - resource.updateFunction( - funcConfig.getTenant(), - funcConfig.getNamespace(), - funcConfig.getName(), - null, - mockedFormData, - null, - funcConfig, - null, - updateOptions); - fail("Update without changes should fail"); - } catch (RestException e) { - assertTrue(e.getMessage().contains("Update contains no change")); - } - - // no changes but set the auth-update flag to true, should not fail - UpdateOptionsImpl updateOptions = new UpdateOptionsImpl(); - updateOptions.setUpdateAuthData(true); - resource.updateFunction( - funcConfig.getTenant(), - funcConfig.getNamespace(), - funcConfig.getName(), - null, - mockedFormData, - null, - funcConfig, - null, - updateOptions); - } - - - private void registerDefaultFunction() { - registerDefaultFunctionWithPackageUrl(null); - } - - private void registerDefaultFunctionWithPackageUrl(String packageUrl) { - FunctionConfig functionConfig = createDefaultFunctionConfig(); - resource.registerFunction( - tenant, - namespace, - function, - mockedInputStream, - mockedFormData, - packageUrl, - functionConfig, - null); - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function already exists") - public void testRegisterExistedFunction() { - try { - Configurator.setRootLevel(Level.DEBUG); - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "upload failure") - public void testRegisterFunctionUploadFailure() throws Exception { - try { - mockWorkerUtils(ctx -> { - ctx.when(() -> { - WorkerUtils.uploadFileToBookkeeper( - anyString(), - any(File.class), - any(Namespace.class)); - } - ).thenThrow(new IOException("upload failure")); - ; - }); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); - throw re; - } - } - - @Test - public void testRegisterFunctionSuccess() throws Exception { - try { - mockWorkerUtils(); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(timeOut = 20000) - public void testRegisterFunctionSuccessWithPackageName() { - registerDefaultFunctionWithPackageUrl("function://public/default/test@v1"); - } - - @Test(timeOut = 20000) - public void testRegisterFunctionFailedWithWrongPackageName() throws PulsarAdminException { - try { - doThrow(new PulsarAdminException("package name is invalid")) - .when(mockedPackages).download(anyString(), anyString()); - registerDefaultFunctionWithPackageUrl("function://"); - } catch (RestException e) { - // expected exception - assertEquals(e.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace does not exist") - public void testRegisterFunctionNonExistingNamespace() { - try { - this.namespaceList.clear(); - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant does not exist") - public void testRegisterFunctionNonexistantTenant() throws Exception { - try { - when(mockedTenants.getTenantInfo(any())).thenThrow(PulsarAdminException.NotFoundException.class); - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "function failed to register") - public void testRegisterFunctionFailure() throws Exception { - try { - mockWorkerUtils(); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - - doThrow(new IllegalArgumentException("function failed to register")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); - - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function registration interrupted") - public void testRegisterFunctionInterrupted() throws Exception { - try { - mockWorkerUtils(); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - - doThrow(new IllegalStateException("Function registration interrupted")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); - - registerDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); - throw re; - } - } - - /* - Externally managed runtime, - uploadBuiltinSinksSources == false - Make sure uploadFileToBookkeeper is not called - */ - @Test - public void testRegisterFunctionSuccessK8sNoUpload() throws Exception { - mockedWorkerService.getWorkerConfig().setUploadBuiltinSinksSources(false); - - mockStatic(WorkerUtils.class, ctx -> { - ctx.when(() -> WorkerUtils.uploadFileToBookkeeper( - anyString(), - any(File.class), - any(Namespace.class))) - .thenThrow(new RuntimeException("uploadFileToBookkeeper triggered")); - - }); - - NarClassLoader mockedClassLoader = mock(NarClassLoader.class); - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.getFunctionTypes(any(FunctionConfig.class), any(Class.class))).thenReturn(new Class[]{String.class, String.class}); - ctx.when(() -> FunctionCommon.convertRuntime(any(FunctionConfig.Runtime.class))).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.isFunctionCodeBuiltin(any())).thenReturn(true); - - }); - - doReturn(Function.class).when(mockedClassLoader).loadClass(anyString()); - - FunctionsManager mockedFunctionsManager = mock(FunctionsManager.class); - FunctionArchive functionArchive = FunctionArchive.builder() - .classLoader(mockedClassLoader) - .build(); - when(mockedFunctionsManager.getFunction("exclamation")).thenReturn(functionArchive); - when(mockedFunctionsManager.getFunctionArchive(any())).thenReturn(getPulsarApiExamplesNar().toPath()); - - when(mockedWorkerService.getFunctionsManager()).thenReturn(mockedFunctionsManager); - - when(mockedRuntimeFactory.externallyManaged()).thenReturn(true); - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - - FunctionConfig functionConfig = createDefaultFunctionConfig(); - functionConfig.setJar("builtin://exclamation"); - - try (FileInputStream inputStream = new FileInputStream(getPulsarApiExamplesNar())) { - resource.registerFunction( - tenant, - namespace, - function, - inputStream, - mockedFormData, - null, - functionConfig, - null); - } - } - - /* - Externally managed runtime, - uploadBuiltinSinksSources == true - Make sure uploadFileToBookkeeper is called - */ - @Test - public void testRegisterFunctionSuccessK8sWithUpload() throws Exception { - final String injectedErrMsg = "uploadFileToBookkeeper triggered"; - mockedWorkerService.getWorkerConfig().setUploadBuiltinSinksSources(true); - - mockStatic(WorkerUtils.class, ctx -> { - ctx.when(() -> WorkerUtils.uploadFileToBookkeeper( - anyString(), - any(File.class), - any(Namespace.class))) - .thenThrow(new RuntimeException(injectedErrMsg)); - - }); - - NarClassLoader mockedClassLoader = mock(NarClassLoader.class); - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.getFunctionTypes(any(FunctionConfig.class), any(Class.class))).thenReturn(new Class[]{String.class, String.class}); - ctx.when(() -> FunctionCommon.convertRuntime(any(FunctionConfig.Runtime.class))).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.isFunctionCodeBuiltin(any())).thenReturn(true); - }); - - doReturn(Function.class).when(mockedClassLoader).loadClass(anyString()); - - FunctionsManager mockedFunctionsManager = mock(FunctionsManager.class); - FunctionArchive functionArchive = FunctionArchive.builder() - .classLoader(mockedClassLoader) - .build(); - when(mockedFunctionsManager.getFunction("exclamation")).thenReturn(functionArchive); - when(mockedFunctionsManager.getFunctionArchive(any())).thenReturn(getPulsarApiExamplesNar().toPath()); - - when(mockedWorkerService.getFunctionsManager()).thenReturn(mockedFunctionsManager); - - when(mockedRuntimeFactory.externallyManaged()).thenReturn(true); - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - - FunctionConfig functionConfig = createDefaultFunctionConfig(); - functionConfig.setJar("builtin://exclamation"); - - try (FileInputStream inputStream = new FileInputStream(getPulsarApiExamplesNar())) { - try { - resource.registerFunction( - tenant, - namespace, - function, - inputStream, - mockedFormData, - null, - functionConfig, - null); - Assert.fail(); - } catch (RuntimeException e) { - Assert.assertEquals(e.getMessage(), injectedErrMsg); - } - } - } - - // - // Update Functions - // - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") - public void testUpdateFunctionMissingTenant() throws Exception { - try { - testUpdateFunctionMissingArguments( - null, - namespace, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "Tenant is not provided"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") - public void testUpdateFunctionMissingNamespace() throws Exception { - try { - testUpdateFunctionMissingArguments( - tenant, - null, - function, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "Namespace is not provided"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") - public void testUpdateFunctionMissingFunctionName() throws Exception { - try { - testUpdateFunctionMissingArguments( - tenant, - namespace, - null, - mockedInputStream, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "Function name is not provided"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Update contains no change") - public void testUpdateFunctionMissingPackage() throws Exception { - try { - mockWorkerUtils(); - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "Update contains no change"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Update contains no change") - public void testUpdateFunctionMissingInputTopic() throws Exception { - try { - mockWorkerUtils(); - - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - null, - mockedFormData, - outputTopic, - outputSerdeClassName, - className, - parallelism, - "Update contains no change"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Update contains no change") - public void testUpdateFunctionMissingClassName() throws Exception { - try { - mockWorkerUtils(); - - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - null, - parallelism, - "Update contains no change"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test - public void testUpdateFunctionChangedParallelism() throws Exception { - try { - mockWorkerUtils(); - - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - mockedFormData, - outputTopic, - outputSerdeClassName, - null, - parallelism + 1, - null); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test - public void testUpdateFunctionChangedInputs() throws Exception { - mockWorkerUtils(); - - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - topicsToSerDeClassName, - mockedFormData, - "DifferentOutput", - outputSerdeClassName, - null, - parallelism, - null); - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Input Topics cannot be altered") - public void testUpdateFunctionChangedOutput() throws Exception { - try { - mockWorkerUtils(); - - Map someOtherInput = new HashMap<>(); - someOtherInput.put("DifferentTopic", TopicSchema.DEFAULT_SERDE); - testUpdateFunctionMissingArguments( - tenant, - namespace, - function, - null, - someOtherInput, - mockedFormData, - outputTopic, - outputSerdeClassName, - null, - parallelism, - "Input Topics cannot be altered"); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - private void testUpdateFunctionMissingArguments( - String tenant, - String namespace, - String function, - InputStream inputStream, - Map topicsToSerDeClassName, - FormDataContentDisposition details, - String outputTopic, - String outputSerdeClassName, - String className, - Integer parallelism, - String expectedError) throws Exception { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - FunctionConfig functionConfig = new FunctionConfig(); - if (tenant != null) { - functionConfig.setTenant(tenant); - } - if (namespace != null) { - functionConfig.setNamespace(namespace); - } - if (function != null) { - functionConfig.setName(function); - } - if (topicsToSerDeClassName != null) { - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - } - if (outputTopic != null) { - functionConfig.setOutput(outputTopic); - } - if (outputSerdeClassName != null) { - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - } - if (className != null) { - functionConfig.setClassName(className); - } - if (parallelism != null) { - functionConfig.setParallelism(parallelism); - } - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - - if (expectedError != null) { - doThrow(new IllegalArgumentException(expectedError)) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); - } - - resource.updateFunction( - tenant, - namespace, - function, - inputStream, - details, - null, - functionConfig, - null, null); - - } - - private void updateDefaultFunction() { - updateDefaultFunctionWithPackageUrl(null); - } - - private void updateDefaultFunctionWithPackageUrl(String packageUrl) { - FunctionConfig functionConfig = new FunctionConfig(); - functionConfig.setTenant(tenant); - functionConfig.setNamespace(namespace); - functionConfig.setName(function); - functionConfig.setClassName(className); - functionConfig.setParallelism(parallelism); - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - functionConfig.setOutput(outputTopic); - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - - resource.updateFunction( - tenant, - namespace, - function, - mockedInputStream, - mockedFormData, - packageUrl, - functionConfig, - null, null); - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function doesn't exist") - public void testUpdateNotExistedFunction() { - try { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - updateDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "upload failure") - public void testUpdateFunctionUploadFailure() throws Exception { - try { - mockWorkerUtils(ctx -> { - ctx.when(() -> { - WorkerUtils.uploadFileToBookkeeper( - anyString(), - any(File.class), - any(Namespace.class)); - - }).thenThrow(new IOException("upload failure")); - ctx.when(() -> WorkerUtils.dumpToTmpFile(any())).thenCallRealMethod(); - }); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - updateDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); - throw re; - } - } - - @Test - public void testUpdateFunctionSuccess() throws Exception { - mockWorkerUtils(); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - updateDefaultFunction(); - } - - @Test - public void testUpdateFunctionWithUrl() { - Configurator.setRootLevel(Level.DEBUG); - - String fileLocation = FutureUtil.class.getProtectionDomain().getCodeSource().getLocation().getPath(); - String filePackageUrl = "file://" + fileLocation; - - FunctionConfig functionConfig = new FunctionConfig(); - functionConfig.setOutput(outputTopic); - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - functionConfig.setTenant(tenant); - functionConfig.setNamespace(namespace); - functionConfig.setName(function); - functionConfig.setClassName(className); - functionConfig.setParallelism(parallelism); - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - resource.updateFunction( - tenant, - namespace, - function, - null, - null, - filePackageUrl, - functionConfig, - null, null); - - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "function failed to register") - public void testUpdateFunctionFailure() throws Exception { - try { - mockWorkerUtils(); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - doThrow(new IllegalArgumentException("function failed to register")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); - - updateDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function registeration interrupted") - public void testUpdateFunctionInterrupted() throws Exception { - try { - mockWorkerUtils(); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - doThrow(new IllegalStateException("Function registeration interrupted")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); - - updateDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); - throw re; - } - } - - - @Test(timeOut = 20000) - public void testUpdateFunctionSuccessWithPackageName() { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - updateDefaultFunctionWithPackageUrl("function://public/default/test@v1"); - } - - @Test(timeOut = 20000) - public void testUpdateFunctionFailedWithWrongPackageName() throws PulsarAdminException { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - try { - doThrow(new PulsarAdminException("package name is invalid")) - .when(mockedPackages).download(anyString(), anyString()); - registerDefaultFunctionWithPackageUrl("function://"); - } catch (RestException e) { - // expected exception - assertEquals(e.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - } - } - - // - // deregister function - // - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") - public void testDeregisterFunctionMissingTenant() { - try { - - testDeregisterFunctionMissingArguments( - null, - namespace, - function - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") - public void testDeregisterFunctionMissingNamespace() { - try { - testDeregisterFunctionMissingArguments( - tenant, - null, - function - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") - public void testDeregisterFunctionMissingFunctionName() { - try { - testDeregisterFunctionMissingArguments( - tenant, - namespace, - null - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - private void testDeregisterFunctionMissingArguments( - String tenant, - String namespace, - String function - ) { - resource.deregisterFunction( - tenant, - namespace, - function, - null); - } - - private void deregisterDefaultFunction() { - resource.deregisterFunction( - tenant, - namespace, - function, - null); - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function doesn't exist") - public void testDeregisterNotExistedFunction() { - try { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - deregisterDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.NOT_FOUND); - throw re; - } - } - - @Test - public void testDeregisterFunctionSuccess() { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - deregisterDefaultFunction(); - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "function failed to deregister") - public void testDeregisterFunctionFailure() throws Exception { - try { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - doThrow(new IllegalArgumentException("function failed to deregister")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); - - deregisterDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function deregisteration interrupted") - public void testDeregisterFunctionInterrupted() throws Exception { - try { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - doThrow(new IllegalStateException("Function deregisteration interrupted")) - .when(mockedManager).updateFunctionOnLeader(any(FunctionMetaData.class), Mockito.anyBoolean()); - - deregisterDefaultFunction(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); - throw re; - } - } - - // - // Get Function Info - // - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") - public void testGetFunctionMissingTenant() { - try { - testGetFunctionMissingArguments( - null, - namespace, - function - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") - public void testGetFunctionMissingNamespace() { - try { - testGetFunctionMissingArguments( - tenant, - null, - function - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function name is not provided") - public void testGetFunctionMissingFunctionName() { - try { - testGetFunctionMissingArguments( - tenant, - namespace, - null - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } - } - - private void testGetFunctionMissingArguments( - String tenant, - String namespace, - String function - ) { - resource.getFunctionInfo( - tenant, - namespace, - function,null - ); - - } - - private FunctionConfig getDefaultFunctionInfo() { - return resource.getFunctionInfo( - tenant, - namespace, - function, - null - ); - } - - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function doesn't exist") - public void testGetNotExistedFunction() { - try { - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - getDefaultFunctionInfo(); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.NOT_FOUND); - throw re; - } - } - - @Test - public void testGetFunctionSuccess() { - mockInstanceUtils(); - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - - SinkSpec sinkSpec = SinkSpec.newBuilder() - .setTopic(outputTopic) - .setSerDeClassName(outputSerdeClassName).build(); - FunctionDetails functionDetails = FunctionDetails.newBuilder() - .setClassName(className) - .setSink(sinkSpec) - .setAutoAck(true) - .setName(function) - .setNamespace(namespace) - .setProcessingGuarantees(ProcessingGuarantees.ATMOST_ONCE) - .setTenant(tenant) - .setParallelism(parallelism) - .setSource(SourceSpec.newBuilder().setSubscriptionType(subscriptionType) - .putAllTopicsToSerDeClassName(topicsToSerDeClassName)).build(); - FunctionMetaData metaData = FunctionMetaData.newBuilder() - .setCreateTime(System.currentTimeMillis()) - .setFunctionDetails(functionDetails) - .setPackageLocation(PackageLocationMetaData.newBuilder().setPackagePath("/path/to/package")) - .setVersion(1234) - .build(); - when(mockedManager.getFunctionMetaData(eq(tenant), eq(namespace), eq(function))).thenReturn(metaData); - - FunctionConfig functionConfig = getDefaultFunctionInfo(); - assertEquals( - FunctionConfigUtils.convertFromDetails(functionDetails), - functionConfig); - } - - // - // List Functions - // + protected void registerFunction(String tenant, String namespace, String function, InputStream inputStream, + FormDataContentDisposition details, String functionPkgUrl, FunctionConfig functionConfig) { + resource.registerFunction( + tenant, + namespace, + function, + inputStream, + details, + functionPkgUrl, + functionConfig, + null); + } + protected void updateFunction(String tenant, + String namespace, + String functionName, + InputStream uploadedInputStream, + FormDataContentDisposition fileDetail, + String functionPkgUrl, + FunctionConfig functionConfig, + AuthenticationParameters authParams, + UpdateOptionsImpl updateOptions) { + resource.updateFunction(tenant, namespace, functionName, uploadedInputStream, fileDetail, functionPkgUrl, + functionConfig, authParams, updateOptions); + } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") - public void testListFunctionsMissingTenant() { - try { - testListFunctionsMissingArguments( - null, - namespace - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; - } + protected StreamingOutput downloadFunction(String tenant, String namespace, String componentName, + AuthenticationParameters authParams, boolean transformFunction) { + return resource.downloadFunction(tenant, namespace, componentName, authParams, transformFunction); } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Namespace is not provided") - public void testListFunctionsMissingNamespace() { - try { - testListFunctionsMissingArguments( - tenant, - null - ); - } catch (RestException re) { - assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); - throw re; + protected File downloadFunction(final String path, final AuthenticationParameters authParams) throws IOException { + StreamingOutput streamingOutput = resource.downloadFunction(path, authParams); + File pkgFile = File.createTempFile("testpkg", "nar"); + try(OutputStream output = new FileOutputStream(pkgFile)) { + streamingOutput.write(output); } + return pkgFile; } - private void testListFunctionsMissingArguments( + protected void testDeregisterFunctionMissingArguments( String tenant, - String namespace + String namespace, + String function ) { - resource.listFunctions( - tenant, - namespace,null - ); - - } - - private List listDefaultFunctions() { - return resource.listFunctions( - tenant, - namespace,null - ); - } - - @Test - public void testListFunctionsSuccess() { - mockInstanceUtils(); - final List functions = Lists.newArrayList("test-1", "test-2"); - final List metaDataList = new LinkedList<>(); - FunctionMetaData functionMetaData1 = FunctionMetaData.newBuilder().setFunctionDetails( - FunctionDetails.newBuilder().setName("test-1").build() - ).build(); - FunctionMetaData functionMetaData2 = FunctionMetaData.newBuilder().setFunctionDetails( - FunctionDetails.newBuilder().setName("test-2").build() - ).build(); - metaDataList.add(functionMetaData1); - metaDataList.add(functionMetaData2); - when(mockedManager.listFunctions(eq(tenant), eq(namespace))).thenReturn(metaDataList); - - List functionList = listDefaultFunctions(); - assertEquals(functions, functionList); + resource.deregisterFunction( + tenant, + namespace, + function, + null); } - @Test - public void testOnlyGetSources() { - List functions = Lists.newArrayList("test-2"); - List functionMetaDataList = new LinkedList<>(); - FunctionMetaData f1 = FunctionMetaData.newBuilder().setFunctionDetails( - FunctionDetails.newBuilder() - .setName("test-1") - .setComponentType(FunctionDetails.ComponentType.SOURCE) - .build()).build(); - functionMetaDataList.add(f1); - FunctionMetaData f2 = FunctionMetaData.newBuilder().setFunctionDetails( - FunctionDetails.newBuilder() - .setName("test-2") - .setComponentType(FunctionDetails.ComponentType.FUNCTION) - .build()).build(); - functionMetaDataList.add(f2); - FunctionMetaData f3 = FunctionMetaData.newBuilder().setFunctionDetails( - FunctionDetails.newBuilder() - .setName("test-3") - .setComponentType(FunctionDetails.ComponentType.SINK) - .build()).build(); - functionMetaDataList.add(f3); - when(mockedManager.listFunctions(eq(tenant), eq(namespace))).thenReturn(functionMetaDataList); - - List functionList = listDefaultFunctions(); - assertEquals(functions, functionList); + protected void deregisterDefaultFunction() { + resource.deregisterFunction( + tenant, + namespace, + function, + null); } - @Test - public void testDownloadFunctionHttpUrl() throws Exception { - String jarHttpUrl = - "https://repo1.maven.org/maven2/org/apache/pulsar/pulsar-common/2.4.2/pulsar-common-2.4.2.jar"; - String testDir = FunctionApiV3ResourceTest.class.getProtectionDomain().getCodeSource().getLocation().getPath(); + protected void testGetFunctionMissingArguments( + String tenant, + String namespace, + String function + ) { + resource.getFunctionInfo( + tenant, + namespace, + function, null + ); - StreamingOutput streamOutput = resource.downloadFunction(jarHttpUrl, null); - File pkgFile = new File(testDir, UUID.randomUUID().toString()); - OutputStream output = new FileOutputStream(pkgFile); - streamOutput.write(output); - Assert.assertTrue(pkgFile.exists()); - pkgFile.delete(); } - @Test - public void testDownloadFunctionFile() throws Exception { - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String fileLocation = file.getAbsolutePath().replace('\\', '/'); - String testDir = FunctionApiV3ResourceTest.class.getProtectionDomain().getCodeSource().getLocation().getPath(); - - StreamingOutput streamOutput = resource.downloadFunction("file:///" + fileLocation, null); - File pkgFile = new File(testDir, UUID.randomUUID().toString()); - OutputStream output = new FileOutputStream(pkgFile); - streamOutput.write(output); - Assert.assertTrue(pkgFile.exists()); - Assert.assertEquals(file.length(), pkgFile.length()); - pkgFile.delete(); + protected FunctionConfig getDefaultFunctionInfo() { + return resource.getFunctionInfo( + tenant, + namespace, + function, + null + ); } - @Test - public void testDownloadFunctionBuiltinConnector() throws Exception { - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String testDir = FunctionApiV3ResourceTest.class.getProtectionDomain().getCodeSource().getLocation().getPath(); - - WorkerConfig config = new WorkerConfig() - .setUploadBuiltinSinksSources(false); - when(mockedWorkerService.getWorkerConfig()).thenReturn(config); - - Connector connector = Connector.builder().archivePath(file.toPath()).build(); - ConnectorsManager connectorsManager = mock(ConnectorsManager.class); - when(connectorsManager.getConnector("cassandra")).thenReturn(connector); - when(mockedWorkerService.getConnectorsManager()).thenReturn(connectorsManager); - - StreamingOutput streamOutput = resource.downloadFunction("builtin://cassandra", null); + protected void testListFunctionsMissingArguments( + String tenant, + String namespace + ) { + resource.listFunctions( + tenant, + namespace, null + ); - File pkgFile = new File(testDir, UUID.randomUUID().toString()); - OutputStream output = new FileOutputStream(pkgFile); - streamOutput.write(output); - output.flush(); - output.close(); - Assert.assertTrue(pkgFile.exists()); - Assert.assertTrue(pkgFile.exists()); - Assert.assertEquals(file.length(), pkgFile.length()); - pkgFile.delete(); } - @Test - public void testDownloadFunctionBuiltinFunction() throws Exception { - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String testDir = FunctionApiV3ResourceTest.class.getProtectionDomain().getCodeSource().getLocation().getPath(); - - WorkerConfig config = new WorkerConfig() - .setUploadBuiltinSinksSources(false); - when(mockedWorkerService.getWorkerConfig()).thenReturn(config); - - FunctionsManager functionsManager = mock(FunctionsManager.class); - FunctionArchive functionArchive = FunctionArchive.builder().archivePath(file.toPath()).build(); - when(functionsManager.getFunction("exclamation")).thenReturn(functionArchive); - when(mockedWorkerService.getConnectorsManager()).thenReturn(mock(ConnectorsManager.class)); - when(mockedWorkerService.getFunctionsManager()).thenReturn(functionsManager); - - StreamingOutput streamOutput = resource.downloadFunction("builtin://exclamation", null); - - File pkgFile = new File(testDir, UUID.randomUUID().toString()); - OutputStream output = new FileOutputStream(pkgFile); - streamOutput.write(output); - output.flush(); - output.close(); - Assert.assertTrue(pkgFile.exists()); - Assert.assertEquals(file.length(), pkgFile.length()); - pkgFile.delete(); + protected List listDefaultFunctions() { + return resource.listFunctions( + tenant, + namespace, null + ); } @Test public void testDownloadFunctionBuiltinConnectorByName() throws Exception { - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String testDir = FunctionApiV3ResourceTest.class.getProtectionDomain().getCodeSource().getLocation().getPath(); + File file = getPulsarApiExamplesNar(); WorkerConfig config = new WorkerConfig() - .setUploadBuiltinSinksSources(false); + .setUploadBuiltinSinksSources(false); when(mockedWorkerService.getWorkerConfig()).thenReturn(config); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - FunctionMetaData metaData = FunctionMetaData.newBuilder() - .setPackageLocation(PackageLocationMetaData.newBuilder().setPackagePath("builtin://cassandra")) - .setTransformFunctionPackageLocation(PackageLocationMetaData.newBuilder().setPackagePath("http://invalid")) - .setFunctionDetails(FunctionDetails.newBuilder().setComponentType(FunctionDetails.ComponentType.SINK)) + Function.FunctionMetaData metaData = Function.FunctionMetaData.newBuilder() + .setPackageLocation(Function.PackageLocationMetaData.newBuilder().setPackagePath("builtin://cassandra")) + .setTransformFunctionPackageLocation( + Function.PackageLocationMetaData.newBuilder().setPackagePath("http://invalid")) + .setFunctionDetails(Function.FunctionDetails.newBuilder().setComponentType(Function.FunctionDetails.ComponentType.SINK)) .build(); when(mockedManager.getFunctionMetaData(eq(tenant), eq(namespace), eq(function))).thenReturn(metaData); - Connector connector = Connector.builder().archivePath(file.toPath()).build(); - ConnectorsManager connectorsManager = mock(ConnectorsManager.class); - when(connectorsManager.getConnector("cassandra")).thenReturn(connector); - when(mockedWorkerService.getConnectorsManager()).thenReturn(connectorsManager); + registerBuiltinConnector("cassandra", file); - StreamingOutput streamOutput = resource.downloadFunction(tenant, namespace, function, + StreamingOutput streamOutput = downloadFunction(tenant, namespace, function, AuthenticationParameters.builder().build(), false); - File pkgFile = new File(testDir, UUID.randomUUID().toString()); + File pkgFile = File.createTempFile("testpkg", "nar"); OutputStream output = new FileOutputStream(pkgFile); streamOutput.write(output); Assert.assertTrue(pkgFile.exists()); @@ -1812,31 +189,27 @@ public void testDownloadFunctionBuiltinConnectorByName() throws Exception { @Test public void testDownloadFunctionBuiltinFunctionByName() throws Exception { - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String testDir = FunctionApiV3ResourceTest.class.getProtectionDomain().getCodeSource().getLocation().getPath(); + File file = getPulsarApiExamplesNar(); WorkerConfig config = new WorkerConfig() - .setUploadBuiltinSinksSources(false); + .setUploadBuiltinSinksSources(false); when(mockedWorkerService.getWorkerConfig()).thenReturn(config); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - FunctionMetaData metaData = FunctionMetaData.newBuilder() - .setPackageLocation(PackageLocationMetaData.newBuilder().setPackagePath("builtin://exclamation")) - .setTransformFunctionPackageLocation(PackageLocationMetaData.newBuilder().setPackagePath("http://invalid")) - .setFunctionDetails(FunctionDetails.newBuilder().setComponentType(FunctionDetails.ComponentType.FUNCTION)) - .build(); + Function.FunctionMetaData metaData = Function.FunctionMetaData.newBuilder() + .setPackageLocation(Function.PackageLocationMetaData.newBuilder().setPackagePath("builtin://exclamation")) + .setTransformFunctionPackageLocation( + Function.PackageLocationMetaData.newBuilder().setPackagePath("http://invalid")) + .setFunctionDetails( + Function.FunctionDetails.newBuilder().setComponentType(Function.FunctionDetails.ComponentType.FUNCTION)) + .build(); when(mockedManager.getFunctionMetaData(eq(tenant), eq(namespace), eq(function))).thenReturn(metaData); - FunctionsManager functionsManager = mock(FunctionsManager.class); - FunctionArchive functionArchive = FunctionArchive.builder().archivePath(file.toPath()).build(); - when(functionsManager.getFunction("exclamation")).thenReturn(functionArchive); - when(mockedWorkerService.getConnectorsManager()).thenReturn(mock(ConnectorsManager.class)); - when(mockedWorkerService.getFunctionsManager()).thenReturn(functionsManager); + registerBuiltinFunction("exclamation", file); - StreamingOutput streamOutput = resource.downloadFunction(tenant, namespace, function, + StreamingOutput streamOutput = downloadFunction(tenant, namespace, function, AuthenticationParameters.builder().build(), false); - File pkgFile = new File(testDir, UUID.randomUUID().toString()); + File pkgFile = File.createTempFile("testpkg", "nar"); OutputStream output = new FileOutputStream(pkgFile); streamOutput.write(output); Assert.assertTrue(pkgFile.exists()); @@ -1846,32 +219,26 @@ public void testDownloadFunctionBuiltinFunctionByName() throws Exception { @Test public void testDownloadTransformFunctionByName() throws Exception { - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String testDir = FunctionApiV3ResourceTest.class.getProtectionDomain().getCodeSource().getLocation().getPath(); + File file = getPulsarApiExamplesNar(); WorkerConfig workerConfig = new WorkerConfig() - .setUploadBuiltinSinksSources(false); + .setUploadBuiltinSinksSources(false); when(mockedWorkerService.getWorkerConfig()).thenReturn(workerConfig); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - FunctionMetaData metaData = FunctionMetaData.newBuilder() - .setPackageLocation(PackageLocationMetaData.newBuilder().setPackagePath("http://invalid")) - .setTransformFunctionPackageLocation(PackageLocationMetaData.newBuilder() + Function.FunctionMetaData metaData = Function.FunctionMetaData.newBuilder() + .setPackageLocation(Function.PackageLocationMetaData.newBuilder().setPackagePath("http://invalid")) + .setTransformFunctionPackageLocation(Function.PackageLocationMetaData.newBuilder() .setPackagePath("builtin://exclamation")) .build(); when(mockedManager.getFunctionMetaData(eq(tenant), eq(namespace), eq(function))).thenReturn(metaData); - FunctionsManager functionsManager = mock(FunctionsManager.class); - FunctionArchive functionArchive = FunctionArchive.builder().archivePath(file.toPath()).build(); - when(functionsManager.getFunction("exclamation")).thenReturn(functionArchive); - when(mockedWorkerService.getConnectorsManager()).thenReturn(mock(ConnectorsManager.class)); - when(mockedWorkerService.getFunctionsManager()).thenReturn(functionsManager); + registerBuiltinFunction("exclamation", file); - StreamingOutput streamOutput = resource.downloadFunction(tenant, namespace, function, + StreamingOutput streamOutput = downloadFunction(tenant, namespace, function, AuthenticationParameters.builder().build(), true); - File pkgFile = new File(testDir, UUID.randomUUID().toString()); + File pkgFile = File.createTempFile("testpkg", "nar"); OutputStream output = new FileOutputStream(pkgFile); streamOutput.write(output); Assert.assertTrue(pkgFile.exists()); @@ -1879,15 +246,58 @@ public void testDownloadTransformFunctionByName() throws Exception { pkgFile.delete(); } + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function test-function doesn't" + + " exist") + public void testGetNotExistedFunction() throws IOException { + try { + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); + getDefaultFunctionInfo(); + } catch (RestException re) { + assertEquals(re.getResponse().getStatusInfo(), Response.Status.NOT_FOUND); + throw re; + } + } @Test - public void testRegisterFunctionFileUrlWithValidSinkClass() throws Exception { + public void testGetFunctionSuccess() throws IOException { + mockInstanceUtils(); + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); + + Function.SinkSpec sinkSpec = Function.SinkSpec.newBuilder() + .setTopic(outputTopic) + .setSerDeClassName(outputSerdeClassName).build(); + Function.FunctionDetails functionDetails = Function.FunctionDetails.newBuilder() + .setClassName(className) + .setSink(sinkSpec) + .setAutoAck(true) + .setName(function) + .setNamespace(namespace) + .setProcessingGuarantees(Function.ProcessingGuarantees.ATMOST_ONCE) + .setTenant(tenant) + .setParallelism(parallelism) + .setSource(Function.SourceSpec.newBuilder().setSubscriptionType(subscriptionType) + .putAllTopicsToSerDeClassName(topicsToSerDeClassName)).build(); + Function.FunctionMetaData metaData = Function.FunctionMetaData.newBuilder() + .setCreateTime(System.currentTimeMillis()) + .setFunctionDetails(functionDetails) + .setPackageLocation(Function.PackageLocationMetaData.newBuilder().setPackagePath("/path/to/package")) + .setVersion(1234) + .build(); + when(mockedManager.getFunctionMetaData(eq(tenant), eq(namespace), eq(function))).thenReturn(metaData); + + FunctionConfig functionConfig = getDefaultFunctionInfo(); + assertEquals( + FunctionConfigUtils.convertFromDetails(functionDetails), + functionConfig); + } + + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function language runtime is " + + "either not set or cannot be determined") + public void testCreateFunctionWithoutSettingRuntime() throws Exception { Configurator.setRootLevel(Level.DEBUG); - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String fileLocation = file.getAbsolutePath().replace('\\', '/'); - String filePackageUrl = "file:///" + fileLocation; + File file = getPulsarApiExamplesNar(); + String filePackageUrl = file.toURI().toString(); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); FunctionConfig functionConfig = new FunctionConfig(); @@ -1896,82 +306,135 @@ public void testRegisterFunctionFileUrlWithValidSinkClass() throws Exception { functionConfig.setName(function); functionConfig.setClassName(className); functionConfig.setParallelism(parallelism); - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); functionConfig.setOutput(outputTopic); functionConfig.setOutputSerdeClassName(outputSerdeClassName); - resource.registerFunction(tenant, namespace, function, null, null, filePackageUrl, functionConfig, null); + registerFunction(tenant, namespace, function, null, null, filePackageUrl, functionConfig); } @Test - public void testRegisterFunctionWithConflictingFields() throws Exception { - Configurator.setRootLevel(Level.DEBUG); - String actualTenant = "DIFFERENT_TENANT"; - String actualNamespace = "DIFFERENT_NAMESPACE"; - String actualName = "DIFFERENT_NAME"; - this.namespaceList.add(actualTenant + "/" + actualNamespace); + public void testUpdateSourceWithNoChange() throws IOException { + mockWorkerUtils(); - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String fileLocation = file.getAbsolutePath().replace('\\', '/'); - String filePackageUrl = "file:///" + fileLocation; when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(true); - when(mockedManager.containsFunction(eq(actualTenant), eq(actualNamespace), eq(actualName))).thenReturn(false); + FunctionConfig funcConfig = createDefaultFunctionConfig(); - FunctionConfig functionConfig = new FunctionConfig(); - functionConfig.setTenant(tenant); - functionConfig.setNamespace(namespace); - functionConfig.setName(function); - functionConfig.setClassName(className); - functionConfig.setParallelism(parallelism); - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - functionConfig.setOutput(outputTopic); - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - resource.registerFunction(actualTenant, actualNamespace, actualName, null, null, filePackageUrl, functionConfig, - null); + // config has not changes and don't update auth, should fail + try { + updateFunction( + funcConfig.getTenant(), + funcConfig.getNamespace(), + funcConfig.getName(), + null, + mockedFormData, + null, + funcConfig, + null, + null); + fail("Update without changes should fail"); + } catch (RestException e) { + assertThat(e.getMessage()).contains("Update contains no change"); + } + + try { + UpdateOptionsImpl updateOptions = new UpdateOptionsImpl(); + updateOptions.setUpdateAuthData(false); + updateFunction( + funcConfig.getTenant(), + funcConfig.getNamespace(), + funcConfig.getName(), + null, + mockedFormData, + null, + funcConfig, + null, + updateOptions); + fail("Update without changes should fail"); + } catch (RestException e) { + assertTrue(e.getMessage().contains("Update contains no change")); + } + + // no changes but set the auth-update flag to true, should not fail + UpdateOptionsImpl updateOptions = new UpdateOptionsImpl(); + updateOptions.setUpdateAuthData(true); + updateFunction( + funcConfig.getTenant(), + funcConfig.getNamespace(), + funcConfig.getName(), + null, + mockedFormData, + null, + funcConfig, + null, + updateOptions); } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function language runtime is either not set or cannot be determined") - public void testCreateFunctionWithoutSettingRuntime() throws Exception { - Configurator.setRootLevel(Level.DEBUG); + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Function config is not provided") + public void testMissingFunctionConfig() throws IOException { + registerFunction(tenant, namespace, function, mockedInputStream, mockedFormData, null, null); + } + + /* + Externally managed runtime, + uploadBuiltinSinksSources == false + Make sure uploadFileToBookkeeper is not called + */ + @Test + public void testRegisterFunctionSuccessK8sNoUpload() throws Exception { + mockedWorkerService.getWorkerConfig().setUploadBuiltinSinksSources(false); + + mockStatic(WorkerUtils.class, ctx -> { + ctx.when(() -> WorkerUtils.uploadFileToBookkeeper( + anyString(), + any(File.class), + any(Namespace.class))) + .thenThrow(new RuntimeException("uploadFileToBookkeeper triggered")); + + }); - URL fileUrl = getClass().getClassLoader().getResource("test_worker_config.yml"); - File file = Paths.get(fileUrl.toURI()).toFile(); - String fileLocation = file.getAbsolutePath().replace('\\', '/'); - String filePackageUrl = "file:///" + fileLocation; + registerBuiltinFunction("exclamation", getPulsarApiExamplesNar()); + when(mockedRuntimeFactory.externallyManaged()).thenReturn(true); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - FunctionConfig functionConfig = new FunctionConfig(); - functionConfig.setTenant(tenant); - functionConfig.setNamespace(namespace); - functionConfig.setName(function); - functionConfig.setClassName(className); - functionConfig.setParallelism(parallelism); - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - functionConfig.setOutput(outputTopic); - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - resource.registerFunction(tenant, namespace, function, null, null, filePackageUrl, functionConfig, null); + FunctionConfig functionConfig = createDefaultFunctionConfig(); + functionConfig.setJar("builtin://exclamation"); + registerFunction(tenant, namespace, function, null, mockedFormData, null, functionConfig); } - public static FunctionConfig createDefaultFunctionConfig() { - FunctionConfig functionConfig = new FunctionConfig(); - functionConfig.setTenant(tenant); - functionConfig.setNamespace(namespace); - functionConfig.setName(function); - functionConfig.setClassName(className); - functionConfig.setParallelism(parallelism); - functionConfig.setCustomSerdeInputs(topicsToSerDeClassName); - functionConfig.setOutput(outputTopic); - functionConfig.setOutputSerdeClassName(outputSerdeClassName); - functionConfig.setRuntime(FunctionConfig.Runtime.JAVA); - return functionConfig; - } + /* + Externally managed runtime, + uploadBuiltinSinksSources == true + Make sure uploadFileToBookkeeper is called + */ + @Test + public void testRegisterFunctionSuccessK8sWithUpload() throws Exception { + final String injectedErrMsg = "uploadFileToBookkeeper triggered"; + mockedWorkerService.getWorkerConfig().setUploadBuiltinSinksSources(true); + + mockStatic(WorkerUtils.class, ctx -> { + ctx.when(() -> WorkerUtils.uploadFileToBookkeeper( + anyString(), + any(File.class), + any(Namespace.class))) + .thenThrow(new RuntimeException(injectedErrMsg)); + + }); + + registerBuiltinFunction("exclamation", getPulsarApiExamplesNar()); + when(mockedRuntimeFactory.externallyManaged()).thenReturn(true); + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(function))).thenReturn(false); - public static FunctionDetails createDefaultFunctionDetails() { FunctionConfig functionConfig = createDefaultFunctionConfig(); - return FunctionConfigUtils.convert(functionConfig, (ClassLoader) null); + functionConfig.setJar("builtin://exclamation"); + + try { + registerFunction(tenant, namespace, function, null, mockedFormData, null, functionConfig); + Assert.fail(); + } catch (RuntimeException e) { + Assert.assertEquals(e.getMessage(), injectedErrMsg); + } } + } diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java index 5dcc795304ef5..b9833380d7087 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java @@ -18,246 +18,78 @@ */ package org.apache.pulsar.functions.worker.rest.api.v3; +import static org.apache.pulsar.functions.proto.Function.ProcessingGuarantees.ATLEAST_ONCE; +import static org.apache.pulsar.functions.source.TopicSchema.DEFAULT_SERDE; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; import com.google.common.collect.Lists; import java.io.File; import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.nio.file.StandardCopyOption; import java.util.Collections; import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Objects; -import java.util.function.Consumer; import javax.ws.rs.core.Response; import org.apache.distributedlog.api.namespace.Namespace; import org.apache.logging.log4j.Level; import org.apache.logging.log4j.core.config.Configurator; import org.apache.pulsar.broker.authentication.AuthenticationParameters; -import org.apache.pulsar.client.admin.Functions; -import org.apache.pulsar.client.admin.Namespaces; -import org.apache.pulsar.client.admin.Packages; -import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.client.admin.Tenants; -import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.functions.UpdateOptionsImpl; import org.apache.pulsar.common.functions.Utils; import org.apache.pulsar.common.io.SinkConfig; -import org.apache.pulsar.common.nar.NarClassLoader; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; -import org.apache.pulsar.common.util.ClassLoaderUtils; import org.apache.pulsar.common.util.RestException; -import org.apache.pulsar.functions.api.examples.ExclamationFunction; import org.apache.pulsar.functions.api.examples.RecordFunction; import org.apache.pulsar.functions.api.utils.IdentityFunction; import org.apache.pulsar.functions.instance.InstanceUtils; import org.apache.pulsar.functions.proto.Function; import org.apache.pulsar.functions.proto.Function.FunctionDetails; import org.apache.pulsar.functions.proto.Function.FunctionMetaData; -import org.apache.pulsar.functions.runtime.RuntimeFactory; -import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.utils.SinkConfigUtils; -import org.apache.pulsar.functions.utils.functions.FunctionArchive; -import org.apache.pulsar.functions.utils.functions.FunctionUtils; -import org.apache.pulsar.functions.utils.io.Connector; -import org.apache.pulsar.functions.utils.io.ConnectorUtils; -import org.apache.pulsar.functions.worker.ConnectorsManager; -import org.apache.pulsar.functions.worker.FunctionMetaDataManager; -import org.apache.pulsar.functions.worker.FunctionRuntimeManager; -import org.apache.pulsar.functions.worker.FunctionsManager; -import org.apache.pulsar.functions.worker.LeaderService; -import org.apache.pulsar.functions.worker.PulsarWorkerService; -import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerUtils; -import org.apache.pulsar.functions.worker.rest.api.PulsarFunctionTestTemporaryDirectory; import org.apache.pulsar.functions.worker.rest.api.SinksImpl; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import org.mockito.MockedStatic; import org.mockito.Mockito; import org.testng.Assert; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import static org.apache.pulsar.functions.proto.Function.ProcessingGuarantees.ATLEAST_ONCE; -import static org.apache.pulsar.functions.source.TopicSchema.DEFAULT_SERDE; -import static org.mockito.ArgumentMatchers.anyBoolean; -import static org.mockito.Mockito.any; -import static org.mockito.Mockito.anyString; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.eq; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.when; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNotNull; -import static org.testng.Assert.assertTrue; -import static org.testng.Assert.fail; /** * Unit test of {@link SinksApiV3Resource}. */ -public class SinkApiV3ResourceTest { +public class SinkApiV3ResourceTest extends AbstractFunctionsResourceTest { - private static final String tenant = "test-tenant"; - private static final String namespace = "test-namespace"; private static final String sink = "test-sink"; - private static final Map topicsToSerDeClassName = new HashMap<>(); - - static { - topicsToSerDeClassName.put("test_src", DEFAULT_SERDE); - } - - private static final String subscriptionName = "test-subscription"; - private static final String CASSANDRA_STRING_SINK = "org.apache.pulsar.io.cassandra.CassandraStringSink"; - private static final int parallelism = 1; - - private PulsarWorkerService mockedWorkerService; - private PulsarAdmin mockedPulsarAdmin; - private Tenants mockedTenants; - private Namespaces mockedNamespaces; - private Functions mockedFunctions; - private TenantInfoImpl mockedTenantInfo; - private List namespaceList = new LinkedList<>(); - private FunctionMetaDataManager mockedManager; - private FunctionRuntimeManager mockedFunctionRunTimeManager; - private RuntimeFactory mockedRuntimeFactory; - private Namespace mockedNamespace; + private SinksImpl resource; - private InputStream mockedInputStream; - private FormDataContentDisposition mockedFormData; - private FunctionMetaData mockedFunctionMetaData; - private LeaderService mockedLeaderService; - private Packages mockedPackages; - private PulsarFunctionTestTemporaryDirectory tempDirectory; - private static Map mockStaticContexts = new HashMap<>(); - - private static final String SYSTEM_PROPERTY_NAME_CASSANDRA_NAR_FILE_PATH = "pulsar-io-cassandra.nar.path"; - - public static File getPulsarIOCassandraNar() { - return new File(Objects.requireNonNull(System.getProperty(SYSTEM_PROPERTY_NAME_CASSANDRA_NAR_FILE_PATH) - , "pulsar-io-cassandra.nar file location must be specified with " - + SYSTEM_PROPERTY_NAME_CASSANDRA_NAR_FILE_PATH + " system property")); - } - - private static final String SYSTEM_PROPERTY_NAME_TWITTER_NAR_FILE_PATH = "pulsar-io-twitter.nar.path"; - - public static File getPulsarIOTwitterNar() { - return new File(Objects.requireNonNull(System.getProperty(SYSTEM_PROPERTY_NAME_TWITTER_NAR_FILE_PATH) - , "pulsar-io-twitter.nar file location must be specified with " - + SYSTEM_PROPERTY_NAME_TWITTER_NAR_FILE_PATH + " system property")); - } - - private static final String SYSTEM_PROPERTY_NAME_INVALID_NAR_FILE_PATH = "pulsar-io-invalid.nar.path"; - - public static File getPulsarIOInvalidNar() { - return new File(Objects.requireNonNull(System.getProperty(SYSTEM_PROPERTY_NAME_INVALID_NAR_FILE_PATH) - , "invalid nar file location must be specified with " - + SYSTEM_PROPERTY_NAME_INVALID_NAR_FILE_PATH + " system property")); - } - - @BeforeMethod - public void setup() throws Exception { - this.mockedManager = mock(FunctionMetaDataManager.class); - this.mockedFunctionRunTimeManager = mock(FunctionRuntimeManager.class); - this.mockedRuntimeFactory = mock(RuntimeFactory.class); - this.mockedInputStream = mock(InputStream.class); - this.mockedNamespace = mock(Namespace.class); - this.mockedFormData = mock(FormDataContentDisposition.class); - when(mockedFormData.getFileName()).thenReturn("test"); - this.mockedTenantInfo = mock(TenantInfoImpl.class); - this.mockedPulsarAdmin = mock(PulsarAdmin.class); - this.mockedTenants = mock(Tenants.class); - this.mockedNamespaces = mock(Namespaces.class); - this.mockedFunctions = mock(Functions.class); - this.mockedLeaderService = mock(LeaderService.class); - this.mockedPackages = mock(Packages.class); - namespaceList.add(tenant + "/" + namespace); - - this.mockedWorkerService = mock(PulsarWorkerService.class); - when(mockedWorkerService.getFunctionMetaDataManager()).thenReturn(mockedManager); - when(mockedWorkerService.getLeaderService()).thenReturn(mockedLeaderService); - when(mockedWorkerService.getFunctionRuntimeManager()).thenReturn(mockedFunctionRunTimeManager); - when(mockedFunctionRunTimeManager.getRuntimeFactory()).thenReturn(mockedRuntimeFactory); - when(mockedWorkerService.getDlogNamespace()).thenReturn(mockedNamespace); - when(mockedWorkerService.isInitialized()).thenReturn(true); - when(mockedWorkerService.getBrokerAdmin()).thenReturn(mockedPulsarAdmin); - when(mockedWorkerService.getFunctionAdmin()).thenReturn(mockedPulsarAdmin); - when(mockedPulsarAdmin.tenants()).thenReturn(mockedTenants); - when(mockedPulsarAdmin.namespaces()).thenReturn(mockedNamespaces); - when(mockedPulsarAdmin.functions()).thenReturn(mockedFunctions); - when(mockedPulsarAdmin.packages()).thenReturn(mockedPackages); - when(mockedTenants.getTenantInfo(any())).thenReturn(mockedTenantInfo); - when(mockedNamespaces.getNamespaces(any())).thenReturn(namespaceList); - when(mockedLeaderService.isLeader()).thenReturn(true); - doAnswer(invocationOnMock -> { - Files.copy(getPulsarIOCassandraNar().toPath(), Paths.get(invocationOnMock.getArgument(1, String.class)), - StandardCopyOption.REPLACE_EXISTING); - return null; - }).when(mockedPackages).download(any(), any()); - - // worker config - WorkerConfig workerConfig = new WorkerConfig() - .setWorkerId("test") - .setWorkerPort(8080) - .setFunctionMetadataTopicName("pulsar/functions") - .setNumFunctionPackageReplicas(3) - .setPulsarServiceUrl("pulsar://localhost:6650/"); - tempDirectory = PulsarFunctionTestTemporaryDirectory.create(getClass().getSimpleName()); - tempDirectory.useTemporaryDirectoriesForWorkerConfig(workerConfig); - when(mockedWorkerService.getWorkerConfig()).thenReturn(workerConfig); + @Override + protected void doSetup() { this.resource = spy(new SinksImpl(() -> mockedWorkerService)); - } - @AfterMethod(alwaysRun = true) - public void cleanup() { - if (tempDirectory != null) { - tempDirectory.delete(); - } - mockStaticContexts.values().forEach(MockedStatic::close); - mockStaticContexts.clear(); + @Override + protected Function.FunctionDetails.ComponentType getComponentType() { + return Function.FunctionDetails.ComponentType.SINK; } - private void mockStatic(Class classStatic, Consumer> consumer) { - final MockedStatic mockedStatic = - mockStaticContexts.computeIfAbsent(classStatic.getName(), name -> Mockito.mockStatic(classStatic)); - consumer.accept(mockedStatic); - } - - private void mockWorkerUtils() { - mockWorkerUtils(null); - } - - private void mockWorkerUtils(Consumer> consumer) { - mockStatic(WorkerUtils.class, ctx -> { - ctx.when(() -> WorkerUtils.dumpToTmpFile(any())).thenCallRealMethod(); - if (consumer != null) { - consumer.accept(ctx); - } - }); + @Override + protected File getDefaultNarFile() { + return getPulsarIOCassandraNar(); } - private void mockInstanceUtils() { - mockStatic(InstanceUtils.class, ctx -> { - ctx.when(() -> InstanceUtils.calculateSubjectType(any())) - .thenReturn(FunctionDetails.ComponentType.SINK); - }); - } - - - // - // Register Functions - // - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Tenant is not provided") public void testRegisterSinkMissingTenant() { try { @@ -337,8 +169,8 @@ public void testRegisterSinkMissingPackage() { } } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Sink class UnknownClass must " - + "be in class path") + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Sink class UnknownClass not " + + "found") public void testRegisterSinkWrongClassName() { mockInstanceUtils(); try { @@ -359,10 +191,8 @@ public void testRegisterSinkWrongClassName() { } } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Sink package does not have the" - + " correct format. Pulsar cannot determine if the package is a NAR package" - + " or JAR package. Sink classname is not provided and attempts to load it as a NAR package produced the " - + "following error.") + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Sink package doesn't contain " + + "the META-INF/services/pulsar-io.yaml file.") public void testRegisterSinkMissingPackageDetails() { mockInstanceUtils(); try { @@ -722,30 +552,11 @@ public void testRegisterSinkSuccessWithTransformFunction() throws Exception { when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(sink))).thenReturn(false); - NarClassLoader mockedClassLoader = mock(NarClassLoader.class); - doReturn(RecordFunction.class).when(mockedClassLoader).loadClass("RecordFunction"); - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(FunctionCommon::createPkgTempFile).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getRawFunctionTypes(any(), anyBoolean())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getFunctionTypes(any(), anyBoolean())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getFunctionClassParent(any(), anyBoolean())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getClassLoaderFromPackage(any(), any(), any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.extractNarClassLoader(any(), any())).thenReturn(mockedClassLoader); - }); - - mockStatic(FunctionUtils.class, ctx -> { - ctx.when(() -> FunctionUtils.getFunctionClass(any())).thenReturn("RecordFunction"); - }); - - FunctionsManager mockedFunctionsManager = mock(FunctionsManager.class); - FunctionArchive functionArchive = FunctionArchive.builder() - .classLoader(mockedClassLoader) - .build(); - when(mockedFunctionsManager.getFunction("transform")).thenReturn(functionArchive); - - when(mockedWorkerService.getFunctionsManager()).thenReturn(mockedFunctionsManager); + registerBuiltinConnector("recordfunction", RecordFunction.class.getName()); + registerBuiltinFunction("transform", RecordFunction.class.getName()); SinkConfig sinkConfig = createDefaultSinkConfig(); + sinkConfig.setSinkType("builtin://recordfunction"); sinkConfig.setTransformFunction("builtin://transform"); sinkConfig.setTransformFunctionConfig("{\"dummy\": \"dummy\"}"); @@ -770,28 +581,7 @@ public void testRegisterSinkFailureWithInvalidTransformFunction() throws Excepti when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(sink))).thenReturn(false); - NarClassLoader mockedClassLoader = mock(NarClassLoader.class); - doReturn(ExclamationFunction.class).when(mockedClassLoader).loadClass("ExclamationFunction"); - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(FunctionCommon::createPkgTempFile).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getRawFunctionTypes(any(), anyBoolean())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getFunctionTypes(any(), anyBoolean())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getFunctionClassParent(any(), anyBoolean())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getClassLoaderFromPackage(any(), any(), any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.extractNarClassLoader(any(), any())).thenReturn(mockedClassLoader); - }); - - mockStatic(FunctionUtils.class, ctx -> { - ctx.when(() -> FunctionUtils.getFunctionClass(any())).thenReturn("ExclamationFunction"); - }); - - FunctionsManager mockedFunctionsManager = mock(FunctionsManager.class); - FunctionArchive functionArchive = FunctionArchive.builder() - .classLoader(mockedClassLoader) - .build(); - when(mockedFunctionsManager.getFunction("transform")).thenReturn(functionArchive); - - when(mockedWorkerService.getFunctionsManager()).thenReturn(mockedFunctionsManager); + registerBuiltinFunction("transform", getPulsarApiExamplesNar()); SinkConfig sinkConfig = createDefaultSinkConfig(); sinkConfig.setTransformFunction("builtin://transform"); @@ -946,16 +736,18 @@ public void testUpdateSinkDifferentInputs() throws Exception { public void testUpdateSinkDifferentParallelism() throws Exception { mockWorkerUtils(); - testUpdateSinkMissingArguments( - tenant, - namespace, - sink, - null, - mockedFormData, - topicsToSerDeClassName, - CASSANDRA_STRING_SINK, - parallelism + 1, - null); + try (FileInputStream inputStream = new FileInputStream(getPulsarIOCassandraNar())) { + testUpdateSinkMissingArguments( + tenant, + namespace, + sink, + inputStream, + mockedFormData, + topicsToSerDeClassName, + CASSANDRA_STRING_SINK, + parallelism + 1, + null); + } } private void testUpdateSinkMissingArguments( @@ -1007,32 +799,6 @@ private void testUpdateSinkMissingArguments( } - private void mockFunctionCommon(String tenant, String namespace, String sink) throws IOException { - mockStatic(ConnectorUtils.class, ctx -> { - ctx.when(() -> ConnectorUtils.getIOSinkClass(any(NarClassLoader.class))) - .thenReturn(CASSANDRA_STRING_SINK); - }); - - mockStatic(ClassLoaderUtils.class, ctx -> { - }); - - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.createPkgTempFile()).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getClassLoaderFromPackage(any(), any(), any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getSinkType(any())).thenReturn(String.class); - ctx.when(() -> FunctionCommon.extractNarClassLoader(any(), any())).thenReturn(mock(NarClassLoader.class)); - ctx.when(() -> FunctionCommon - .convertProcessingGuarantee(eq(FunctionConfig.ProcessingGuarantees.ATLEAST_ONCE))) - .thenReturn(ATLEAST_ONCE); - }); - - this.mockedFunctionMetaData = - FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); - when(mockedManager.getFunctionMetaData(eq(tenant), eq(namespace), eq(sink))).thenReturn(mockedFunctionMetaData); - - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(sink))).thenReturn(true); - } - private void updateDefaultSink() throws Exception { updateDefaultSinkWithPackageUrl(null); } @@ -1040,25 +806,6 @@ private void updateDefaultSink() throws Exception { private void updateDefaultSinkWithPackageUrl(String packageUrl) throws Exception { SinkConfig sinkConfig = createDefaultSinkConfig(); - mockStatic(ConnectorUtils.class, ctx -> { - ctx.when(() -> ConnectorUtils.getIOSinkClass(any(NarClassLoader.class))) - .thenReturn(CASSANDRA_STRING_SINK); - }); - - mockStatic(ClassLoaderUtils.class, ctx -> { - }); - - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.createPkgTempFile()).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getClassLoaderFromPackage(any(), any(), any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getSinkType(any())).thenReturn(String.class); - ctx.when(() -> FunctionCommon.extractNarClassLoader(any(), any())).thenReturn(mock(NarClassLoader.class)); - ctx.when(() -> FunctionCommon - .convertProcessingGuarantee(eq(FunctionConfig.ProcessingGuarantees.ATLEAST_ONCE))) - .thenReturn(ATLEAST_ONCE); - }); - - this.mockedFunctionMetaData = FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); when(mockedManager.getFunctionMetaData(any(), any(), any())).thenReturn(mockedFunctionMetaData); @@ -1091,7 +838,6 @@ public void testUpdateNotExistedSink() throws Exception { public void testUpdateSinkUploadFailure() throws Exception { try { mockWorkerUtils(ctx -> { - ctx.when(() -> WorkerUtils.dumpToTmpFile(any())).thenCallRealMethod(); ctx.when(() -> WorkerUtils.uploadFileToBookkeeper( anyString(), any(File.class), @@ -1127,24 +873,6 @@ public void testUpdateSinkWithUrl() throws Exception { when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(sink))).thenReturn(true); - mockStatic(ConnectorUtils.class, ctx -> { - ctx.when(() -> ConnectorUtils.getIOSinkClass(any())) - .thenReturn(CASSANDRA_STRING_SINK); - }); - - mockStatic(ClassLoaderUtils.class, ctx -> { - }); - - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.extractFileFromPkgURL(any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getClassLoaderFromPackage(any(), any(), any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getSinkType(any())).thenReturn(String.class); - ctx.when(() -> FunctionCommon.extractNarClassLoader(any(), any())).thenReturn(mock(NarClassLoader.class)); - ctx.when(() -> FunctionCommon - .convertProcessingGuarantee(eq(FunctionConfig.ProcessingGuarantees.ATLEAST_ONCE))) - .thenReturn(ATLEAST_ONCE); - }); - this.mockedFunctionMetaData = FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); when(mockedManager.getFunctionMetaData(any(), any(), any())).thenReturn(mockedFunctionMetaData); @@ -1219,35 +947,17 @@ public void testUpdateSinkDifferentTransformFunction() throws Exception { SinkConfig sinkConfig = createDefaultSinkConfig(); sinkConfig.setTransformFunction("builtin://transform"); - sinkConfig.setTransformFunctionClassName("DummyFunction"); sinkConfig.setTransformFunctionConfig("{\"dummy\": \"dummy\"}"); - NarClassLoader mockedClassLoader = mock(NarClassLoader.class); - doReturn(RecordFunction.class).when(mockedClassLoader).loadClass("DummyFunction"); - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(FunctionCommon::createPkgTempFile).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getRawFunctionTypes(any(), anyBoolean())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getFunctionTypes(any(), anyBoolean())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getFunctionClassParent(any(), anyBoolean())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getClassLoaderFromPackage(any(), any(), any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.extractNarClassLoader(any(), any())).thenReturn(mockedClassLoader); - }); + registerBuiltinFunction("transform", RecordFunction.class.getName()); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(sink))).thenReturn(true); this.mockedFunctionMetaData = FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); when(mockedManager.getFunctionMetaData(eq(tenant), eq(namespace), eq(sink))).thenReturn(mockedFunctionMetaData); - when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(sink))).thenReturn(true); - FunctionsManager mockedFunctionsManager = mock(FunctionsManager.class); - FunctionArchive functionArchive = FunctionArchive.builder() - .classLoader(mockedClassLoader) - .build(); - when(mockedFunctionsManager.getFunction("transform")).thenReturn(functionArchive); - - when(mockedWorkerService.getFunctionsManager()).thenReturn(mockedFunctionsManager); - - try (FileInputStream inputStream = new FileInputStream(getPulsarIOCassandraNar())) { resource.updateSink( tenant, @@ -1737,6 +1447,14 @@ private SinkConfig createDefaultSinkConfig() { return sinkConfig; } + private void mockFunctionCommon(String tenant, String namespace, String sink) throws IOException { + this.mockedFunctionMetaData = + Function.FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); + when(mockedManager.getFunctionMetaData(eq(tenant), eq(namespace), eq(sink))).thenReturn(mockedFunctionMetaData); + + when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(sink))).thenReturn(true); + } + private FunctionDetails createDefaultFunctionDetails() throws IOException { return SinkConfigUtils.convert(createDefaultSinkConfig(), new SinkConfigUtils.ExtractedSinkDetails(null, null, null)); @@ -1760,21 +1478,7 @@ public void testRegisterSinkSuccessK8sNoUpload() throws Exception { }); - NarClassLoader mockedClassLoader = mock(NarClassLoader.class); - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.getSinkType(any())).thenReturn(String.class); - ctx.when(() -> FunctionCommon.getClassLoaderFromPackage(any(), any(), any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.isFunctionCodeBuiltin(any())).thenReturn(true); - ctx.when(() -> FunctionCommon.extractNarClassLoader(any(), any())).thenReturn(mockedClassLoader); - }); - - ConnectorsManager mockedConnManager = mock(ConnectorsManager.class); - Connector connector = Connector.builder() - .classLoader(mockedClassLoader) - .build(); - when(mockedConnManager.getConnector("cassandra")).thenReturn(connector); - when(mockedConnManager.getSinkArchive(any())).thenReturn(getPulsarIOCassandraNar().toPath()); - when(mockedWorkerService.getConnectorsManager()).thenReturn(mockedConnManager); + registerBuiltinConnector("cassandra", getPulsarIOCassandraNar()); when(mockedRuntimeFactory.externallyManaged()).thenReturn(true); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(sink))).thenReturn(false); @@ -1814,23 +1518,7 @@ public void testRegisterSinkSuccessK8sWithUpload() throws Exception { }); - NarClassLoader mockedClassLoader = mock(NarClassLoader.class); - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.getSinkType(any())).thenReturn(String.class); - ctx.when(() -> FunctionCommon.getClassLoaderFromPackage(any(), any(), any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.isFunctionCodeBuiltin(any())).thenReturn(true); - ctx.when(() -> FunctionCommon.extractNarClassLoader(any(), any())).thenReturn(mockedClassLoader); - }); - - ConnectorsManager mockedConnManager = mock(ConnectorsManager.class); - Connector connector = Connector.builder() - .classLoader(mockedClassLoader) - .build(); - when(mockedConnManager.getConnector("cassandra")).thenReturn(connector); - when(mockedConnManager.getSinkArchive(any())).thenReturn(getPulsarIOCassandraNar().toPath()); - - when(mockedWorkerService.getConnectorsManager()).thenReturn(mockedConnManager); - + registerBuiltinConnector("cassandra", getPulsarIOCassandraNar()); when(mockedRuntimeFactory.externallyManaged()).thenReturn(true); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(sink))).thenReturn(false); @@ -1865,11 +1553,6 @@ public void testUpdateSinkWithNoChange() throws IOException { mockStatic(SinkConfigUtils.class, ctx -> { ctx.when(() -> SinkConfigUtils.convertFromDetails(any())).thenReturn(sinkConfig); - ctx.when(() -> SinkConfigUtils.convert(any(), any())).thenCallRealMethod(); - ctx.when(() -> SinkConfigUtils.validateUpdate(any(), any())).thenCallRealMethod(); - ctx.when(() -> SinkConfigUtils.clone(any())).thenCallRealMethod(); - ctx.when(() -> SinkConfigUtils.collectAllInputTopics(any())).thenCallRealMethod(); - ctx.when(() -> SinkConfigUtils.validateAndExtractDetails(any(),any(),any(),anyBoolean())).thenCallRealMethod(); }); mockFunctionCommon(sinkConfig.getTenant(), sinkConfig.getNamespace(), sinkConfig.getName()); @@ -1912,15 +1595,17 @@ public void testUpdateSinkWithNoChange() throws IOException { // no changes but set the auth-update flag to true, should not fail UpdateOptionsImpl updateOptions = new UpdateOptionsImpl(); updateOptions.setUpdateAuthData(true); - resource.updateSink( - sinkConfig.getTenant(), - sinkConfig.getNamespace(), - sinkConfig.getName(), - null, - mockedFormData, - null, - sinkConfig, - null, - updateOptions); + try (FileInputStream inputStream = new FileInputStream(getPulsarIOCassandraNar())) { + resource.updateSink( + sinkConfig.getTenant(), + sinkConfig.getNamespace(), + sinkConfig.getName(), + inputStream, + mockedFormData, + null, + sinkConfig, + null, + updateOptions); + } } } diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SourceApiV3ResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SourceApiV3ResourceTest.java index eabf954cc77b1..c7e69484d3019 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SourceApiV3ResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SourceApiV3ResourceTest.java @@ -18,17 +18,10 @@ */ package org.apache.pulsar.functions.worker.rest.api.v3; -import static org.apache.pulsar.functions.worker.rest.api.v3.SinkApiV3ResourceTest.getPulsarIOCassandraNar; -import static org.apache.pulsar.functions.worker.rest.api.v3.SinkApiV3ResourceTest.getPulsarIOInvalidNar; -import static org.apache.pulsar.functions.worker.rest.api.v3.SinkApiV3ResourceTest.getPulsarIOTwitterNar; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.when; @@ -41,31 +34,17 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.nio.file.StandardCopyOption; -import java.util.HashMap; import java.util.LinkedList; import java.util.List; -import java.util.Map; -import java.util.function.Consumer; import javax.ws.rs.core.Response; import org.apache.distributedlog.api.namespace.Namespace; import org.apache.logging.log4j.Level; import org.apache.logging.log4j.core.config.Configurator; import org.apache.pulsar.broker.authentication.AuthenticationParameters; -import org.apache.pulsar.client.admin.Functions; -import org.apache.pulsar.client.admin.Namespaces; -import org.apache.pulsar.client.admin.Packages; -import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.client.admin.Tenants; import org.apache.pulsar.common.functions.UpdateOptionsImpl; import org.apache.pulsar.common.functions.Utils; import org.apache.pulsar.common.io.SourceConfig; -import org.apache.pulsar.common.nar.NarClassLoader; -import org.apache.pulsar.common.nar.NarClassLoaderBuilder; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.ClassLoaderUtils; import org.apache.pulsar.common.util.RestException; import org.apache.pulsar.functions.api.utils.IdentityFunction; @@ -75,159 +54,35 @@ import org.apache.pulsar.functions.proto.Function.ProcessingGuarantees; import org.apache.pulsar.functions.proto.Function.SinkSpec; import org.apache.pulsar.functions.proto.Function.SourceSpec; -import org.apache.pulsar.functions.runtime.RuntimeFactory; import org.apache.pulsar.functions.source.TopicSchema; -import org.apache.pulsar.functions.utils.FunctionCommon; import org.apache.pulsar.functions.utils.SourceConfigUtils; import org.apache.pulsar.functions.utils.io.ConnectorUtils; -import org.apache.pulsar.functions.worker.FunctionMetaDataManager; -import org.apache.pulsar.functions.worker.FunctionRuntimeManager; -import org.apache.pulsar.functions.worker.LeaderService; -import org.apache.pulsar.functions.worker.PulsarWorkerService; -import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerUtils; -import org.apache.pulsar.functions.worker.rest.api.PulsarFunctionTestTemporaryDirectory; import org.apache.pulsar.functions.worker.rest.api.SourcesImpl; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; import org.mockito.MockedStatic; import org.mockito.Mockito; -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; /** * Unit test of {@link SourcesApiV3Resource}. */ -public class SourceApiV3ResourceTest { +public class SourceApiV3ResourceTest extends AbstractFunctionsResourceTest { - private static final String tenant = "test-tenant"; - private static final String namespace = "test-namespace"; private static final String source = "test-source"; private static final String outputTopic = "test-output-topic"; private static final String outputSerdeClassName = TopicSchema.DEFAULT_SERDE; private static final String TWITTER_FIRE_HOSE = "org.apache.pulsar.io.twitter.TwitterFireHose"; - private static final int parallelism = 1; - - private PulsarWorkerService mockedWorkerService; - private PulsarAdmin mockedPulsarAdmin; - private Tenants mockedTenants; - private Namespaces mockedNamespaces; - private Functions mockedFunctions; - private TenantInfoImpl mockedTenantInfo; - private List namespaceList = new LinkedList<>(); - private FunctionMetaDataManager mockedManager; - private FunctionRuntimeManager mockedFunctionRunTimeManager; - private RuntimeFactory mockedRuntimeFactory; - private Namespace mockedNamespace; private SourcesImpl resource; - private InputStream mockedInputStream; - private FormDataContentDisposition mockedFormData; - private FunctionMetaData mockedFunctionMetaData; - private LeaderService mockedLeaderService; - private Packages mockedPackages; - private PulsarFunctionTestTemporaryDirectory tempDirectory; - - private static NarClassLoader narClassLoader; - private static Map mockStaticContexts = new HashMap<>(); - - @BeforeClass - public void setupNarClassLoader() throws IOException { - narClassLoader = NarClassLoaderBuilder.builder().narFile(getPulsarIOTwitterNar()).build(); - } - - @AfterClass(alwaysRun = true) - public void cleanupNarClassLoader() throws IOException { - if (narClassLoader != null) { - narClassLoader.close(); - narClassLoader = null; - } - } - - @BeforeMethod - public void setup() throws Exception { - this.mockedManager = mock(FunctionMetaDataManager.class); - this.mockedFunctionRunTimeManager = mock(FunctionRuntimeManager.class); - this.mockedRuntimeFactory = mock(RuntimeFactory.class); - this.mockedInputStream = mock(InputStream.class); - this.mockedNamespace = mock(Namespace.class); - this.mockedFormData = mock(FormDataContentDisposition.class); - when(mockedFormData.getFileName()).thenReturn("test"); - this.mockedTenantInfo = mock(TenantInfoImpl.class); - this.mockedPulsarAdmin = mock(PulsarAdmin.class); - this.mockedTenants = mock(Tenants.class); - this.mockedNamespaces = mock(Namespaces.class); - this.mockedFunctions = mock(Functions.class); - this.mockedLeaderService = mock(LeaderService.class); - this.mockedPackages = mock(Packages.class); - namespaceList.add(tenant + "/" + namespace); - - this.mockedWorkerService = mock(PulsarWorkerService.class); - when(mockedWorkerService.getFunctionMetaDataManager()).thenReturn(mockedManager); - when(mockedWorkerService.getLeaderService()).thenReturn(mockedLeaderService); - when(mockedWorkerService.getFunctionRuntimeManager()).thenReturn(mockedFunctionRunTimeManager); - when(mockedFunctionRunTimeManager.getRuntimeFactory()).thenReturn(mockedRuntimeFactory); - when(mockedWorkerService.getDlogNamespace()).thenReturn(mockedNamespace); - when(mockedWorkerService.isInitialized()).thenReturn(true); - when(mockedWorkerService.getBrokerAdmin()).thenReturn(mockedPulsarAdmin); - when(mockedWorkerService.getFunctionAdmin()).thenReturn(mockedPulsarAdmin); - when(mockedPulsarAdmin.tenants()).thenReturn(mockedTenants); - when(mockedPulsarAdmin.namespaces()).thenReturn(mockedNamespaces); - when(mockedPulsarAdmin.functions()).thenReturn(mockedFunctions); - when(mockedPulsarAdmin.packages()).thenReturn(mockedPackages); - when(mockedTenants.getTenantInfo(any())).thenReturn(mockedTenantInfo); - when(mockedNamespaces.getNamespaces(any())).thenReturn(namespaceList); - when(mockedLeaderService.isLeader()).thenReturn(true); - doAnswer(invocationOnMock -> { - Files.copy(getPulsarIOTwitterNar().toPath(), Paths.get(invocationOnMock.getArgument(1, String.class)), - StandardCopyOption.REPLACE_EXISTING); - return null; - }).when(mockedPackages).download(any(), any()); - - // worker config - WorkerConfig workerConfig = new WorkerConfig() - .setWorkerId("test") - .setWorkerPort(8080) - .setFunctionMetadataTopicName("pulsar/functions") - .setNumFunctionPackageReplicas(3) - .setPulsarServiceUrl("pulsar://localhost:6650/"); - tempDirectory = PulsarFunctionTestTemporaryDirectory.create(getClass().getSimpleName()); - tempDirectory.useTemporaryDirectoriesForWorkerConfig(workerConfig); - when(mockedWorkerService.getWorkerConfig()).thenReturn(workerConfig); + @Override + protected void doSetup() { this.resource = spy(new SourcesImpl(() -> mockedWorkerService)); } - private void mockStatic(Class classStatic, Consumer> consumer) { - final MockedStatic mockedStatic = - mockStaticContexts.computeIfAbsent(classStatic.getName(), name -> Mockito.mockStatic(classStatic)); - consumer.accept(mockedStatic); - } - - @AfterMethod(alwaysRun = true) - public void cleanup() { - if (tempDirectory != null) { - tempDirectory.delete(); - } - mockStaticContexts.values().forEach(MockedStatic::close); - mockStaticContexts.clear(); - } - - private void mockWorkerUtils() { - mockStatic(WorkerUtils.class, - ctx -> { - ctx.when(() -> WorkerUtils.dumpToTmpFile(any())).thenCallRealMethod(); - }); - } - - private void mockWorkerUtils(Consumer> consumer) { - mockStatic(WorkerUtils.class, ctx -> { - ctx.when(() -> WorkerUtils.dumpToTmpFile(any())).thenCallRealMethod(); - if (consumer != null) { - consumer.accept(ctx); - } - }); + @Override + protected FunctionDetails.ComponentType getComponentType() { + return FunctionDetails.ComponentType.SOURCE; } // @@ -297,8 +152,8 @@ public void testRegisterSourceMissingSourceName() { } } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Source class UnknownClass must" - + " be in class path") + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Source class UnknownClass not " + + "found in class loader") public void testRegisterSourceWrongClassName() { try { testRegisterSourceMissingArguments( @@ -361,10 +216,8 @@ public void testRegisterSourceMissingPackageDetails() throws IOException { } } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Source package does not have the" - + " correct format. Pulsar cannot determine if the package is a NAR package" - + " or JAR package. Source classname is not provided and attempts to load it as a NAR package " - + "produced the following error.") + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Source package doesn't contain" + + " the META-INF/services/pulsar-io.yaml file.") public void testRegisterSourceMissingPackageDetailsAndClassname() { try { testRegisterSourceMissingArguments( @@ -385,8 +238,8 @@ public void testRegisterSourceMissingPackageDetailsAndClassname() { } } - @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Failed to extract source class" - + " from archive") + @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Source package doesn't contain" + + " the META-INF/services/pulsar-io.yaml file.") public void testRegisterSourceInvalidJarWithNoSource() throws IOException { try (InputStream inputStream = new FileInputStream(getPulsarIOInvalidNar())) { testRegisterSourceMissingArguments( @@ -524,7 +377,7 @@ public void testUpdateMissingSinkConfig() { } private void registerDefaultSource() throws IOException { - registerDefaultSourceWithPackageUrl("source://public/default/test@v1"); + registerDefaultSourceWithPackageUrl(getPulsarIOTwitterNar().toURI().toString()); } private void registerDefaultSourceWithPackageUrl(String packageUrl) throws IOException { @@ -565,8 +418,6 @@ public void testRegisterSourceUploadFailure() throws Exception { any(File.class), any(Namespace.class))) .thenThrow(new IOException("upload failure")); - - ctx.when(() -> WorkerUtils.dumpToTmpFile(any())).thenCallRealMethod(); }); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(source))).thenReturn(false); @@ -593,7 +444,7 @@ public void testRegisterSourceSuccess() throws Exception { @Test(timeOut = 20000) public void testRegisterSourceSuccessWithPackageName() throws IOException { - registerDefaultSourceWithPackageUrl("source://public/default/test@v1"); + registerDefaultSource(); } @Test(timeOut = 20000) @@ -621,14 +472,7 @@ public void testRegisterSourceConflictingFields() throws Exception { when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(source))).thenReturn(true); when(mockedManager.containsFunction(eq(actualTenant), eq(actualNamespace), eq(actualName))).thenReturn(false); - SourceConfig sourceConfig = new SourceConfig(); - sourceConfig.setTenant(tenant); - sourceConfig.setNamespace(namespace); - sourceConfig.setName(source); - sourceConfig.setClassName(TWITTER_FIRE_HOSE); - sourceConfig.setParallelism(parallelism); - sourceConfig.setTopicName(outputTopic); - sourceConfig.setSerdeClassName(outputSerdeClassName); + SourceConfig sourceConfig = createDefaultSourceConfig(); try (InputStream inputStream = new FileInputStream(getPulsarIOTwitterNar())) { resource.registerSource( actualTenant, @@ -815,17 +659,19 @@ public void testUpdateSourceChangedParallelism() throws Exception { try { mockWorkerUtils(); - testUpdateSourceMissingArguments( - tenant, - namespace, - source, - null, - mockedFormData, - outputTopic, - outputSerdeClassName, - TWITTER_FIRE_HOSE, - parallelism + 1, - null); + try(FileInputStream inputStream = new FileInputStream(getPulsarIOTwitterNar())) { + testUpdateSourceMissingArguments( + tenant, + namespace, + source, + inputStream, + mockedFormData, + outputTopic, + outputSerdeClassName, + TWITTER_FIRE_HOSE, + parallelism + 1, + null); + } } catch (RestException re) { assertEquals(re.getResponse().getStatusInfo(), Response.Status.BAD_REQUEST); throw re; @@ -836,31 +682,29 @@ public void testUpdateSourceChangedParallelism() throws Exception { public void testUpdateSourceChangedTopic() throws Exception { mockWorkerUtils(); - testUpdateSourceMissingArguments( - tenant, - namespace, - source, - null, - mockedFormData, - "DifferentTopic", - outputSerdeClassName, - TWITTER_FIRE_HOSE, - parallelism, - null); + try(FileInputStream inputStream = new FileInputStream(getPulsarIOTwitterNar())) { + testUpdateSourceMissingArguments( + tenant, + namespace, + source, + inputStream, + mockedFormData, + "DifferentTopic", + outputSerdeClassName, + TWITTER_FIRE_HOSE, + parallelism, + null); + } } @Test - public void testUpdateSourceWithNoChange() { + public void testUpdateSourceWithNoChange() throws IOException { mockWorkerUtils(); // No change on config, SourceConfig sourceConfig = createDefaultSourceConfig(); mockStatic(SourceConfigUtils.class, ctx -> { ctx.when(() -> SourceConfigUtils.convertFromDetails(any())).thenReturn(sourceConfig); - ctx.when(() -> SourceConfigUtils.convert(any(), any())).thenCallRealMethod(); - ctx.when(() -> SourceConfigUtils.validateUpdate(any(), any())).thenCallRealMethod(); - ctx.when(() -> SourceConfigUtils.clone(any())).thenCallRealMethod(); - ctx.when(() -> SourceConfigUtils.validateAndExtractDetails(any(),any(),anyBoolean())).thenCallRealMethod(); }); mockFunctionCommon(sourceConfig.getTenant(), sourceConfig.getNamespace(), sourceConfig.getName()); @@ -903,16 +747,18 @@ public void testUpdateSourceWithNoChange() { // no changes but set the auth-update flag to true, should not fail UpdateOptionsImpl updateOptions = new UpdateOptionsImpl(); updateOptions.setUpdateAuthData(true); - resource.updateSource( - sourceConfig.getTenant(), - sourceConfig.getNamespace(), - sourceConfig.getName(), - null, - mockedFormData, - null, - sourceConfig, - null, - updateOptions); + try (InputStream inputStream = new FileInputStream(getPulsarIOTwitterNar())) { + resource.updateSource( + sourceConfig.getTenant(), + sourceConfig.getNamespace(), + sourceConfig.getName(), + inputStream, + mockedFormData, + null, + sourceConfig, + null, + updateOptions); + } } @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Source parallelism must be a " @@ -997,14 +843,6 @@ private void mockFunctionCommon(String tenant, String namespace, String function }); mockStatic(ClassLoaderUtils.class, c -> { }); - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.createPkgTempFile()).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getClassLoaderFromPackage(any(), any(), any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getSourceType(argThat(clazz -> clazz.getName().equals(TWITTER_FIRE_HOSE)))) - .thenReturn(String.class); - ctx.when(() -> FunctionCommon.extractNarClassLoader(any(), any())) - .thenReturn(narClassLoader); - }); this.mockedFunctionMetaData = FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); @@ -1014,49 +852,25 @@ private void mockFunctionCommon(String tenant, String namespace, String function } private void updateDefaultSource() throws Exception { - updateDefaultSourceWithPackageUrl(null); + updateDefaultSourceWithPackageUrl(getPulsarIOTwitterNar().toURI().toString()); } private void updateDefaultSourceWithPackageUrl(String packageUrl) throws Exception { - SourceConfig sourceConfig = new SourceConfig(); - sourceConfig.setTenant(tenant); - sourceConfig.setNamespace(namespace); - sourceConfig.setName(source); - sourceConfig.setClassName(TWITTER_FIRE_HOSE); - sourceConfig.setParallelism(parallelism); - sourceConfig.setTopicName(outputTopic); - sourceConfig.setSerdeClassName(outputSerdeClassName); - - mockStatic(ConnectorUtils.class, c -> { - }); - - mockStatic(ClassLoaderUtils.class, c -> { - }); - - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.createPkgTempFile()).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getClassLoaderFromPackage(any(), any(), any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getSourceType(argThat(clazz -> clazz.getName().equals(TWITTER_FIRE_HOSE)))) - .thenReturn(String.class); - ctx.when(() -> FunctionCommon.extractNarClassLoader(any(), any())) - .thenReturn(narClassLoader); - }); + SourceConfig sourceConfig = createDefaultSourceConfig(); this.mockedFunctionMetaData = FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); when(mockedManager.getFunctionMetaData(any(), any(), any())).thenReturn(mockedFunctionMetaData); - try (InputStream inputStream = new FileInputStream(getPulsarIOCassandraNar())) { - resource.updateSource( - tenant, - namespace, - source, - inputStream, - mockedFormData, - packageUrl, - sourceConfig, - null, null); - } + resource.updateSource( + tenant, + namespace, + source, + null, + mockedFormData, + packageUrl, + sourceConfig, + null, null); } @Test(expectedExceptions = RestException.class, expectedExceptionsMessageRegExp = "Source test-source doesn't " + @@ -1079,11 +893,25 @@ public void testUpdateSourceUploadFailure() throws Exception { anyString(), any(File.class), any(Namespace.class))).thenThrow(new IOException("upload failure")); - ctx.when(() -> WorkerUtils.dumpToTmpFile(any())).thenCallRealMethod(); }); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(source))).thenReturn(true); - updateDefaultSource(); + SourceConfig sourceConfig = createDefaultSourceConfig(); + this.mockedFunctionMetaData = + FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); + when(mockedManager.getFunctionMetaData(any(), any(), any())).thenReturn(mockedFunctionMetaData); + + try(InputStream inputStream = new FileInputStream(getPulsarIOTwitterNar())) { + resource.updateSource( + tenant, + namespace, + source, + inputStream, + mockedFormData, + null, + sourceConfig, + null, null); + } } catch (RestException re) { assertEquals(re.getResponse().getStatusInfo(), Response.Status.INTERNAL_SERVER_ERROR); throw re; @@ -1103,16 +931,9 @@ public void testUpdateSourceSuccess() throws Exception { public void testUpdateSourceWithUrl() throws Exception { Configurator.setRootLevel(Level.DEBUG); - String filePackageUrl = getPulsarIOCassandraNar().toURI().toString(); + String filePackageUrl = getPulsarIOTwitterNar().toURI().toString(); - SourceConfig sourceConfig = new SourceConfig(); - sourceConfig.setTopicName(outputTopic); - sourceConfig.setSerdeClassName(outputSerdeClassName); - sourceConfig.setTenant(tenant); - sourceConfig.setNamespace(namespace); - sourceConfig.setName(source); - sourceConfig.setClassName(TWITTER_FIRE_HOSE); - sourceConfig.setParallelism(parallelism); + SourceConfig sourceConfig = createDefaultSourceConfig(); when(mockedManager.containsFunction(eq(tenant), eq(namespace), eq(source))).thenReturn(true); mockStatic(ConnectorUtils.class, c -> { @@ -1120,15 +941,6 @@ public void testUpdateSourceWithUrl() throws Exception { mockStatic(ClassLoaderUtils.class, c -> { }); - mockStatic(FunctionCommon.class, ctx -> { - ctx.when(() -> FunctionCommon.extractFileFromPkgURL(any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getClassLoaderFromPackage(any(), any(), any(), any())).thenCallRealMethod(); - ctx.when(() -> FunctionCommon.getSourceType(argThat(clazz -> clazz.getName().equals(TWITTER_FIRE_HOSE)))) - .thenReturn(String.class); - ctx.when(() -> FunctionCommon.extractNarClassLoader(any(), any())) - .thenReturn(narClassLoader); - }); - this.mockedFunctionMetaData = FunctionMetaData.newBuilder().setFunctionDetails(createDefaultFunctionDetails()).build(); when(mockedManager.getFunctionMetaData(any(), any(), any())).thenReturn(mockedFunctionMetaData); diff --git a/tests/docker-images/latest-version-image/conf/functions_worker.conf b/tests/docker-images/latest-version-image/conf/functions_worker.conf index 8072639a0d4a2..6feb660231cec 100644 --- a/tests/docker-images/latest-version-image/conf/functions_worker.conf +++ b/tests/docker-images/latest-version-image/conf/functions_worker.conf @@ -22,7 +22,7 @@ autostart=false redirect_stderr=true stdout_logfile=/var/log/pulsar/functions_worker.log directory=/pulsar -environment=PULSAR_MEM="-Xmx128M",PULSAR_GC="-XX:+UseZGC" +environment=PULSAR_MEM="-Xmx128M -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/pulsar/logs/functions",PULSAR_GC="-XX:+UseZGC" command=/pulsar/bin/pulsar functions-worker user=pulsar stopwaitsecs=15 \ No newline at end of file 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 9b4823f46d4cc..781f964e21bfd 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 @@ -511,37 +511,18 @@ public synchronized void setupFunctionWorkers(String suffix, FunctionRuntimeType } private void startFunctionWorkersWithProcessContainerFactory(String suffix, int numFunctionWorkers) { - String serviceUrl = "pulsar://pulsar-broker-0:" + PulsarContainer.BROKER_PORT; - String httpServiceUrl = "http://pulsar-broker-0:" + PulsarContainer.BROKER_HTTP_PORT; workerContainers.putAll(runNumContainers( "functions-worker-process-" + suffix, numFunctionWorkers, - (name) -> new WorkerContainer(clusterName, name) - .withNetwork(network) - .withNetworkAliases(name) - // worker settings - .withEnv("PF_workerId", name) - .withEnv("PF_workerHostname", name) - .withEnv("PF_workerPort", "" + PulsarContainer.BROKER_HTTP_PORT) - .withEnv("PF_pulsarFunctionsCluster", clusterName) - .withEnv("PF_pulsarServiceUrl", serviceUrl) - .withEnv("PF_pulsarWebServiceUrl", httpServiceUrl) - // script - .withEnv("clusterName", clusterName) - .withEnv("zookeeperServers", ZKContainer.NAME) - // bookkeeper tools - .withEnv("zkServers", ZKContainer.NAME) + (name) -> createWorkerContainer(name) )); this.startWorkers(); } - private void startFunctionWorkersWithThreadContainerFactory(String suffix, int numFunctionWorkers) { + private WorkerContainer createWorkerContainer(String name) { String serviceUrl = "pulsar://pulsar-broker-0:" + PulsarContainer.BROKER_PORT; String httpServiceUrl = "http://pulsar-broker-0:" + PulsarContainer.BROKER_HTTP_PORT; - workerContainers.putAll(runNumContainers( - "functions-worker-thread-" + suffix, - numFunctionWorkers, - (name) -> new WorkerContainer(clusterName, name) + return new WorkerContainer(clusterName, name) .withNetwork(network) .withNetworkAliases(name) // worker settings @@ -551,13 +532,21 @@ private void startFunctionWorkersWithThreadContainerFactory(String suffix, int n .withEnv("PF_pulsarFunctionsCluster", clusterName) .withEnv("PF_pulsarServiceUrl", serviceUrl) .withEnv("PF_pulsarWebServiceUrl", httpServiceUrl) - .withEnv("PF_functionRuntimeFactoryClassName", "org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactory") - .withEnv("PF_functionRuntimeFactoryConfigs_threadGroupName", "pf-container-group") // script .withEnv("clusterName", clusterName) .withEnv("zookeeperServers", ZKContainer.NAME) // bookkeeper tools - .withEnv("zkServers", ZKContainer.NAME) + .withEnv("zkServers", ZKContainer.NAME); + } + + private void startFunctionWorkersWithThreadContainerFactory(String suffix, int numFunctionWorkers) { + workerContainers.putAll(runNumContainers( + "functions-worker-thread-" + suffix, + numFunctionWorkers, + (name) -> createWorkerContainer(name) + .withEnv("PF_functionRuntimeFactoryClassName", + "org.apache.pulsar.functions.runtime.thread.ThreadRuntimeFactory") + .withEnv("PF_functionRuntimeFactoryConfigs_threadGroupName", "pf-container-group") )); this.startWorkers(); } From 9cd39997208a1708ea68f47d43d3e3986bfe38fb Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 26 Feb 2024 17:06:08 +0200 Subject: [PATCH 271/699] Fix byte-buddy version in presto LICENSE (cherry picked from commit e3f51157340658ce6cc1f7c0f78ed1e1e50a130b) (cherry picked from commit a13326b23a69a90b921205919d000077bbe5d37f) --- pulsar-sql/presto-distribution/LICENSE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 392d83d9f13bc..966c3c4e9d20a 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -300,7 +300,7 @@ The Apache Software License, Version 2.0 - jetty-util-9.4.53.v20231009.jar - jetty-util-ajax-9.4.53.v20231009.jar * Byte Buddy - - byte-buddy-1.11.13.jar + - byte-buddy-1.14.12.jar * Apache BVal - bval-jsr-2.0.5.jar * Bytecode From 512a2aae6e6169dc4b1e6c31ab7827a889806c89 Mon Sep 17 00:00:00 2001 From: Hang Chen Date: Wed, 31 Jan 2024 11:17:59 +0800 Subject: [PATCH 272/699] [fix] [bk] Fix the BookKeeper license (#22000) (cherry picked from commit 503f85f6b312448ac6d1fd96b0bd2dda29070927) (cherry picked from commit 9e8cf15d73ead56554eaa1442ba1f97962d8d09f) --- pulsar-sql/presto-distribution/LICENSE | 30 +++++++++++++------------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 966c3c4e9d20a..32fcf0437ffac 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -430,21 +430,21 @@ The Apache Software License, Version 2.0 - async-http-client-2.12.1.jar - async-http-client-netty-utils-2.12.1.jar * Apache Bookkeeper - - bookkeeper-common-4.16.3.jar - - bookkeeper-common-allocator-4.16.3.jar - - bookkeeper-proto-4.16.3.jar - - bookkeeper-server-4.16.3.jar - - bookkeeper-stats-api-4.16.3.jar - - bookkeeper-tools-framework-4.16.3.jar - - circe-checksum-4.16.3.jar - - codahale-metrics-provider-4.16.3.jar - - cpu-affinity-4.16.3.jar - - http-server-4.16.3.jar - - prometheus-metrics-provider-4.16.3.jar - - codahale-metrics-provider-4.16.3.jar - - bookkeeper-slogger-api-4.16.3.jar - - bookkeeper-slogger-slf4j-4.16.3.jar - - native-io-4.16.3.jar + - bookkeeper-common-4.16.4.jar + - bookkeeper-common-allocator-4.16.4.jar + - bookkeeper-proto-4.16.4.jar + - bookkeeper-server-4.16.4.jar + - bookkeeper-stats-api-4.16.4.jar + - bookkeeper-tools-framework-4.16.4.jar + - circe-checksum-4.16.4.jar + - codahale-metrics-provider-4.16.4.jar + - cpu-affinity-4.16.4.jar + - http-server-4.16.4.jar + - prometheus-metrics-provider-4.16.4.jar + - codahale-metrics-provider-4.16.4.jar + - bookkeeper-slogger-api-4.16.4.jar + - bookkeeper-slogger-slf4j-4.16.4.jar + - native-io-4.16.4.jar * Apache Commons - commons-cli-1.5.0.jar - commons-codec-1.15.jar From ae57e0f2ea6bf77777d16533a146981a7fad8699 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 9 Feb 2024 16:58:41 -0800 Subject: [PATCH 273/699] [fix] Bump org.apache.solr:solr-core from 8.11.1 to 8.11.3 in /pulsar-io/solr (#22047) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> (cherry picked from commit 7a90426253e96a995e5d3a254c76cb80a3d54c7b) (cherry picked from commit c36da81ee0cb814dbfb60fd24e015f4bcb4aec75) --- pulsar-io/solr/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-io/solr/pom.xml b/pulsar-io/solr/pom.xml index d75932f046688..86b6345e15d96 100644 --- a/pulsar-io/solr/pom.xml +++ b/pulsar-io/solr/pom.xml @@ -29,7 +29,7 @@ - 8.11.1 + 8.11.3 pulsar-io-solr From 2e608f797ca2c1571c2bf6300ad5fbd176718987 Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Tue, 27 Feb 2024 00:03:16 +0800 Subject: [PATCH 274/699] [fix][test] Fix test testTransactionBufferMetrics (#22117) (cherry picked from commit 0fc9f4465288d1f9938ea717ea2e7c8ff02ebb60) (cherry picked from commit b10f9c8bffa8040493a2b53683e73537da70747b) --- .../broker/transaction/buffer/TransactionBufferClientTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java index 3873d9d37b20b..1dc086dbe3470 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java @@ -210,6 +210,8 @@ public void testAbortOnSubscription() throws ExecutionException, InterruptedExce @Test public void testTransactionBufferMetrics() throws Exception { + this.cleanup(); + this.setup(); //Test commit for (int i = 0; i < partitions; i++) { String topic = partitionedTopicName.getPartition(i).toString(); From 33d426da1e754c2324d689913744c492badd3d8d Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Sat, 24 Feb 2024 08:04:37 +0800 Subject: [PATCH 275/699] [improve][broker] Cache the internal writer when sent to system topic. (#22099) (cherry picked from commit 6e8ad696141f28c37c6fc0086cf8d797cdb64ebc) --- .../SystemTopicBasedTopicPoliciesService.java | 84 ++++++++++++------- .../TopicPoliciesSystemTopicClient.java | 10 ++- ...temTopicBasedTopicPoliciesServiceTest.java | 35 ++++++++ 3 files changed, 97 insertions(+), 32 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 80fecbe67b646..71f78e21f938f 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 @@ -19,6 +19,8 @@ package org.apache.pulsar.broker.service; import static java.util.Objects.requireNonNull; +import com.github.benmanes.caffeine.cache.AsyncLoadingCache; +import com.github.benmanes.caffeine.cache.Caffeine; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Sets; import java.util.HashSet; @@ -29,6 +31,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import javax.annotation.Nonnull; import org.apache.commons.lang3.tuple.MutablePair; @@ -84,10 +87,25 @@ public class SystemTopicBasedTopicPoliciesService implements TopicPoliciesServic @VisibleForTesting final Map>> listeners = new ConcurrentHashMap<>(); + private final AsyncLoadingCache> writerCaches; + public SystemTopicBasedTopicPoliciesService(PulsarService pulsarService) { this.pulsarService = pulsarService; this.clusterName = pulsarService.getConfiguration().getClusterName(); this.localCluster = Sets.newHashSet(clusterName); + this.writerCaches = Caffeine.newBuilder() + .expireAfterAccess(5, TimeUnit.MINUTES) + .removalListener((namespaceName, writer, cause) -> { + ((SystemTopicClient.Writer) writer).closeAsync().exceptionally(ex -> { + log.error("[{}] Close writer error.", namespaceName, ex); + return null; + }); + }) + .buildAsync((namespaceName, executor) -> { + SystemTopicClient systemTopicClient = namespaceEventsSystemTopicFactory + .createTopicPoliciesSystemTopicClient(namespaceName); + return systemTopicClient.newWriterAsync(); + }); } @Override @@ -122,39 +140,32 @@ private CompletableFuture sendTopicPolicyEvent(TopicName topicName, Action } catch (PulsarServerException e) { return CompletableFuture.failedFuture(e); } - - SystemTopicClient systemTopicClient = namespaceEventsSystemTopicFactory - .createTopicPoliciesSystemTopicClient(topicName.getNamespaceObject()); - - return systemTopicClient.newWriterAsync() - .thenCompose(writer -> { - PulsarEvent event = getPulsarEvent(topicName, actionType, policies); - CompletableFuture writeFuture = - ActionType.DELETE.equals(actionType) ? writer.deleteAsync(getEventKey(event), event) - : writer.writeAsync(getEventKey(event), event); - return writeFuture.handle((messageId, e) -> { - if (e != null) { - return CompletableFuture.failedFuture(e); + CompletableFuture result = new CompletableFuture<>(); + writerCaches.get(topicName.getNamespaceObject()) + .whenComplete((writer, cause) -> { + if (cause != null) { + writerCaches.synchronous().invalidate(topicName.getNamespaceObject()); + result.completeExceptionally(cause); } else { - if (messageId != null) { - return CompletableFuture.completedFuture(null); - } else { - return CompletableFuture.failedFuture( - new RuntimeException("Got message id is null.")); - } - } - }).thenRun(() -> - writer.closeAsync().whenComplete((v, cause) -> { - if (cause != null) { - log.error("[{}] Close writer error.", topicName, cause); + PulsarEvent event = getPulsarEvent(topicName, actionType, policies); + CompletableFuture writeFuture = ActionType.DELETE.equals(actionType) + ? writer.deleteAsync(getEventKey(event), event) + : writer.writeAsync(getEventKey(event), event); + writeFuture.whenComplete((messageId, e) -> { + if (e != null) { + result.completeExceptionally(e); + } else { + if (messageId != null) { + result.complete(null); } else { - if (log.isDebugEnabled()) { - log.debug("[{}] Close writer success.", topicName); - } + result.completeExceptionally( + new RuntimeException("Got message id is null.")); } - }) - ); + } + }); + } }); + return result; }); } @@ -364,7 +375,7 @@ public CompletableFuture removeOwnedNamespaceBundleAsync(NamespaceBundle n } AtomicInteger bundlesCount = ownedBundlesCountPerNamespace.get(namespace); if (bundlesCount == null || bundlesCount.decrementAndGet() <= 0) { - cleanCacheAndCloseReader(namespace, true); + cleanCacheAndCloseReader(namespace, true, true); } return CompletableFuture.completedFuture(null); } @@ -440,6 +451,14 @@ private void initPolicesCache(SystemTopicClient.Reader reader, Comp } private void cleanCacheAndCloseReader(@Nonnull NamespaceName namespace, boolean cleanOwnedBundlesCount) { + cleanCacheAndCloseReader(namespace, cleanOwnedBundlesCount, false); + } + + private void cleanCacheAndCloseReader(@Nonnull NamespaceName namespace, boolean cleanOwnedBundlesCount, + boolean cleanWriterCache) { + if (cleanWriterCache) { + writerCaches.synchronous().invalidate(namespace); + } CompletableFuture> readerFuture = readerCaches.remove(namespace); if (cleanOwnedBundlesCount) { @@ -688,5 +707,10 @@ protected Map>> getListeners( return listeners; } + @VisibleForTesting + protected AsyncLoadingCache> getWriterCaches() { + return writerCaches; + } + private static final Logger log = LoggerFactory.getLogger(SystemTopicBasedTopicPoliciesService.class); } 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 3fd8921c15efa..b7cff2e08c2d0 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 @@ -30,6 +30,8 @@ import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.TypedMessageBuilder; +import org.apache.pulsar.client.api.schema.SchemaDefinition; +import org.apache.pulsar.client.internal.DefaultImplementation; import org.apache.pulsar.common.events.ActionType; import org.apache.pulsar.common.events.PulsarEvent; import org.apache.pulsar.common.naming.TopicName; @@ -41,13 +43,17 @@ */ public class TopicPoliciesSystemTopicClient extends SystemTopicClientBase { + static Schema avroSchema = DefaultImplementation.getDefaultImplementation() + .newAvroSchema(SchemaDefinition.builder().withPojo(PulsarEvent.class).build()); + public TopicPoliciesSystemTopicClient(PulsarClient client, TopicName topicName) { super(client, topicName); + } @Override protected CompletableFuture> newWriterAsyncInternal() { - return client.newProducer(Schema.AVRO(PulsarEvent.class)) + return client.newProducer(avroSchema) .topic(topicName.toString()) .enableBatching(false) .createAsync() @@ -61,7 +67,7 @@ protected CompletableFuture> newWriterAsyncInternal() { @Override protected CompletableFuture> newReaderAsyncInternal() { - return client.newReader(Schema.AVRO(PulsarEvent.class)) + return client.newReader(avroSchema) .topic(topicName.toString()) .startMessageId(MessageId.earliest) .readCompacted(true) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java index ba5e42867d31f..cde41ffae6835 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java @@ -33,6 +33,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; 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; @@ -42,6 +43,7 @@ import org.apache.pulsar.broker.service.BrokerServiceException.TopicPoliciesCacheNotInitException; import org.apache.pulsar.broker.systopic.SystemTopicClient; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.Backoff; import org.apache.pulsar.client.impl.BackoffBuilder; import org.apache.pulsar.common.events.PulsarEvent; @@ -66,6 +68,8 @@ public class SystemTopicBasedTopicPoliciesServiceTest extends MockedPulsarServic private static final String NAMESPACE2 = "system-topic/namespace-2"; private static final String NAMESPACE3 = "system-topic/namespace-3"; + private static final String NAMESPACE4 = "system-topic/namespace-4"; + private static final TopicName TOPIC1 = TopicName.get("persistent", NamespaceName.get(NAMESPACE1), "topic-1"); private static final TopicName TOPIC2 = TopicName.get("persistent", NamespaceName.get(NAMESPACE1), "topic-2"); private static final TopicName TOPIC3 = TopicName.get("persistent", NamespaceName.get(NAMESPACE2), "topic-1"); @@ -428,4 +432,35 @@ public void testGetTopicPoliciesWithCleanCache() throws Exception { result.join(); } + + @Test + public void testWriterCache() throws Exception { + admin.namespaces().createNamespace(NAMESPACE4); + for (int i = 1; i <= 5; i ++) { + final String topicName = "persistent://" + NAMESPACE4 + "/testWriterCache" + i; + admin.topics().createNonPartitionedTopic(topicName); + pulsarClient.newProducer(Schema.STRING).topic(topicName).create().close(); + } + @Cleanup("shutdown") + ExecutorService executorService = Executors.newFixedThreadPool(5); + for (int i = 1; i <= 5; i ++) { + int finalI = i; + executorService.execute(() -> { + final String topicName = "persistent://" + NAMESPACE4 + "/testWriterCache" + finalI; + try { + admin.topicPolicies().setMaxConsumers(topicName, 2); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + SystemTopicBasedTopicPoliciesService service = (SystemTopicBasedTopicPoliciesService) pulsar.getTopicPoliciesService(); + Assert.assertNotNull(service.getWriterCaches().synchronous().get(NamespaceName.get(NAMESPACE4))); + for (int i = 1; i <= 5; i ++) { + final String topicName = "persistent://" + NAMESPACE4 + "/testWriterCache" + i; + admin.topics().delete(topicName); + } + admin.namespaces().deleteNamespace(NAMESPACE4); + Assert.assertNull(service.getWriterCaches().synchronous().getIfPresent(NamespaceName.get(NAMESPACE4))); + } } From 16ba4c93789ac5764477b3d2ec13f317e913731d Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 26 Feb 2024 11:06:01 +0800 Subject: [PATCH 276/699] [fix] [broker] Enabling batch causes negative unackedMessages due to ack and delivery concurrency (#22090) (cherry picked from commit aa3f53b3a311534abb36e71ab2121b7380b5b286) --- .../pulsar/broker/service/Consumer.java | 2 +- .../BatchMessageWithBatchIndexLevelTest.java | 182 ++++++++++++++++++ 2 files changed, 183 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 dc349aa798845..9485931304b0c 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 @@ -327,7 +327,7 @@ public Future sendMessages(final List entries, EntryBatch if (pendingAcks != null) { int batchSize = batchSizes.getBatchSize(i); int stickyKeyHash = getStickyKeyHash(entry); - long[] ackSet = getCursorAckSet(PositionImpl.get(entry.getLedgerId(), entry.getEntryId())); + long[] ackSet = batchIndexesAcks == null ? null : batchIndexesAcks.getAckSet(i); if (ackSet != null) { unackedMessages -= (batchSize - BitSet.valueOf(ackSet).cardinality()); } 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 b2fbe824b3305..3a4cee7f2be83 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 @@ -18,8 +18,17 @@ */ package org.apache.pulsar.broker.service; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyList; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; +import com.carrotsearch.hppc.ObjectSet; +import java.lang.reflect.Field; import java.util.ArrayList; import java.util.List; import java.util.UUID; @@ -28,10 +37,14 @@ import lombok.Cleanup; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.Entry; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.ConsumerBuilder; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageIdAdv; @@ -39,8 +52,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.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.ConsumerImpl; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.collections.BitSetRecyclable; import org.awaitility.Awaitility; import org.testng.Assert; import org.testng.annotations.BeforeClass; @@ -401,4 +416,171 @@ public void testMixIndexAndNonIndexUnAckMessageCount() throws Exception { assertEquals(admin.topics().getStats(topicName).getSubscriptions() .get("sub").getUnackedMessages(), 0); } + + @Test + public void testUnAckMessagesWhenConcurrentDeliveryAndAck() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://prop/ns-abc/tp"); + final String subName = "s1"; + final int receiverQueueSize = 500; + admin.topics().createNonPartitionedTopic(topicName); + admin.topics().createSubscription(topicName, subName, MessageId.earliest); + ConsumerBuilder consumerBuilder = pulsarClient.newConsumer(Schema.STRING) + .topic(topicName) + .receiverQueueSize(receiverQueueSize) + .subscriptionName(subName) + .enableBatchIndexAcknowledgment(true) + .subscriptionType(SubscriptionType.Shared) + .isAckReceiptEnabled(true); + + // Send 100 messages. + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topicName) + .enableBatching(true) + .batchingMaxPublishDelay(1, TimeUnit.HOURS) + .create(); + CompletableFuture lastSent = null; + for (int i = 0; i < 100; i++) { + lastSent = producer.sendAsync(i + ""); + } + producer.flush(); + lastSent.join(); + + // When consumer1 is closed, may some messages are in the client memory(it they are being acked now). + Consumer consumer1 = consumerBuilder.consumerName("c1").subscribe(); + Message[] messagesInClientMemory = new Message[2]; + for (int i = 0; i < 2; i++) { + Message msg = consumer1.receive(2, TimeUnit.SECONDS); + assertNotNull(msg); + messagesInClientMemory[i] = msg; + } + ConsumerImpl consumer2 = (ConsumerImpl) consumerBuilder.consumerName("c2").subscribe(); + Awaitility.await().until(() -> consumer2.isConnected()); + + // The consumer2 will receive messages after consumer1 closed. + // Insert a delay mechanism to make the flow like below: + // 1. Close consumer1, then the 100 messages will be redelivered. + // 2. Read redeliver messages. No messages were acked at this time. + // 3. The in-flight ack of two messages is finished. + // 4. Send the messages to consumer2, consumer2 will get all the 100 messages. + CompletableFuture receiveMessageSignal2 = new CompletableFuture<>(); + org.apache.pulsar.broker.service.Consumer serviceConsumer2 = + makeConsumerReceiveMessagesDelay(topicName, subName, "c2", receiveMessageSignal2); + // step 1: close consumer. + consumer1.close(); + // step 2: wait for read messages from replay queue. + Thread.sleep(2 * 1000); + // step 3: wait for the in-flight ack. + BitSetRecyclable bitSetRecyclable = createBitSetRecyclable(100); + long ledgerId = 0, entryId = 0; + for (Message message : messagesInClientMemory) { + BatchMessageIdImpl msgId = (BatchMessageIdImpl) message.getMessageId(); + bitSetRecyclable.clear(msgId.getBatchIndex()); + ledgerId = msgId.getLedgerId(); + entryId = msgId.getEntryId(); + } + getCursor(topicName, subName).delete(PositionImpl.get(ledgerId, entryId, bitSetRecyclable.toLongArray())); + // step 4: send messages to consumer2. + receiveMessageSignal2.complete(null); + // Verify: Consumer2 will get all the 100 messages, and "unAckMessages" is 100. + List messages2 = new ArrayList<>(); + while (true) { + Message msg = consumer2.receive(2, TimeUnit.SECONDS); + if (msg == null) { + break; + } + messages2.add(msg); + } + assertEquals(messages2.size(), 100); + assertEquals(serviceConsumer2.getUnackedMessages(), 100); + // After the messages were pop out, the permits in the client memory went to 100. + Awaitility.await().untilAsserted(() -> { + assertEquals(serviceConsumer2.getAvailablePermits() + consumer2.getAvailablePermits(), + receiverQueueSize); + }); + + // cleanup. + producer.close(); + consumer2.close(); + admin.topics().delete(topicName, false); + } + + private BitSetRecyclable createBitSetRecyclable(int batchSize) { + BitSetRecyclable bitSetRecyclable = new BitSetRecyclable(batchSize); + for (int i = 0; i < batchSize; i++) { + bitSetRecyclable.set(i); + } + return bitSetRecyclable; + } + + private ManagedCursorImpl getCursor(String topic, String sub) { + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + PersistentDispatcherMultipleConsumers dispatcher = + (PersistentDispatcherMultipleConsumers) persistentTopic.getSubscription(sub).getDispatcher(); + return (ManagedCursorImpl) dispatcher.getCursor(); + } + + /*** + * After {@param signal} complete, the consumer({@param consumerName}) start to receive messages. + */ + private org.apache.pulsar.broker.service.Consumer makeConsumerReceiveMessagesDelay(String topic, String sub, + String consumerName, + CompletableFuture signal) throws Exception { + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + PersistentDispatcherMultipleConsumers dispatcher = + (PersistentDispatcherMultipleConsumers) 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)) { + serviceConsumer = c; + break; + } + } + final org.apache.pulsar.broker.service.Consumer originalConsumer = serviceConsumer; + + // Insert a delay signal. + org.apache.pulsar.broker.service.Consumer spyServiceConsumer = spy(originalConsumer); + doAnswer(invocation -> { + List entries = (List) invocation.getArguments()[0]; + EntryBatchSizes batchSizes = (EntryBatchSizes) invocation.getArguments()[1]; + EntryBatchIndexesAcks batchIndexesAcks = (EntryBatchIndexesAcks) invocation.getArguments()[2]; + int totalMessages = (int) invocation.getArguments()[3]; + long totalBytes = (long) invocation.getArguments()[4]; + long totalChunkedMessages = (long) invocation.getArguments()[5]; + RedeliveryTracker redeliveryTracker = (RedeliveryTracker) invocation.getArguments()[6]; + return signal.thenApply(__ -> originalConsumer.sendMessages(entries, batchSizes, batchIndexesAcks, totalMessages, totalBytes, + totalChunkedMessages, redeliveryTracker)).join(); + }).when(spyServiceConsumer) + .sendMessages(anyList(), any(), any(), anyInt(), anyLong(), anyLong(), any()); + doAnswer(invocation -> { + List entries = (List) invocation.getArguments()[0]; + EntryBatchSizes batchSizes = (EntryBatchSizes) invocation.getArguments()[1]; + EntryBatchIndexesAcks batchIndexesAcks = (EntryBatchIndexesAcks) invocation.getArguments()[2]; + int totalMessages = (int) invocation.getArguments()[3]; + long totalBytes = (long) invocation.getArguments()[4]; + long totalChunkedMessages = (long) invocation.getArguments()[5]; + RedeliveryTracker redeliveryTracker = (RedeliveryTracker) invocation.getArguments()[6]; + long epoch = (long) invocation.getArguments()[7]; + return signal.thenApply(__ -> originalConsumer.sendMessages(entries, batchSizes, batchIndexesAcks, totalMessages, totalBytes, + totalChunkedMessages, redeliveryTracker, epoch)).join(); + }).when(spyServiceConsumer) + .sendMessages(anyList(), any(), any(), anyInt(), anyLong(), anyLong(), any(), anyLong()); + + // Replace the consumer. + Field fConsumerList = AbstractDispatcherMultipleConsumers.class.getDeclaredField("consumerList"); + Field fConsumerSet = AbstractDispatcherMultipleConsumers.class.getDeclaredField("consumerSet"); + fConsumerList.setAccessible(true); + fConsumerSet.setAccessible(true); + List consumerList = + (List) fConsumerList.get(dispatcher); + ObjectSet consumerSet = + (ObjectSet) fConsumerSet.get(dispatcher); + + consumerList.remove(originalConsumer); + consumerSet.removeAll(originalConsumer); + consumerList.add(spyServiceConsumer); + consumerSet.add(spyServiceConsumer); + return originalConsumer; + } } From e319a2f69f5d5d6d86fe3e4499c107d2e63db20f Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 26 Feb 2024 22:45:55 +0800 Subject: [PATCH 277/699] [fix] [client] fix huge permits if acked a half batched message (#22091) (cherry picked from commit b42787b5c5577be1c449ff480284ec5bfbc2664b) --- .../BatchMessageWithBatchIndexLevelTest.java | 85 +++++++++++++++++++ .../pulsar/client/impl/ConsumerBase.java | 5 ++ .../pulsar/client/impl/ConsumerImpl.java | 11 ++- 3 files changed, 99 insertions(+), 2 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 3a4cee7f2be83..8e902d5d1e700 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 @@ -583,4 +583,89 @@ private org.apache.pulsar.broker.service.Consumer makeConsumerReceiveMessagesDel consumerSet.add(spyServiceConsumer); return originalConsumer; } + + /*** + * 1. Send a batch message contains 100 single messages. + * 2. Ack 2 messages. + * 3. Redeliver the batch message and ack them. + * 4. Verify: the permits is correct. + */ + @Test + public void testPermitsIfHalfAckBatchMessage() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://prop/ns-abc/tp"); + final String subName = "s1"; + final int receiverQueueSize = 1000; + final int ackedMessagesCountInTheFistStep = 2; + admin.topics().createNonPartitionedTopic(topicName); + admin.topics(). createSubscription(topicName, subName, MessageId.earliest); + ConsumerBuilder consumerBuilder = pulsarClient.newConsumer(Schema.STRING) + .topic(topicName) + .receiverQueueSize(receiverQueueSize) + .subscriptionName(subName) + .enableBatchIndexAcknowledgment(true) + .subscriptionType(SubscriptionType.Shared) + .isAckReceiptEnabled(true); + + // Send 100 messages. + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topicName) + .enableBatching(true) + .batchingMaxPublishDelay(1, TimeUnit.HOURS) + .create(); + CompletableFuture lastSent = null; + for (int i = 1; i <= 100; i++) { + lastSent = producer. sendAsync(i + ""); + } + producer.flush(); + lastSent.join(); + + // Ack 2 messages, and trigger a redelivery. + Consumer consumer1 = consumerBuilder.subscribe(); + for (int i = 0; i < ackedMessagesCountInTheFistStep; i++) { + Message msg = consumer1. receive(2, TimeUnit.SECONDS); + assertNotNull(msg); + consumer1.acknowledge(msg); + } + consumer1.close(); + + // Receive the left 98 messages, and ack them. + // Verify the permits is correct. + ConsumerImpl consumer2 = (ConsumerImpl) consumerBuilder.subscribe(); + Awaitility.await().until(() -> consumer2.isConnected()); + List messages = new ArrayList<>(); + int nextMessageValue = ackedMessagesCountInTheFistStep + 1; + while (true) { + Message msg = consumer2.receive(2, TimeUnit.SECONDS); + if (msg == null) { + break; + } + assertEquals(msg.getValue(), nextMessageValue + ""); + messages.add(msg.getMessageId()); + nextMessageValue++; + } + assertEquals(messages.size(), 98); + consumer2.acknowledge(messages); + + org.apache.pulsar.broker.service.Consumer serviceConsumer2 = + getTheUniqueServiceConsumer(topicName, subName); + Awaitility.await().untilAsserted(() -> { + // After the messages were pop out, the permits in the client memory went to 98. + int permitsInClientMemory = consumer2.getAvailablePermits(); + int permitsInBroker = serviceConsumer2.getAvailablePermits(); + assertEquals(permitsInClientMemory + permitsInBroker, receiverQueueSize); + }); + + // cleanup. + producer.close(); + consumer2.close(); + admin.topics().delete(topicName, false); + } + + 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(); + return dispatcher.getConsumers().iterator().next(); + } } 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 fec428824c205..67bddf525c6b4 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 @@ -64,6 +64,7 @@ import org.apache.pulsar.common.api.proto.CommandSubscribe; 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; @@ -1266,6 +1267,10 @@ protected boolean isValidConsumerEpoch(MessageImpl message) { return true; } + protected boolean isSingleMessageAcked(BitSetRecyclable ackBitSet, int batchIndex) { + return ackBitSet != null && !ackBitSet.get(batchIndex); + } + public boolean hasBatchReceiveTimeout() { return batchReceiveTimeout != null; } 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 e2333065f8881..fa5a8bf3c5aa4 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 @@ -1189,7 +1189,7 @@ protected MessageImpl newSingleMessage(final int index, return null; } - if (ackBitSet != null && !ackBitSet.get(index)) { + if (isSingleMessageAcked(ackBitSet, index)) { return null; } @@ -1639,7 +1639,14 @@ void receiveIndividualMessagesFromBatch(BrokerEntryMetadata brokerEntryMetadata, singleMessageMetadata, uncompressedPayload, batchMessage, schema, true, ackBitSet, ackSetInMessageId, redeliveryCount, consumerEpoch); if (message == null) { - skippedMessages++; + // If it is not in ackBitSet, it means Broker does not want to deliver it to the client, and + // did not decrease the permits in the broker-side. + // So do not acquire more permits for this message. + // Why not skip this single message in the first line of for-loop block? We need call + // "newSingleMessage" to move "payload.readerIndex" to a correct value to get the correct data. + if (!isSingleMessageAcked(ackBitSet, i)) { + skippedMessages++; + } continue; } if (possibleToDeadLetter != null) { From de26aede31c658ad43c498de60cdcb0f7e28e20e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 15 Feb 2024 11:07:10 +0200 Subject: [PATCH 278/699] [fix][broker] Fix hash collision when using a consumer name that ends with a number (#22053) (cherry picked from commit ae2299c243f1a1d9eb7c8faa8b180496a429c771) --- ...stentHashingStickyKeyConsumerSelector.java | 14 ++-- ...tHashingStickyKeyConsumerSelectorTest.java | 74 +++++++++++++++---- ...ckyKeyDispatcherMultipleConsumersTest.java | 4 +- 3 files changed, 70 insertions(+), 22 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 ea491bd40d332..b2b2b512c8cfc 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 @@ -39,7 +39,8 @@ * number of keys assigned to each consumer. */ public class ConsistentHashingStickyKeyConsumerSelector implements StickyKeyConsumerSelector { - + // use NUL character as field separator for hash key calculation + private static final String KEY_SEPARATOR = "\0"; private final ReadWriteLock rwLock = new ReentrantReadWriteLock(); // Consistent-Hash ring @@ -59,8 +60,7 @@ public CompletableFuture addConsumer(Consumer 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++) { - String key = consumer.consumerName() + i; - int hash = Murmur3_32Hash.getInstance().makeHash(key.getBytes()); + int hash = calculateHashForConsumerAndIndex(consumer, i); hashRing.compute(hash, (k, v) -> { if (v == null) { return Lists.newArrayList(consumer); @@ -79,14 +79,18 @@ public CompletableFuture addConsumer(Consumer consumer) { } } + private static int calculateHashForConsumerAndIndex(Consumer consumer, int index) { + String key = consumer.consumerName() + KEY_SEPARATOR + index; + return Murmur3_32Hash.getInstance().makeHash(key.getBytes()); + } + @Override 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++) { - String key = consumer.consumerName() + i; - int hash = Murmur3_32Hash.getInstance().makeHash(key.getBytes()); + int hash = calculateHashForConsumerAndIndex(consumer, i); hashRing.compute(hash, (k, v) -> { if (v == null) { return null; 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 dbca31416bb9d..48311c57338b5 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 @@ -21,18 +21,18 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; - -import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerAssignException; -import org.apache.pulsar.client.api.Range; -import org.testng.Assert; -import org.testng.annotations.Test; - import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerAssignException; +import org.apache.pulsar.client.api.Range; +import org.testng.Assert; +import org.testng.annotations.Test; @Test(groups = "broker") public class ConsistentHashingStickyKeyConsumerSelectorTest { @@ -154,17 +154,17 @@ public void testGetConsumerKeyHashRanges() throws BrokerServiceException.Consume } Map> expectedResult = new HashMap<>(); expectedResult.put(consumers.get(0), Arrays.asList( - Range.of(0, 330121749), - Range.of(330121750, 618146114), - Range.of(1797637922, 1976098885))); + Range.of(119056335, 242013991), + Range.of(722195657, 1656011842), + Range.of(1707482098, 1914695766))); expectedResult.put(consumers.get(1), Arrays.asList( - Range.of(938427576, 1094135919), - Range.of(1138613629, 1342907082), - Range.of(1342907083, 1797637921))); + Range.of(0, 90164503), + Range.of(90164504, 119056334), + Range.of(382436668, 722195656))); expectedResult.put(consumers.get(2), Arrays.asList( - Range.of(618146115, 772640562), - Range.of(772640563, 938427575), - Range.of(1094135920, 1138613628))); + 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())); @@ -172,4 +172,48 @@ public void testGetConsumerKeyHashRanges() throws BrokerServiceException.Consume } Assert.assertEquals(expectedResult.size(), 0); } + + // reproduces https://github.com/apache/pulsar/issues/22050 + @Test + public void shouldNotCollideWithConsumerNameEndsWithNumber() { + ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(12); + List consumerName = Arrays.asList("consumer1", "consumer11"); + List consumers = new ArrayList<>(); + for (String s : consumerName) { + Consumer consumer = mock(Consumer.class); + when(consumer.consumerName()).thenReturn(s); + selector.addConsumer(consumer); + consumers.add(consumer); + } + Map rangeToConsumer = new HashMap<>(); + for (Map.Entry> entry : selector.getConsumerKeyHashRanges().entrySet()) { + for (Range range : entry.getValue()) { + Consumer previous = rangeToConsumer.put(range, entry.getKey()); + if (previous != null) { + Assert.fail("Ranges are colliding between " + previous.consumerName() + " and " + entry.getKey() + .consumerName()); + } + } + } + } + + @Test + public void shouldRemoveConsumersFromConsumerKeyHashRanges() { + ConsistentHashingStickyKeyConsumerSelector selector = new ConsistentHashingStickyKeyConsumerSelector(12); + List consumers = IntStream.range(1, 100).mapToObj(i -> "consumer" + i) + .map(consumerName -> { + Consumer consumer = mock(Consumer.class); + when(consumer.consumerName()).thenReturn(consumerName); + return consumer; + }).collect(Collectors.toList()); + + // when consumers are added + consumers.forEach(selector::addConsumer); + // then each consumer should have a range + Assert.assertEquals(selector.getConsumerKeyHashRanges().size(), consumers.size()); + // when consumers are removed + consumers.forEach(selector::removeConsumer); + // then there should be no mapping remaining + Assert.assertEquals(selector.getConsumerKeyHashRanges().size(), 0); + } } 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 48a4bfc923608..7e1b5f8c71e6d 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 @@ -250,7 +250,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, "key2"))); + readEntries.add(EntryImpl.create(1, 3, createMessage("message3", 3, "key22"))); try { Field totalAvailablePermitsField = PersistentDispatcherMultipleConsumers.class.getDeclaredField("totalAvailablePermits"); @@ -346,7 +346,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, "key2"))); + allEntries.add(EntryImpl.create(1, 1, createMessage("message1", 1, "key22"))); 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()); From 116d5cfe3ab77adc4c21f850d34890ef927107db Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Tue, 27 Feb 2024 15:38:40 +0800 Subject: [PATCH 279/699] [fix][test] fix test testSyncNormalPositionWhenTBRecover (#22120) ### Motivation 1. Change to None state before invoking the recovery. 2. Improve the method `checkTopicTransactionBufferState` to see the test result easier. ``` org.awaitility.core.ConditionTimeoutException: Condition with org.apache.pulsar.broker.transaction.buffer.TransactionStablePositionTest was not fulfilled within 10 seconds. at org.awaitility.core.ConditionAwaiter.await(ConditionAwaiter.java:167) at org.awaitility.core.CallableCondition.await(CallableCondition.java:78) at org.awaitility.core.CallableCondition.await(CallableCondition.java:26) at org.awaitility.core.ConditionFactory.until(ConditionFactory.java:985) at org.awaitility.core.ConditionFactory.until(ConditionFactory.java:954) at org.apache.pulsar.broker.transaction.buffer.TransactionStablePositionTest.checkTopicTransactionBufferState(TransactionStablePositionTest.java:239) at org.apache.pulsar.broker.transaction.buffer.TransactionStablePositionTest.testSyncNormalPositionWhenTBRecover(TransactionStablePositionTest.java:229) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77) at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.base/java.lang.reflect.Method.invoke(Method.java:568) at org.testng.internal.invokers.MethodInvocationHelper.invokeMethod(MethodInvocationHelper.java:139) at org.testng.internal.invokers.TestInvoker.invokeMethod(TestInvoker.java:677) at org.testng.internal.invokers.TestInvoker.invokeTestMethod(TestInvoker.java:221) at org.testng.internal.invokers.MethodRunner.runInSequence(MethodRunner.java:50) at org.testng.internal.invokers.TestInvoker$MethodInvocationAgent.invoke(TestInvoker.java:969) at org.testng.internal.invokers.TestInvoker.invokeTestMethods(TestInvoker.java:194) at org.testng.internal.invokers.TestMethodWorker.invokeTestMethods(TestMethodWorker.java:148) at org.testng.internal.invokers.TestMethodWorker.run(TestMethodWorker.java:128) at java.base/java.util.ArrayList.forEach(ArrayList.java:1511) at org.testng.TestRunner.privateRun(TestRunner.java:829) at org.testng.TestRunner.run(TestRunner.java:602) at org.testng.SuiteRunner.runTest(SuiteRunner.java:437) at org.testng.SuiteRunner.runSequentially(SuiteRunner.java:431) at org.testng.SuiteRunner.privateRun(SuiteRunner.java:391) at org.testng.SuiteRunner.run(SuiteRunner.java:330) at org.testng.SuiteRunnerWorker.runSuite(SuiteRunnerWorker.java:52) at org.testng.SuiteRunnerWorker.run(SuiteRunnerWorker.java:95) at org.testng.TestNG.runSuitesSequentially(TestNG.java:1256) at org.testng.TestNG.runSuitesLocally(TestNG.java:1176) at org.testng.TestNG.runSuites(TestNG.java:1099) at org.testng.TestNG.run(TestNG.java:1067) at com.intellij.rt.testng.IDEARemoteTestNG.run(IDEARemoteTestNG.java:65) at com.intellij.rt.testng.RemoteTestNGStarter.main(RemoteTestNGStarter.java:105) ``` ### Modifications 1. Change to None state before invoking the recovery. 2. Improve the method `checkTopicTransactionBufferState` to see the test result easier. (cherry picked from commit 2a6b644bb6d22cde910be5eca44a2c201c13b6bb) --- .../buffer/TransactionStablePositionTest.java | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionStablePositionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionStablePositionTest.java index 7493b25ac1d90..6dcf7931d73f4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionStablePositionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionStablePositionTest.java @@ -213,14 +213,14 @@ public void testSyncNormalPositionWhenTBRecover(boolean clientEnableTransaction, position = topicTransactionBuffer.getMaxReadPosition(); assertEquals(position, PositionImpl.EARLIEST); + // change to None state can recover + field.set(topicTransactionBuffer, TopicTransactionBufferState.State.None); + // invoke recover Method method = TopicTransactionBuffer.class.getDeclaredMethod("recover"); method.setAccessible(true); method.invoke(topicTransactionBuffer); - // change to None state can recover - field.set(topicTransactionBuffer, TopicTransactionBufferState.State.None); - // recover success again checkTopicTransactionBufferState(clientEnableTransaction, topicTransactionBuffer); @@ -232,13 +232,15 @@ public void testSyncNormalPositionWhenTBRecover(boolean clientEnableTransaction, private void checkTopicTransactionBufferState(boolean clientEnableTransaction, TopicTransactionBuffer topicTransactionBuffer) { // recover success - Awaitility.await().until(() -> { + Awaitility.await().untilAsserted(() -> { if (clientEnableTransaction) { // recover success, client enable transaction will change to Ready State - return topicTransactionBuffer.getStats(false, false).state.equals(Ready.name()); + assertEquals(topicTransactionBuffer.getStats(false, false).state, + Ready.name()); } else { // recover success, client disable transaction will change to NoSnapshot State - return topicTransactionBuffer.getStats(false, false).state.equals(NoSnapshot.name()); + assertEquals(topicTransactionBuffer.getStats(false, false).state, + NoSnapshot.name()); } }); } From ee35c45582a4dc35e2c6d15e9a5d34147d95ed8d Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Tue, 6 Feb 2024 15:28:43 +0800 Subject: [PATCH 280/699] [improve][broker] Do not retain the data in the system topic (#22022) ### Motivation For some use case, the users need to store all the messages even though these message are acked by all subscription. So they set the retention policy of the namespace to infinite retention (setting both time and size limits to `-1`). But the data in the system topic does not need for infinite retention. ### Modifications For system topics, do not retain messages that have already been acknowledged. (cherry picked from commit 4e97827fe7bb50ea3c5d95735764ffba0099f14f) --- .../pulsar/broker/service/BrokerService.java | 15 ++++-- .../compaction/CompactionRetentionTest.java | 48 +++++++++++++++++++ 2 files changed, 59 insertions(+), 4 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 a6db3e6aeaa33..76cd460d3d8f6 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 @@ -1839,10 +1839,17 @@ private CompletableFuture getManagedLedgerConfig(@Nonnull T } if (retentionPolicies == null) { - retentionPolicies = policies.map(p -> p.retention_policies).orElseGet( - () -> new RetentionPolicies(serviceConfig.getDefaultRetentionTimeInMinutes(), - serviceConfig.getDefaultRetentionSizeInMB()) - ); + if (SystemTopicNames.isSystemTopic(topicName)) { + if (log.isDebugEnabled()) { + log.debug("{} Disable data retention policy for system topic.", topicName); + } + retentionPolicies = new RetentionPolicies(0, 0); + } else { + retentionPolicies = policies.map(p -> p.retention_policies).orElseGet( + () -> new RetentionPolicies(serviceConfig.getDefaultRetentionTimeInMinutes(), + serviceConfig.getDefaultRetentionSizeInMB()) + ); + } } ManagedLedgerConfig managedLedgerConfig = new ManagedLedgerConfig(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionRetentionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionRetentionTest.java index 055c595fbfec8..98bf2b819c2ba 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionRetentionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionRetentionTest.java @@ -38,6 +38,7 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; @@ -45,9 +46,13 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Reader; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.naming.SystemTopicNames; +import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.awaitility.Awaitility; +import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -212,6 +217,49 @@ public void testCompactionRetentionOnTopicCreationWithTopicPolicies() throws Exc ); } + @Test + public void testRetentionPolicesForSystemTopic() throws Exception { + String namespace = "my-tenant/my-ns"; + String topicPrefix = "persistent://" + namespace + "/"; + admin.namespaces().setRetention(namespace, new RetentionPolicies(-1, -1)); + // Check event topics and transaction internal topics. + for (String eventTopic : SystemTopicNames.EVENTS_TOPIC_NAMES) { + checkSystemTopicRetentionPolicy(topicPrefix + eventTopic); + } + checkSystemTopicRetentionPolicy(topicPrefix + SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN); + checkSystemTopicRetentionPolicy(topicPrefix + SystemTopicNames.TRANSACTION_COORDINATOR_LOG); + checkSystemTopicRetentionPolicy(topicPrefix + SystemTopicNames.PENDING_ACK_STORE_SUFFIX); + + // Check common topics. + checkCommonTopicRetentionPolicy(topicPrefix + "my-topic" + System.nanoTime()); + // Specify retention policies for system topic. + pulsar.getConfiguration().setTopicLevelPoliciesEnabled(true); + pulsar.getConfiguration().setSystemTopicEnabled(true); + admin.topics().createNonPartitionedTopic(topicPrefix + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT); + admin.topicPolicies().setRetention(topicPrefix + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT, + new RetentionPolicies(10, 10)); + Awaitility.await().untilAsserted(() -> { + checkTopicRetentionPolicy(topicPrefix + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT, + new RetentionPolicies(10, 10)); + }); + } + + private void checkSystemTopicRetentionPolicy(String topicName) throws Exception { + checkTopicRetentionPolicy(topicName, new RetentionPolicies(0, 0)); + + } + + private void checkCommonTopicRetentionPolicy(String topicName) throws Exception { + checkTopicRetentionPolicy(topicName, new RetentionPolicies(-1, -1)); + } + + private void checkTopicRetentionPolicy(String topicName, RetentionPolicies retentionPolicies) throws Exception { + ManagedLedgerConfig config = pulsar.getBrokerService() + .getManagedLedgerConfig(TopicName.get(topicName)).get(); + Assert.assertEquals(config.getRetentionSizeInMB(), retentionPolicies.getRetentionSizeInMB()); + Assert.assertEquals(config.getRetentionTimeMillis(),retentionPolicies.getRetentionTimeInMinutes() * 60000L); + } + private void testCompactionCursorRetention(String topic) throws Exception { Set keys = Sets.newHashSet("a", "b", "c"); Set keysToExpire = Sets.newHashSet("x1", "x2"); From cb0703d84e95a8df0b83ae58b0c9bbd3a2975ca6 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Sun, 18 Feb 2024 15:46:52 +0800 Subject: [PATCH 281/699] [improve] [broker] Do not print an Error log when responding to `HTTP-404` when calling `Admin API` and the topic does not exist. (#21995) (cherry picked from commit 5ab1c054abf61bab10077537cdd13d2288b1c0a4) --- .../pulsar/broker/admin/AdminResource.java | 4 + .../admin/impl/PersistentTopicsBase.java | 88 +++++++++---------- .../admin/impl/SchemasResourceBase.java | 2 +- .../broker/admin/v2/NonPersistentTopics.java | 6 +- .../broker/admin/v2/PersistentTopics.java | 36 ++++---- .../pulsar/broker/admin/v3/Transactions.java | 12 +-- 6 files changed, 75 insertions(+), 73 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 828b18c2df9ab..7de426fa1f02f 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 @@ -831,6 +831,10 @@ protected static boolean isNotFoundException(Throwable ex) { == Status.NOT_FOUND.getStatusCode(); } + protected static boolean isNot307And404Exception(Throwable ex) { + return !isRedirectException(ex) && !isNotFoundException(ex); + } + protected static String getTopicNotFoundErrorMessage(String topic) { return String.format("Topic %s not found", topic); } 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 7892861d5df1e..c5a8d83f9c573 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 @@ -887,7 +887,7 @@ protected void internalUnloadTopic(AsyncResponse asyncResponse, boolean authorit } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned metadata while unloading topic {}", clientAppId(), topicName, ex); } @@ -897,7 +897,7 @@ protected void internalUnloadTopic(AsyncResponse asyncResponse, boolean authorit } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to validate the global namespace ownership while unloading topic {}", clientAppId(), topicName, ex); } @@ -1065,7 +1065,7 @@ private void internalUnloadNonPartitionedTopicAsync(AsyncResponse asyncResponse, })) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to unload topic {}, {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1087,7 +1087,7 @@ private void internalUnloadTransactionCoordinatorAsync(AsyncResponse asyncRespon })) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to unload tc {},{}", clientAppId(), topicName.getPartitionIndex(), ex); } @@ -1189,7 +1189,7 @@ protected void internalGetSubscriptions(AsyncResponse asyncResponse, boolean aut } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned topic metadata while get" + " subscriptions for topic {}", clientAppId(), topicName, ex); } @@ -1199,7 +1199,7 @@ protected void internalGetSubscriptions(AsyncResponse asyncResponse, boolean aut } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to validate the global namespace/topic ownership while get subscriptions" + " for topic {}", clientAppId(), topicName, ex); } @@ -1208,7 +1208,7 @@ protected void internalGetSubscriptions(AsyncResponse asyncResponse, boolean aut }) ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get subscriptions for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1247,7 +1247,7 @@ private void internalGetSubscriptionsForNonPartitionedTopic(AsyncResponse asyncR .thenAccept(topic -> asyncResponse.resume(new ArrayList<>(topic.getSubscriptions().keys()))) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get list of subscriptions for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1359,7 +1359,7 @@ protected void internalGetManagedLedgerInfo(AsyncResponse asyncResponse, boolean } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned metadata while get managed info for {}", clientAppId(), topicName, ex); } @@ -1369,7 +1369,7 @@ protected void internalGetManagedLedgerInfo(AsyncResponse asyncResponse, boolean } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to validate the global namespace ownership while get managed info for {}", clientAppId(), topicName, ex); } @@ -1483,7 +1483,7 @@ protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned internal stats for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1538,7 +1538,7 @@ protected void internalGetPartitionedStatsInternal(AsyncResponse asyncResponse, }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned internal stats for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1666,7 +1666,7 @@ private void internalAnalyzeSubscriptionBacklogForNonPartitionedTopic(AsyncRespo }).exceptionally(ex -> { Throwable cause = ex.getCause(); // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to analyze subscription backlog {} {}", clientAppId(), topicName, subName, cause); } @@ -1693,7 +1693,7 @@ private void internalUpdateSubscriptionPropertiesForNonPartitionedTopic(AsyncRes }).exceptionally(ex -> { Throwable cause = ex.getCause(); // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to update subscription {} {}", clientAppId(), topicName, subName, cause); } asyncResponse.resume(new RestException(cause)); @@ -1722,7 +1722,7 @@ private void internalGetSubscriptionPropertiesForNonPartitionedTopic(AsyncRespon }).exceptionally(ex -> { Throwable cause = ex.getCause(); // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to update subscription {} {}", clientAppId(), topicName, subName, cause); } asyncResponse.resume(new RestException(cause)); @@ -1891,7 +1891,7 @@ protected void internalSkipAllMessages(AsyncResponse asyncResponse, String subNa } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to skip all messages for subscription {} on topic {}", clientAppId(), subName, topicName, ex); } @@ -1935,7 +1935,7 @@ private CompletableFuture internalSkipAllMessagesForNonPartitionedTopicAsy } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to skip all messages for subscription {} on topic {}", clientAppId(), subName, topicName, ex); } @@ -1999,7 +1999,7 @@ protected void internalSkipMessages(AsyncResponse asyncResponse, String subName, }) ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to skip {} messages {} {}", clientAppId(), numMessages, topicName, subName, ex); } @@ -2069,7 +2069,7 @@ protected void internalExpireMessagesForAllSubscriptions(AsyncResponse asyncResp ) ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to expire messages for all subscription on topic {}", clientAppId(), topicName, ex); } @@ -2136,7 +2136,7 @@ private void internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(Asy }) ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to expire messages for all subscription up to {} on {}", clientAppId(), expireTimeInSeconds, topicName, ex); } @@ -2343,7 +2343,7 @@ protected void internalCreateSubscription(AsyncResponse asyncResponse, String su })).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to create subscription {} on topic {}", clientAppId(), subscriptionName, topicName, ex); } @@ -2353,7 +2353,7 @@ protected void internalCreateSubscription(AsyncResponse asyncResponse, String su } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to create subscription {} on topic {}", clientAppId(), subscriptionName, topicName, ex); } @@ -2484,7 +2484,7 @@ protected void internalUpdateSubscriptionProperties(AsyncResponse asyncResponse, } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to update subscription {} from topic {}", clientAppId(), subName, topicName, ex); } @@ -2524,7 +2524,7 @@ protected void internalAnalyzeSubscriptionBacklog(AsyncResponse asyncResponse, S }) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to analyze back log of subscription {} from topic {}", clientAppId(), subName, topicName, ex); } @@ -2609,7 +2609,7 @@ protected void internalGetSubscriptionProperties(AsyncResponse asyncResponse, St } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to update subscription {} from topic {}", clientAppId(), subName, topicName, ex); } @@ -2695,7 +2695,7 @@ protected void internalResetCursorOnPosition(AsyncResponse asyncResponse, String }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.warn("[{}][{}] Failed to reset cursor on subscription {} to position {}", clientAppId(), topicName, subName, messageId, ex.getCause()); } @@ -2704,7 +2704,7 @@ protected void internalResetCursorOnPosition(AsyncResponse asyncResponse, String }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.warn("[{}][{}] Failed to reset cursor on subscription {} to position {}", clientAppId(), topicName, subName, messageId, ex.getCause()); } @@ -3328,7 +3328,7 @@ protected void internalGetBacklogSizeByMessageId(AsyncResponse asyncResponse, } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get backlog size for topic {}", clientAppId(), topicName, ex); } @@ -3336,7 +3336,7 @@ protected void internalGetBacklogSizeByMessageId(AsyncResponse asyncResponse, return null; })).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to validate global namespace ownership " + "to get backlog size for topic {}", clientAppId(), topicName, ex); } @@ -3874,7 +3874,7 @@ protected void internalTerminatePartitionedTopic(AsyncResponse asyncResponse, bo } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to terminate topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -3882,7 +3882,7 @@ protected void internalTerminatePartitionedTopic(AsyncResponse asyncResponse, bo }) ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to terminate topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -3972,7 +3972,7 @@ protected void internalExpireMessagesByTimestamp(AsyncResponse asyncResponse, St ).exceptionally(ex -> { Throwable cause = FutureUtil.unwrapCompletionException(ex); // If the exception is not redirect exception we need to log it. - if (!isRedirectException(cause)) { + if (!isNot307And404Exception(cause)) { if (cause instanceof RestException) { log.warn("[{}] Failed to expire messages up to {} on {}: {}", clientAppId(), expireTimeInSeconds, topicName, cause.toString()); @@ -4087,7 +4087,7 @@ protected void internalExpireMessagesByPosition(AsyncResponse asyncResponse, Str messageId, isExcluded, batchIndex); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to expire messages up to {} on subscription {} to position {}", clientAppId(), topicName, subName, messageId, ex); } @@ -4237,7 +4237,7 @@ protected void internalTriggerCompaction(AsyncResponse asyncResponse, boolean au } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to trigger compaction on topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -4246,7 +4246,7 @@ protected void internalTriggerCompaction(AsyncResponse asyncResponse, boolean au } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to validate global namespace ownership to trigger compaction on topic {}", clientAppId(), topicName, ex); } @@ -4275,7 +4275,7 @@ protected void internalTriggerCompactionNonPartitionedTopic(AsyncResponse asyncR } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to trigger compaction for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -4311,7 +4311,7 @@ protected void internalTriggerOffload(AsyncResponse asyncResponse, } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to trigger offload for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -4328,7 +4328,7 @@ protected void internalOffloadStatus(AsyncResponse asyncResponse, boolean author asyncResponse.resume(offloadProcessStatus); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to offload status on topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -4653,7 +4653,7 @@ protected void internalGetLastMessageId(AsyncResponse asyncResponse, boolean aut }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get last messageId {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -5031,9 +5031,7 @@ protected CompletableFuture internalRemoveSubscribeRate(boolean isGlobal) protected void handleTopicPolicyException(String methodName, Throwable thr, AsyncResponse asyncResponse) { Throwable cause = thr.getCause(); - if (!(cause instanceof WebApplicationException) || !( - ((WebApplicationException) cause).getResponse().getStatus() == 307 - || ((WebApplicationException) cause).getResponse().getStatus() == 404)) { + if (isNot307And404Exception(cause)) { log.error("[{}] Failed to perform {} on topic {}", clientAppId(), methodName, topicName, cause); } @@ -5159,7 +5157,7 @@ protected void internalSetReplicatedSubscriptionStatus(AsyncResponse asyncRespon resultFuture.exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.warn("[{}] Failed to change replicated subscription status to {} - {} {}", clientAppId(), enabled, topicName, subName, ex); } @@ -5206,7 +5204,7 @@ private void internalSetReplicatedSubscriptionStatusForNonPartitionedTopic( } ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to set replicated subscription status on {} {}", clientAppId(), topicName, subName, ex); } @@ -5307,7 +5305,7 @@ protected void internalGetReplicatedSubscriptionStatus(AsyncResponse asyncRespon } resultFuture.exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get replicated subscription status on {} {}", clientAppId(), topicName, subName, ex); } 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 454b8f0fac82c..286366c8b5834 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 @@ -238,7 +238,7 @@ private CompletableFuture validateOwnershipAndOperationAsync(boolean autho protected boolean shouldPrintErrorLog(Throwable ex) { - return !isRedirectException(ex) && !isNotFoundException(ex); + return isNot307And404Exception(ex); } private static final Logger log = LoggerFactory.getLogger(SchemasResourceBase.class); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java index 386b9749ef959..e0a1b8f89ee3f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java @@ -132,7 +132,7 @@ public void getInternalStats( }) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get internal stats for topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -468,7 +468,7 @@ public void getListFromBundle( } asyncResponse.resume(topicList); }).exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to list topics on namespace bundle {}/{}", clientAppId(), namespaceName, bundleRange, ex); } @@ -477,7 +477,7 @@ public void getListFromBundle( }); } }).exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to list topics on namespace bundle {}/{}", clientAppId(), namespaceName, bundleRange, ex); } 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 1927d4b244aa4..b8457339213c0 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 @@ -117,7 +117,7 @@ public void getList( internalGetListAsync(Optional.ofNullable(bundle)) .thenAccept(topicList -> asyncResponse.resume(filterSystemTopic(topicList, includeSystemTopic))) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get topic list {}", clientAppId(), namespaceName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -148,7 +148,7 @@ public void getPartitionedTopicList( .thenAccept(partitionedTopicList -> asyncResponse.resume( filterSystemTopic(partitionedTopicList, includeSystemTopic))) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned topic list {}", clientAppId(), namespaceName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -333,7 +333,7 @@ public void createNonPartitionedTopic( internalCreateNonPartitionedTopicAsync(authoritative, properties) .thenAccept(__ -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to create non-partitioned topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -822,7 +822,7 @@ public void updatePartitionedTopic( asyncResponse.resume(Response.noContent().build()); }) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}][{}] Failed to update partition to {}", clientAppId(), topicName, numPartitions, ex); } @@ -924,7 +924,7 @@ public void getProperties( internalGetPropertiesAsync(authoritative) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get topic {} properties", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -960,7 +960,7 @@ public void updateProperties( internalUpdatePropertiesAsync(authoritative, properties) .thenAccept(__ -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to update topic {} properties", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -994,7 +994,7 @@ public void removeProperties( internalRemovePropertiesAsync(authoritative, key) .thenAccept(__ -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to remove key {} in properties on topic {}", clientAppId(), key, topicName, ex); } @@ -1115,7 +1115,7 @@ public void deleteTopic( } else if (isManagedLedgerNotFoundException(t)) { ex = new RestException(Response.Status.NOT_FOUND, getTopicNotFoundErrorMessage(topicName.toString())); - } else if (!isRedirectException(ex)) { + } else if (isNot307And404Exception(ex)) { log.error("[{}] Failed to delete topic {}", clientAppId(), topicName, t); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1192,7 +1192,7 @@ public void getStats( .thenAccept(asyncResponse::resume) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get stats for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1226,7 +1226,7 @@ public void getInternalStats( internalGetInternalStatsAsync(authoritative, metadata) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get internal stats for topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1870,7 +1870,7 @@ public void peekNthMessage( internalPeekNthMessageAsync(decode(encodedSubName), messagePosition, authoritative) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get peek nth message for topic {} subscription {}", clientAppId(), topicName, decode(encodedSubName), ex); } @@ -1912,7 +1912,7 @@ public void examineMessage( internalExamineMessageAsync(initialPosition, messagePosition, authoritative) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to examine a specific message on the topic {}", clientAppId(), topicName, ex); } @@ -1954,7 +1954,7 @@ public void getMessageById( .thenAccept(asyncResponse::resume) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get message with ledgerId {} entryId {} from {}", clientAppId(), ledgerId, entryId, topicName, ex); } @@ -1998,7 +1998,7 @@ public void getMessageIdByTimestamp( } }) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get message ID by timestamp {} from {}", clientAppId(), timestamp, topicName, ex); } @@ -2033,7 +2033,7 @@ public void getBacklog( log.warn("[{}] Failed to get topic backlog {}: Namespace does not exist", clientAppId(), namespaceName); ex = new RestException(Response.Status.NOT_FOUND, "Namespace does not exist"); - } else if (!isRedirectException(ex)) { + } else if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get estimated backlog for topic {}", clientAppId(), encodedTopic, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -3071,7 +3071,7 @@ public void terminate( internalTerminateAsync(authoritative) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to terminated topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -3167,7 +3167,7 @@ public void compactionStatus( internalCompactionStatusAsync(authoritative) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get the status of a compaction operation for the topic {}", clientAppId(), topicName, ex); } @@ -3306,7 +3306,7 @@ public void trimTopic( validateTopicName(tenant, namespace, encodedTopic); internalTrimTopic(asyncResponse, authoritative).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isRedirectException(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to trim topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java index 1bdc2255085f1..667d8ce581ece 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java @@ -105,7 +105,7 @@ public void getTransactionInBufferStats(@Suspended final AsyncResponse asyncResp Long.parseLong(leastSigBits)) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get transaction state in transaction buffer {}", clientAppId(), topicName, ex); } @@ -143,7 +143,7 @@ public void getTransactionInPendingAckStats(@Suspended final AsyncResponse async Long.parseLong(leastSigBits), subName) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get transaction state in pending ack {}", clientAppId(), topicName, ex); } @@ -181,7 +181,7 @@ public void getTransactionBufferStats(@Suspended final AsyncResponse asyncRespon internalGetTransactionBufferStats(authoritative, lowWaterMarks, segmentStats) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get transaction buffer stats in topic {}", clientAppId(), topicName, ex); } @@ -217,7 +217,7 @@ public void getPendingAckStats(@Suspended final AsyncResponse asyncResponse, internalGetPendingAckStats(authoritative, subName, lowWaterMarks) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get transaction pending ack stats in topic {}", clientAppId(), topicName, ex); } @@ -314,7 +314,7 @@ public void getPendingAckInternalStats(@Suspended final AsyncResponse asyncRespo internalGetPendingAckInternalStats(authoritative, subName, metadata) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get pending ack internal stats {}", clientAppId(), topicName, ex); } @@ -365,7 +365,7 @@ public void getTransactionBufferInternalStats(@Suspended final AsyncResponse asy internalGetTransactionBufferInternalStats(authoritative, metadata) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isRedirectException(ex)) { + if (!isNot307And404Exception(ex)) { log.error("[{}] Failed to get transaction buffer internal stats {}", clientAppId(), topicName, ex); } From c7091ca7303dfc31dcbe9dd18579c5fe3a68e588 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 19 Feb 2024 00:04:10 +0800 Subject: [PATCH 282/699] [fix] [broker] Subscription stuck due to called Admin API analyzeSubscriptionBacklog (#22019) (cherry picked from commit 2045c5f9e532447f6b72909bb835062f80f011ee) --- .../mledger/impl/ManagedCursorImpl.java | 29 ++++++++++++++++-- .../persistent/PersistentSubscription.java | 30 ++++++++++++++++--- .../pulsar/broker/admin/AdminApi2Test.java | 29 ++++++++++++++++++ .../admin/AnalyzeBacklogSubscriptionTest.java | 18 +++++------ .../util/collections/BitSetRecyclable.java | 8 +++++ 5 files changed, 99 insertions(+), 15 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 ffa92c4f92162..2abf6ca9d1743 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 @@ -196,11 +196,11 @@ public class ManagedCursorImpl implements ManagedCursor { position.ackSet = null; return position; }; - private final RangeSetWrapper individualDeletedMessages; + protected final RangeSetWrapper individualDeletedMessages; // Maintain the deletion status for batch messages // (ledgerId, entryId) -> deletion indexes - private final ConcurrentSkipListMap batchDeletedIndexes; + protected final ConcurrentSkipListMap batchDeletedIndexes; private final ReadWriteLock lock = new ReentrantReadWriteLock(); private RateLimiter markDeleteLimiter; @@ -3588,4 +3588,29 @@ public boolean isCacheReadEntry() { public ManagedLedgerConfig getConfig() { return config; } + + /*** + * Create a non-durable cursor and copy the ack stats. + */ + public ManagedCursor duplicateNonDurableCursor(String nonDurableCursorName) throws ManagedLedgerException { + NonDurableCursorImpl newNonDurableCursor = + (NonDurableCursorImpl) ledger.newNonDurableCursor(getMarkDeletedPosition(), nonDurableCursorName); + if (individualDeletedMessages != null) { + this.individualDeletedMessages.forEach(range -> { + newNonDurableCursor.individualDeletedMessages.addOpenClosed( + range.lowerEndpoint().getLedgerId(), + range.lowerEndpoint().getEntryId(), + range.upperEndpoint().getLedgerId(), + range.upperEndpoint().getEntryId()); + return true; + }); + } + if (batchDeletedIndexes != null) { + for (Map.Entry entry : this.batchDeletedIndexes.entrySet()) { + BitSetRecyclable copiedBitSet = BitSetRecyclable.valueOf(entry.getValue()); + newNonDurableCursor.batchDeletedIndexes.put(entry.getKey(), copiedBitSet); + } + } + return newNonDurableCursor; + } } 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 cf4af49437b8a..b5852610c20cd 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 @@ -29,6 +29,7 @@ import java.util.Map; import java.util.Optional; import java.util.TreeMap; +import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; @@ -511,9 +512,15 @@ public String getTypeString() { return "Null"; } - @Override public CompletableFuture analyzeBacklog(Optional position) { - + final ManagedLedger managedLedger = topic.getManagedLedger(); + final String newNonDurableCursorName = "analyze-backlog-" + UUID.randomUUID(); + ManagedCursor newNonDurableCursor; + try { + newNonDurableCursor = ((ManagedCursorImpl) cursor).duplicateNonDurableCursor(newNonDurableCursorName); + } catch (ManagedLedgerException e) { + return CompletableFuture.failedFuture(e); + } long start = System.currentTimeMillis(); if (log.isDebugEnabled()) { log.debug("[{}][{}] Starting to analyze backlog", topicName, subName); @@ -528,7 +535,7 @@ public CompletableFuture analyzeBacklog(Optional AtomicLong rejectedMessages = new AtomicLong(); AtomicLong rescheduledMessages = new AtomicLong(); - Position currentPosition = cursor.getMarkDeletedPosition(); + Position currentPosition = newNonDurableCursor.getMarkDeletedPosition(); if (log.isDebugEnabled()) { log.debug("[{}][{}] currentPosition {}", @@ -588,7 +595,7 @@ public CompletableFuture analyzeBacklog(Optional return true; }; - return cursor.scan( + CompletableFuture res = newNonDurableCursor.scan( position, condition, batchSize, @@ -615,7 +622,22 @@ public CompletableFuture analyzeBacklog(Optional topicName, subName, end - start, result); return result; }); + res.whenComplete((__, ex) -> { + managedLedger.asyncDeleteCursor(newNonDurableCursorName, + new AsyncCallbacks.DeleteCursorCallback(){ + @Override + public void deleteCursorComplete(Object ctx) { + // Nothing to do. + } + @Override + public void deleteCursorFailed(ManagedLedgerException exception, Object ctx) { + log.warn("[{}][{}] Delete non-durable cursor[{}] failed when analyze backlog.", + topicName, subName, newNonDurableCursor.getName()); + } + }, null); + }); + return res; } @Override 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 e300019b5ed22..fc4aaf10f966c 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 @@ -3394,4 +3394,33 @@ private void testSetBacklogQuotasNamespaceLevelIfRetentionExists() throws Except // cleanup. admin.namespaces().deleteNamespace(ns); } + + @Test + private void testAnalyzeSubscriptionBacklogNotCauseStuck() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp"); + final String subscription = "s1"; + admin.topics().createNonPartitionedTopic(topic); + // Send 10 messages. + Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topic).subscriptionName(subscription) + .receiverQueueSize(0).subscribe(); + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + for (int i = 0; i < 10; i++) { + producer.send(i + ""); + } + + // Verify consumer can receive all messages after calling "analyzeSubscriptionBacklog". + admin.topics().analyzeSubscriptionBacklog(topic, subscription, Optional.of(MessageIdImpl.earliest)); + for (int i = 0; i < 10; i++) { + Awaitility.await().untilAsserted(() -> { + Message m = consumer.receive(); + assertNotNull(m); + consumer.acknowledge(m); + }); + } + + // cleanup. + consumer.close(); + producer.close(); + admin.topics().delete(topic); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AnalyzeBacklogSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AnalyzeBacklogSubscriptionTest.java index 64b2a58ab86e8..f8aa3dc355d92 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AnalyzeBacklogSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AnalyzeBacklogSubscriptionTest.java @@ -154,17 +154,17 @@ private void verifyBacklog(String topic, String subscription, int numEntries, in AnalyzeSubscriptionBacklogResult analyzeSubscriptionBacklogResult = admin.topics().analyzeSubscriptionBacklog(topic, subscription, Optional.empty()); - assertEquals(numEntries, analyzeSubscriptionBacklogResult.getEntries()); - assertEquals(numEntries, analyzeSubscriptionBacklogResult.getFilterAcceptedEntries()); - assertEquals(0, analyzeSubscriptionBacklogResult.getFilterRejectedEntries()); - assertEquals(0, analyzeSubscriptionBacklogResult.getFilterRescheduledEntries()); - assertEquals(0, analyzeSubscriptionBacklogResult.getFilterRescheduledEntries()); + assertEquals(analyzeSubscriptionBacklogResult.getEntries(), numEntries); + assertEquals(analyzeSubscriptionBacklogResult.getFilterAcceptedEntries(), numEntries); + assertEquals(analyzeSubscriptionBacklogResult.getFilterRejectedEntries(), 0); + assertEquals(analyzeSubscriptionBacklogResult.getFilterRescheduledEntries(), 0); + assertEquals(analyzeSubscriptionBacklogResult.getFilterRescheduledEntries(), 0); - assertEquals(numMessages, analyzeSubscriptionBacklogResult.getMessages()); - assertEquals(numMessages, analyzeSubscriptionBacklogResult.getFilterAcceptedMessages()); - assertEquals(0, analyzeSubscriptionBacklogResult.getFilterRejectedMessages()); + assertEquals(analyzeSubscriptionBacklogResult.getMessages(), numMessages); + assertEquals(analyzeSubscriptionBacklogResult.getFilterAcceptedMessages(), numMessages); + assertEquals(analyzeSubscriptionBacklogResult.getFilterRejectedMessages(), 0); - assertEquals(0, analyzeSubscriptionBacklogResult.getFilterRescheduledMessages()); + assertEquals(analyzeSubscriptionBacklogResult.getFilterRescheduledMessages(), 0); assertFalse(analyzeSubscriptionBacklogResult.isAborted()); } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/BitSetRecyclable.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/BitSetRecyclable.java index 12ce7eb74c72b..b801d5f2b05a1 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/BitSetRecyclable.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/BitSetRecyclable.java @@ -216,6 +216,14 @@ public static BitSetRecyclable valueOf(byte[] bytes) { return BitSetRecyclable.valueOf(ByteBuffer.wrap(bytes)); } + /** + * Copy a BitSetRecyclable. + */ + public static BitSetRecyclable valueOf(BitSetRecyclable src) { + // The internal implementation will do the array-copy. + return valueOf(src.words); + } + /** * Returns a new bit set containing all the bits in the given byte * buffer between its position and limit. From a8d370f68593c0532b8cd02469e5196d3cf00432 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 19 Feb 2024 16:43:39 +0800 Subject: [PATCH 283/699] [fix] [broker] Fix can not subscribe partitioned topic with a suffix-matched regexp (#22025) (cherry picked from commit ad09f78f31b9dd640e5d602584ea4c6b857c27bb) --- .../broker/resources/TopicResources.java | 3 ++ .../broker/namespace/NamespaceService.java | 3 ++ .../service/PulsarCommandSenderImpl.java | 6 ++++ .../broker/service/TopicListService.java | 22 ++++++++++-- .../impl/PatternTopicsConsumerImplTest.java | 34 +++++++++++++++---- .../pulsar/client/api/ConsumerBuilder.java | 8 ++--- .../client/impl/MultiTopicsConsumerImpl.java | 10 ++++-- .../impl/PatternMultiTopicsConsumerImpl.java | 24 +++++++++++-- .../pulsar/client/impl/TopicListWatcher.java | 4 ++- .../impl/conf/ConsumerConfigurationData.java | 2 +- .../pulsar/common/protocol/Commands.java | 7 ++++ 11 files changed, 104 insertions(+), 19 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java index 840ced0a1c1c4..0963f25c3d31f 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/resources/TopicResources.java @@ -50,6 +50,9 @@ public TopicResources(MetadataStore store) { store.registerListener(this::handleNotification); } + /*** + * List persistent topics names under a namespace, the topic name contains the partition suffix. + */ public CompletableFuture> listPersistentTopicsAsync(NamespaceName ns) { String path = MANAGED_LEDGER_PATH + "/" + ns + "/persistent"; 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 8509de8f6b6f1..61e045ed304fd 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 @@ -1434,6 +1434,9 @@ private CompletableFuture> getPartitionsForTopic(TopicName topicNam }); } + /*** + * List persistent topics names under a namespace, the topic name contains the partition suffix. + */ public CompletableFuture> getListOfPersistentTopics(NamespaceName namespaceName) { return pulsar.getPulsarResources().getTopicResources().listPersistentTopicsAsync(namespaceName); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSenderImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSenderImpl.java index dd74fc4e71ed2..105650caaaf13 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSenderImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarCommandSenderImpl.java @@ -356,12 +356,18 @@ public void sendEndTxnErrorResponse(long requestId, TxnID txnID, ServerError err writeAndFlush(outBuf); } + /*** + * @param topics topic names which are matching, the topic name contains the partition suffix. + */ @Override public void sendWatchTopicListSuccess(long requestId, long watcherId, String topicsHash, List topics) { BaseCommand command = Commands.newWatchTopicListSuccess(requestId, watcherId, topicsHash, topics); interceptAndWriteCommand(command); } + /*** + * {@inheritDoc} + */ @Override public void sendWatchTopicListUpdate(long watcherId, List newTopics, List deletedTopics, String topicsHash) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java index 7aa50057d73c9..aea5b9fc65b46 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/TopicListService.java @@ -31,6 +31,7 @@ import org.apache.pulsar.common.api.proto.CommandWatchTopicListClose; import org.apache.pulsar.common.api.proto.ServerError; import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.topics.TopicList; import org.apache.pulsar.common.util.collections.ConcurrentLongHashMap; import org.apache.pulsar.metadata.api.NotificationType; @@ -42,11 +43,16 @@ public class TopicListService { public static class TopicListWatcher implements BiConsumer { + /** Topic names which are matching, the topic name contains the partition suffix. **/ private final List matchingTopics; private final TopicListService topicListService; private final long id; + /** The regexp for the topic name(not contains partition suffix). **/ private final Pattern topicsPattern; + /*** + * @param topicsPattern The regexp for the topic name(not contains partition suffix). + */ public TopicListWatcher(TopicListService topicListService, long id, Pattern topicsPattern, List topics) { this.topicListService = topicListService; @@ -59,9 +65,12 @@ public List getMatchingTopics() { return matchingTopics; } + /*** + * @param topicName topic name which contains partition suffix. + */ @Override public void accept(String topicName, NotificationType notificationType) { - if (topicsPattern.matcher(topicName).matches()) { + if (topicsPattern.matcher(TopicName.get(topicName).getPartitionedTopicName()).matches()) { List newTopics; List deletedTopics; if (notificationType == NotificationType.Deleted) { @@ -109,6 +118,9 @@ public void inactivate() { } } + /*** + * @param topicsPattern The regexp for the topic name(not contains partition suffix). + */ public void handleWatchTopicList(NamespaceName namespaceName, long watcherId, long requestId, Pattern topicsPattern, String topicsHash, Semaphore lookupSemaphore) { @@ -184,7 +196,9 @@ public void handleWatchTopicList(NamespaceName namespaceName, long watcherId, lo }); } - + /*** + * @param topicsPattern The regexp for the topic name(not contains partition suffix). + */ public void initializeTopicsListWatcher(CompletableFuture watcherFuture, NamespaceName namespace, long watcherId, Pattern topicsPattern) { namespaceService.getListOfPersistentTopics(namespace). @@ -246,6 +260,10 @@ public void deleteTopicListWatcher(Long watcherId) { log.info("[{}] Closed watcher, watcherId={}", connection.getRemoteAddress(), watcherId); } + /** + * @param deletedTopics topic names deleted(contains the partition suffix). + * @param newTopics topics names added(contains the partition suffix). + */ public void sendTopicListUpdate(long watcherId, String topicsHash, List deletedTopics, List newTopics) { connection.getCommandSender().sendWatchTopicListUpdate(watcherId, newTopics, deletedTopics, topicsHash); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java index 9bcbdfed4c9ee..7707abafde8de 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java @@ -681,16 +681,27 @@ public void testAutoSubscribePatterConsumerFromBrokerWatcher(boolean delayWatchi } } - @DataProvider(name= "partitioned") - public Object[][] partitioned(){ + @DataProvider(name= "regexpConsumerArgs") + public Object[][] regexpConsumerArgs(){ return new Object[][]{ - {true}, - {false} + {true, true}, + {true, false}, + {false, true}, + {false, false} }; } - @Test(timeOut = testTimeout, dataProvider = "partitioned") - public void testPreciseRegexpSubscribe(boolean partitioned) throws Exception { + private void waitForTopicListWatcherStarted(Consumer consumer) { + Awaitility.await().untilAsserted(() -> { + CompletableFuture completableFuture = WhiteboxImpl.getInternalState(consumer, "watcherFuture"); + log.info("isDone: {}, isCompletedExceptionally: {}", completableFuture.isDone(), + completableFuture.isCompletedExceptionally()); + assertTrue(completableFuture.isDone() && !completableFuture.isCompletedExceptionally()); + }); + } + + @Test(timeOut = testTimeout, dataProvider = "regexpConsumerArgs") + public void testPreciseRegexpSubscribe(boolean partitioned, boolean createTopicAfterWatcherStarted) throws Exception { final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); final String subscriptionName = "s1"; final Pattern pattern = Pattern.compile(String.format("%s$", topicName)); @@ -704,6 +715,9 @@ public void testPreciseRegexpSubscribe(boolean partitioned) throws Exception { .ackTimeout(ackTimeOutMillis, TimeUnit.MILLISECONDS) .receiverQueueSize(4) .subscribe(); + if (createTopicAfterWatcherStarted) { + waitForTopicListWatcherStarted(consumer); + } // 1. create topic. if (partitioned) { @@ -733,6 +747,14 @@ public void testPreciseRegexpSubscribe(boolean partitioned) throws Exception { } } + @DataProvider(name= "partitioned") + public Object[][] partitioned(){ + return new Object[][]{ + {true}, + {true} + }; + } + @Test(timeOut = 240 * 1000, dataProvider = "partitioned") public void testPreciseRegexpSubscribeDisabledTopicWatcher(boolean partitioned) throws Exception { final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); 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 870900a48feae..588a34d2069da 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 @@ -126,7 +126,7 @@ public interface ConsumerBuilder extends Cloneable { ConsumerBuilder topics(List topicNames); /** - * Specify a pattern for topics that this consumer subscribes to. + * Specify a pattern for topics(not contains the partition suffix) that this consumer subscribes to. * *

    The pattern is applied to subscribe to all topics, within a single namespace, that match the * pattern. @@ -134,13 +134,13 @@ public interface ConsumerBuilder extends Cloneable { *

    The consumer automatically subscribes to topics created after itself. * * @param topicsPattern - * a regular expression to select a list of topics to subscribe to + * a regular expression to select a list of topics(not contains the partition suffix) to subscribe to * @return the consumer builder instance */ ConsumerBuilder topicsPattern(Pattern topicsPattern); /** - * Specify a pattern for topics that this consumer subscribes to. + * Specify a pattern for topics(not contains the partition suffix) that this consumer subscribes to. * *

    It accepts a regular expression that is compiled into a pattern internally. E.g., * "persistent://public/default/pattern-topic-.*" @@ -151,7 +151,7 @@ public interface ConsumerBuilder extends Cloneable { *

    The consumer automatically subscribes to topics created after itself. * * @param topicsPattern - * given regular expression for topics pattern + * given regular expression for topics(not contains the partition suffix) pattern * @return the consumer builder instance */ ConsumerBuilder topicsPattern(String topicsPattern); 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 4240ad55b096b..0272d3d3baf34 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 @@ -929,7 +929,10 @@ private void removeTopic(String topic) { } } - // subscribe one more given topic + /*** + * Subscribe one more given topic. + * @param topicName topic name without the partition suffix. + */ public CompletableFuture subscribeAsync(String topicName, boolean createTopicIfDoesNotExist) { TopicName topicNameInstance = getTopicName(topicName); if (topicNameInstance == null) { @@ -1251,7 +1254,10 @@ public CompletableFuture unsubscribeAsync(String topicName) { return unsubscribeFuture; } - // Remove a consumer for a topic + /*** + * Remove a consumer for a topic. + * @param topicName topic name contains the partition suffix. + */ public CompletableFuture removeConsumerAsync(String topicName) { checkArgument(TopicName.isValid(topicName), "Invalid topic name:" + topicName); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java index f3ebcdee6c0d9..4d179f7d914c2 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java @@ -67,6 +67,9 @@ public class PatternMultiTopicsConsumerImpl extends MultiTopicsConsumerImpl onTopicsRemoved(Collection removedTopics); - // subscribe and create a list of new ConsumerImpl, added them to the `consumers` map in - // `MultiTopicsConsumerImpl`. + + /*** + * subscribe and create a list of new {@link ConsumerImpl}, added them to the + * {@link MultiTopicsConsumerImpl#consumers} map in {@link MultiTopicsConsumerImpl}. + * @param addedTopics topic names added(contains the partition suffix). + */ CompletableFuture onTopicsAdded(Collection addedTopics); } private class PatternTopicsChangedListener implements TopicsChangedListener { + + /** + * {@inheritDoc} + */ @Override public CompletableFuture onTopicsRemoved(Collection removedTopics) { CompletableFuture removeFuture = new CompletableFuture<>(); @@ -249,6 +264,9 @@ public CompletableFuture onTopicsRemoved(Collection removedTopics) return removeFuture; } + /** + * {@inheritDoc} + */ @Override public CompletableFuture onTopicsAdded(Collection addedTopics) { CompletableFuture addFuture = new CompletableFuture<>(); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java index 489a07a606eb2..86adf69f06e0f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java @@ -59,6 +59,9 @@ public class TopicListWatcher extends HandlerState implements ConnectionHandler. private final Runnable recheckTopicsChangeAfterReconnect; + /*** + * @param topicsPattern The regexp for the topic name(not contains partition suffix). + */ public TopicListWatcher(PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener, PulsarClientImpl client, Pattern topicsPattern, long watcherId, NamespaceName namespace, String topicsHash, @@ -142,7 +145,6 @@ public CompletableFuture connectionOpened(ClientCnx cnx) { return; } } - this.connectionHandler.resetBackoff(); recheckTopicsChangeAfterReconnect.run(); 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 8760926792cd7..3ae0e977d13c4 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 @@ -65,7 +65,7 @@ public class ConsumerConfigurationData implements Serializable, Cloneable { @ApiModelProperty( name = "topicsPattern", - value = "Topic pattern" + value = "The regexp for the topic name(not contains partition suffix)." ) private Pattern topicsPattern; 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 cf0cd820a6d10..3982900041813 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 @@ -1555,6 +1555,9 @@ public static BaseCommand newWatchTopicList( return cmd; } + /*** + * @param topics topic names which are matching, the topic name contains the partition suffix. + */ public static BaseCommand newWatchTopicListSuccess(long requestId, long watcherId, String topicsHash, List topics) { BaseCommand cmd = localCmd(Type.WATCH_TOPIC_LIST_SUCCESS); @@ -1570,6 +1573,10 @@ public static BaseCommand newWatchTopicListSuccess(long requestId, long watcherI return cmd; } + /** + * @param deletedTopics topic names deleted(contains the partition suffix). + * @param newTopics topics names added(contains the partition suffix). + */ public static BaseCommand newWatchTopicUpdate(long watcherId, List newTopics, List deletedTopics, String topicsHash) { BaseCommand cmd = localCmd(Type.WATCH_TOPIC_UPDATE); From a98adaacd57538dca0f61c918cf60d1b51266702 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 28 Sep 2023 00:19:20 +0800 Subject: [PATCH 284/699] [improve] [broker] Not close the socket if lookup failed caused by bundle unloading or metadata ex (#21211) ### Motivation **Background**: The Pulsar client will close the socket if it receives a ServiceNotReady error when doing a lookup. Closing the socket causes the other consumer or producer to reconnect and does not make the lookup more efficient. There are two cases that should be improved: - If the broker gets a metadata read/write error, the broker responds with a `ServiceNotReady` error, but it should respond with a `MetadataError` - If the topic is unloading, the broker responds with a `ServiceNotReady` error. ### Modifications - Respond to the client with a `MetadataError` if the broker gets a metadata read/write error. - Respond to the client with a `MetadataError` if the topic is unloading (cherry picked from commit 16349e613c52232a6361de2602e53b2a43038477) --- .../pulsar/broker/lookup/TopicLookupBase.java | 46 ++++---- .../broker/namespace/ServiceUnitUtils.java | 2 +- .../pulsar/broker/web/PulsarWebResource.java | 1 + .../LeaderElectionServiceTest.java | 6 +- .../client/api/BrokerServiceLookupTest.java | 103 ++++++++++++++++++ 5 files changed, 132 insertions(+), 26 deletions(-) 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 16d3f535780b1..c4a39cd0d4455 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 @@ -26,7 +26,6 @@ import java.net.URISyntaxException; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; import javax.ws.rs.Encoded; import javax.ws.rs.WebApplicationException; import javax.ws.rs.core.Response; @@ -48,6 +47,7 @@ import org.apache.pulsar.common.policies.data.TopicOperation; import org.apache.pulsar.common.util.Codec; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.metadata.api.MetadataStoreException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -319,35 +319,37 @@ public static CompletableFuture lookupTopicAsync(PulsarService pulsarSe requestId, shouldRedirectThroughServiceUrl(conf, lookupData))); } }).exceptionally(ex -> { - if (ex instanceof CompletionException && ex.getCause() instanceof IllegalStateException) { - log.info("Failed to lookup {} for topic {} with error {}", clientAppId, - topicName.toString(), ex.getCause().getMessage()); - } else { - log.warn("Failed to lookup {} for topic {} with error {}", clientAppId, - topicName.toString(), ex.getMessage(), ex); - } - lookupfuture.complete( - newLookupErrorResponse(ServerError.ServiceNotReady, ex.getMessage(), requestId)); - return null; - }); + handleLookupError(lookupfuture, topicName.toString(), clientAppId, requestId, ex); + return null; + }); } - }).exceptionally(ex -> { - if (ex instanceof CompletionException && ex.getCause() instanceof IllegalStateException) { - log.info("Failed to lookup {} for topic {} with error {}", clientAppId, topicName.toString(), - ex.getCause().getMessage()); - } else { - log.warn("Failed to lookup {} for topic {} with error {}", clientAppId, topicName.toString(), - ex.getMessage(), ex); - } - - lookupfuture.complete(newLookupErrorResponse(ServerError.ServiceNotReady, ex.getMessage(), requestId)); + handleLookupError(lookupfuture, topicName.toString(), clientAppId, requestId, ex); return null; }); return lookupfuture; } + private static void handleLookupError(CompletableFuture lookupFuture, String topicName, String clientAppId, + long requestId, Throwable ex){ + final Throwable unwrapEx = FutureUtil.unwrapCompletionException(ex); + final String errorMsg = unwrapEx.getMessage(); + if (unwrapEx instanceof IllegalStateException) { + // Current broker still hold the bundle's lock, but the bundle is being unloading. + log.info("Failed to lookup {} for topic {} with error {}", clientAppId, topicName, errorMsg); + lookupFuture.complete(newLookupErrorResponse(ServerError.MetadataError, errorMsg, requestId)); + } else if (unwrapEx instanceof MetadataStoreException){ + // Load bundle ownership or acquire lock failed. + // Differ with "IllegalStateException", print warning log. + log.warn("Failed to lookup {} for topic {} with error {}", clientAppId, topicName, errorMsg); + lookupFuture.complete(newLookupErrorResponse(ServerError.MetadataError, errorMsg, requestId)); + } else { + log.warn("Failed to lookup {} for topic {} with error {}", clientAppId, topicName, errorMsg); + lookupFuture.complete(newLookupErrorResponse(ServerError.ServiceNotReady, errorMsg, requestId)); + } + } + protected TopicName getTopicName(String topicDomain, String tenant, String cluster, String namespace, @Encoded String encodedTopic) { String decodedName = Codec.decode(encodedTopic); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/ServiceUnitUtils.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/ServiceUnitUtils.java index c86aac5316fb9..432aa29798ebd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/ServiceUnitUtils.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/ServiceUnitUtils.java @@ -36,7 +36,7 @@ public final class ServiceUnitUtils { */ private static final String OWNER_INFO_ROOT = "/namespace"; - static String path(NamespaceBundle suname) { + public static String path(NamespaceBundle suname) { // The ephemeral node path for new namespaces should always have bundle name appended return OWNER_INFO_ROOT + "/" + suname.toString(); } 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 50068e69217a2..cde555e24aaff 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 @@ -533,6 +533,7 @@ protected static CompletableFuture getClusterDataIfDifferentCluster pulsar.getPulsarResources().getClusterResources().getClusterAsync(cluster) .whenComplete((clusterDataResult, ex) -> { if (ex != null) { + log.warn("[{}] Load cluster data failed: requested={}", clientAppId, cluster); clusterDataFuture.completeExceptionally(FutureUtil.unwrapCompletionException(ex)); return; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LeaderElectionServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LeaderElectionServiceTest.java index 57d1da192d7e6..ded4ee8e58d53 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LeaderElectionServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LeaderElectionServiceTest.java @@ -130,10 +130,10 @@ private void checkLookupException(String tenant, String namespace, PulsarClient .topic("persistent://" + tenant + "/" + namespace + "/1p") .create(); } catch (PulsarClientException t) { - Assert.assertTrue(t instanceof PulsarClientException.LookupException); + Assert.assertTrue(t instanceof PulsarClientException.BrokerMetadataException + || t instanceof PulsarClientException.LookupException); Assert.assertTrue( - t.getMessage().contains( - "java.lang.IllegalStateException: The leader election has not yet been completed!")); + t.getMessage().contains("The leader election has not yet been completed")); } } 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 9c50c29f92495..dab4fe9087e79 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 @@ -27,6 +27,7 @@ import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import com.github.benmanes.caffeine.cache.AsyncLoadingCache; import com.google.common.collect.Sets; import com.google.common.util.concurrent.MoreExecutors; import io.netty.handler.codec.http.HttpRequest; @@ -71,9 +72,13 @@ import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerWrapper; import org.apache.pulsar.broker.loadbalance.impl.SimpleResourceUnit; import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.broker.namespace.OwnedBundle; +import org.apache.pulsar.broker.namespace.OwnershipCache; +import org.apache.pulsar.broker.namespace.ServiceUnitUtils; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.impl.BinaryProtoLookupService; +import org.apache.pulsar.client.impl.ClientCnx; import org.apache.pulsar.client.impl.LookupService; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.naming.NamespaceBundle; @@ -87,6 +92,7 @@ import org.apache.pulsar.common.util.SecurityUtility; import org.apache.pulsar.policies.data.loadbalancer.LocalBrokerData; import org.apache.pulsar.policies.data.loadbalancer.NamespaceBundleStats; +import org.apache.zookeeper.KeeperException; import org.asynchttpclient.AsyncCompletionHandler; import org.asynchttpclient.AsyncHttpClient; import org.asynchttpclient.AsyncHttpClientConfig; @@ -1105,4 +1111,101 @@ public String authenticate(AuthenticationDataSource authData) throws Authenticat return "invalid"; } } + + @Test + public void testLookupConnectionNotCloseIfGetUnloadingExOrMetadataEx() throws Exception { + String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(tpName); + PulsarClientImpl pulsarClientImpl = (PulsarClientImpl) pulsarClient; + Producer producer = pulsarClientImpl.newProducer(Schema.STRING).topic(tpName).create(); + Consumer consumer = pulsarClientImpl.newConsumer(Schema.STRING).topic(tpName) + .subscriptionName("s1").isAckReceiptEnabled(true).subscribe(); + LookupService lookupService = pulsarClientImpl.getLookup(); + assertTrue(lookupService instanceof BinaryProtoLookupService); + ClientCnx lookupConnection = pulsarClientImpl.getCnxPool().getConnection(lookupService.resolveHost()).join(); + + // Verify the socket will not be closed if the bundle is unloading. + BundleOfTopic bundleOfTopic = new BundleOfTopic(tpName); + bundleOfTopic.setBundleIsUnloading(); + try { + lookupService.getBroker(TopicName.get(tpName)).get(); + fail("It should failed due to the namespace bundle is unloading."); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("is being unloaded")); + } + // Do unload topic, trigger producer & consumer reconnection. + pulsar.getBrokerService().getTopic(tpName, false).join().get().close(true); + assertTrue(lookupConnection.ctx().channel().isActive()); + bundleOfTopic.setBundleIsNotUnloading(); + // Assert producer & consumer could reconnect successful. + producer.send("1"); + HashSet messagesReceived = new HashSet<>(); + while (true) { + Message msg = consumer.receive(2, TimeUnit.SECONDS); + if (msg == null) { + break; + } + messagesReceived.add(msg.getValue()); + } + assertTrue(messagesReceived.contains("1")); + + // Verify the socket will not be closed if get a metadata ex. + bundleOfTopic.releaseBundleLockAndMakeAcquireFail(); + try { + lookupService.getBroker(TopicName.get(tpName)).get(); + fail("It should failed due to the acquire bundle lock fail."); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("OperationTimeout")); + } + // Do unload topic, trigger producer & consumer reconnection. + pulsar.getBrokerService().getTopic(tpName, false).join().get().close(true); + assertTrue(lookupConnection.ctx().channel().isActive()); + bundleOfTopic.makeAcquireBundleLockSuccess(); + // Assert producer could reconnect successful. + producer.send("2"); + while (true) { + Message msg = consumer.receive(2, TimeUnit.SECONDS); + if (msg == null) { + break; + } + messagesReceived.add(msg.getValue()); + } + assertTrue(messagesReceived.contains("2")); + + // cleanup. + producer.close(); + consumer.close(); + admin.topics().delete(tpName); + } + + private class BundleOfTopic { + + private NamespaceBundle namespaceBundle; + private OwnershipCache ownershipCache; + private AsyncLoadingCache ownedBundlesCache; + + public BundleOfTopic(String tpName) { + namespaceBundle = pulsar.getNamespaceService().getBundle(TopicName.get(tpName)); + ownershipCache = pulsar.getNamespaceService().getOwnershipCache(); + ownedBundlesCache = WhiteboxImpl.getInternalState(ownershipCache, "ownedBundlesCache"); + } + + private void setBundleIsUnloading() { + ownedBundlesCache.get(namespaceBundle).join().setActive(false); + } + + private void setBundleIsNotUnloading() { + ownedBundlesCache.get(namespaceBundle).join().setActive(true); + } + + private void releaseBundleLockAndMakeAcquireFail() throws Exception { + ownedBundlesCache.synchronous().invalidateAll(); + mockZooKeeper.delete(ServiceUnitUtils.path(namespaceBundle), -1); + mockZooKeeper.setAlwaysFail(KeeperException.Code.OPERATIONTIMEOUT); + } + + private void makeAcquireBundleLockSuccess() throws Exception { + mockZooKeeper.unsetAlwaysFail(); + } + } } From dfc75f5944c4b2e06267dbfc8f434fbce0d6f90a Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 2 Feb 2024 10:12:19 +0800 Subject: [PATCH 285/699] [improve][broker] Do not close the socket if lookup failed due to LockBusyException (#21993) (cherry picked from commit bf5639f06be200c9ccf9644a45f75fc95797d335) --- .../pulsar/broker/lookup/TopicLookupBase.java | 8 +++- .../client/api/BrokerServiceLookupTest.java | 39 +++++++++++++++++++ 2 files changed, 45 insertions(+), 2 deletions(-) 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 c4a39cd0d4455..7b2c777414884 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 @@ -30,6 +30,7 @@ import javax.ws.rs.WebApplicationException; import javax.ws.rs.core.Response; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; @@ -333,13 +334,16 @@ public static CompletableFuture lookupTopicAsync(PulsarService pulsarSe private static void handleLookupError(CompletableFuture lookupFuture, String topicName, String clientAppId, long requestId, Throwable ex){ - final Throwable unwrapEx = FutureUtil.unwrapCompletionException(ex); + Throwable unwrapEx = FutureUtil.unwrapCompletionException(ex); final String errorMsg = unwrapEx.getMessage(); + if (unwrapEx instanceof PulsarServerException) { + unwrapEx = FutureUtil.unwrapCompletionException(unwrapEx.getCause()); + } if (unwrapEx instanceof IllegalStateException) { // Current broker still hold the bundle's lock, but the bundle is being unloading. log.info("Failed to lookup {} for topic {} with error {}", clientAppId, topicName, errorMsg); lookupFuture.complete(newLookupErrorResponse(ServerError.MetadataError, errorMsg, requestId)); - } else if (unwrapEx instanceof MetadataStoreException){ + } else if (unwrapEx instanceof MetadataStoreException) { // Load bundle ownership or acquire lock failed. // Differ with "IllegalStateException", print warning log. log.warn("Failed to lookup {} for topic {} with error {}", clientAppId, topicName, errorMsg); 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 dab4fe9087e79..0a4c5b7a318b3 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 @@ -71,6 +71,7 @@ import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl; import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerWrapper; import org.apache.pulsar.broker.loadbalance.impl.SimpleResourceUnit; +import org.apache.pulsar.broker.namespace.NamespaceEphemeralData; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.namespace.OwnedBundle; import org.apache.pulsar.broker.namespace.OwnershipCache; @@ -1208,4 +1209,42 @@ private void makeAcquireBundleLockSuccess() throws Exception { mockZooKeeper.unsetAlwaysFail(); } } + + @Test(timeOut = 30000) + public void testLookupConnectionNotCloseIfFailedToAcquireOwnershipOfBundle() throws Exception { + String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(tpName); + final var pulsarClientImpl = (PulsarClientImpl) pulsarClient; + final var cache = pulsar.getNamespaceService().getOwnershipCache(); + final var bundle = pulsar.getNamespaceService().getBundle(TopicName.get(tpName)); + final var value = cache.getOwnerAsync(bundle).get().orElse(null); + assertNotNull(value); + + cache.invalidateLocalOwnerCache(); + final var lock = pulsar.getCoordinationService().getLockManager(NamespaceEphemeralData.class) + .acquireLock(ServiceUnitUtils.path(bundle), new NamespaceEphemeralData()).join(); + lock.updateValue(null); + log.info("Updated bundle {} with null", bundle.getBundleRange()); + + // wait for the system topic reader to __change_events is closed, otherwise the test will be affected + Thread.sleep(500); + + final var future = pulsarClientImpl.getLookup().getBroker(TopicName.get(tpName)); + final var cnx = pulsarClientImpl.getCnxPool().getConnections().stream().findAny() + .map(CompletableFuture::join).orElse(null); + assertNotNull(cnx); + + try { + future.get(); + fail(); + } catch (ExecutionException e) { + log.info("getBroker failed with {}: {}", e.getCause().getClass().getName(), e.getMessage()); + assertTrue(e.getCause() instanceof PulsarClientException.BrokerMetadataException); + assertTrue(cnx.ctx().channel().isActive()); + lock.updateValue(value); + lock.release(); + assertTrue(e.getMessage().contains("Failed to acquire ownership")); + pulsarClientImpl.getLookup().getBroker(TopicName.get(tpName)).get(); + } + } } From 84eeddaf74c35b5cd2528da83294a12191f1a09a Mon Sep 17 00:00:00 2001 From: thetumbled <52550727+thetumbled@users.noreply.github.com> Date: Thu, 22 Feb 2024 12:07:43 +0800 Subject: [PATCH 286/699] [fix] [txn] Get previous position by managed ledger. (#22024) (cherry picked from commit 98ce27e6aad4c3c2f3f015248b7e9b6de87829dd) --- .../buffer/impl/TopicTransactionBuffer.java | 4 +- .../buffer/TopicTransactionBufferTest.java | 58 +++++++++++++++++++ 2 files changed, 60 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java index f356921d6988e..5392e473947e6 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 @@ -287,8 +287,8 @@ private void handleTransactionMessage(TxnID txnId, Position position) { .checkAbortedTransaction(txnId)) { ongoingTxns.put(txnId, (PositionImpl) position); PositionImpl firstPosition = ongoingTxns.get(ongoingTxns.firstKey()); - //max read position is less than first ongoing transaction message position, so entryId -1 - maxReadPosition = PositionImpl.get(firstPosition.getLedgerId(), firstPosition.getEntryId() - 1); + // max read position is less than first ongoing transaction message position + maxReadPosition = ((ManagedLedgerImpl) topic.getManagedLedger()).getPreviousPosition(firstPosition); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java index 6ab56a613c500..ecba4a9f1d54a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TopicTransactionBufferTest.java @@ -40,8 +40,10 @@ import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBuffer; import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBufferState; 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.Reader; import org.apache.pulsar.client.api.TopicMessageId; import org.apache.pulsar.client.api.transaction.Transaction; import org.apache.pulsar.client.impl.MessageIdImpl; @@ -304,6 +306,62 @@ public void testGetLastMessageIdsWithOngoingTransactions() throws Exception { assertMessageId(consumer, expectedLastMessageID2, 2); } + /** + * produce 3 messages and then trigger a ledger switch, + * then create a transaction and send a transactional message. + * As there are messages in the new ledger, the reader should be able to read the messages. + * But reader.hasMessageAvailable() returns false if the entry id of max read position is -1. + * @throws Exception + */ + @Test + public void testGetLastMessageIdsWithOpenTransactionAtLedgerHead() throws Exception { + String topic = "persistent://" + NAMESPACE1 + "/testGetLastMessageIdsWithOpenTransactionAtLedgerHead"; + String subName = "my-subscription"; + @Cleanup + Producer producer = pulsarClient.newProducer() + .topic(topic) + .create(); + Consumer consumer = pulsarClient.newConsumer() + .topic(topic) + .subscriptionName(subName) + .subscribe(); + MessageId expectedLastMessageID = null; + for (int i = 0; i < 3; i++) { + expectedLastMessageID = producer.newMessage().value(String.valueOf(i).getBytes()).send(); + System.out.println("expectedLastMessageID: " + expectedLastMessageID); + } + triggerLedgerSwitch(topic); + Transaction txn = pulsarClient.newTransaction() + .withTransactionTimeout(5, TimeUnit.HOURS) + .build() + .get(); + producer.newMessage(txn).send(); + + Reader reader = pulsarClient.newReader() + .topic(topic) + .startMessageId(MessageId.earliest) + .create(); + assertTrue(reader.hasMessageAvailable()); + } + + private void triggerLedgerSwitch(String topicName) throws Exception{ + admin.topics().unload(topicName); + Awaitility.await().until(() -> { + CompletableFuture> topicFuture = + getPulsarServiceList().get(0).getBrokerService().getTopic(topicName, false); + if (!topicFuture.isDone() || topicFuture.isCompletedExceptionally()){ + return false; + } + Optional topicOptional = topicFuture.join(); + if (!topicOptional.isPresent()){ + return false; + } + PersistentTopic persistentTopic = (PersistentTopic) topicOptional.get(); + ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + return managedLedger.getState() == ManagedLedgerImpl.State.LedgerOpened; + }); + } + private void assertMessageId(Consumer consumer, MessageIdImpl expected, int entryOffset) throws Exception { TopicMessageIdImpl actual = (TopicMessageIdImpl) consumer.getLastMessageIds().get(0); assertEquals(expected.getEntryId(), actual.getEntryId() - entryOffset); From a475bca6f8d837f85cd1f0b0de3ac31f46431ead Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Mon, 26 Feb 2024 23:40:02 +0800 Subject: [PATCH 287/699] [fix][sec] Add a check for the input time value (#22023) (cherry picked from commit 60fed0db44a0690269d8a576e269171865a5fa51) --- .../org/apache/bookkeeper/mledger/ManagedLedgerConfig.java | 5 ++++- .../mledger/impl/ManagedLedgerFactoryMBeanImpl.java | 2 ++ .../bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java | 2 ++ .../java/org/apache/pulsar/broker/stats/DimensionStats.java | 2 ++ .../broker/stats/prometheus/metrics/LongAdderCounter.java | 2 ++ .../apache/pulsar/compaction/CompactionRetentionTest.java | 4 +++- .../org/apache/pulsar/client/api/ClientConfiguration.java | 1 + .../org/apache/pulsar/client/api/ConsumerConfiguration.java | 1 + .../pulsar/client/admin/internal/PulsarAdminBuilderImpl.java | 4 ++++ .../pulsar/client/admin/internal/TransactionsImpl.java | 1 + .../org/apache/pulsar/client/impl/AutoClusterFailover.java | 1 + .../org/apache/pulsar/client/impl/ClientBuilderImpl.java | 2 ++ .../java/org/apache/pulsar/client/impl/ConsumerBase.java | 4 ++++ .../org/apache/pulsar/client/impl/ConsumerBuilderImpl.java | 1 + .../apache/pulsar/client/impl/ControlledClusterFailover.java | 1 + .../org/apache/pulsar/client/impl/ReaderBuilderImpl.java | 1 + .../pulsar/client/impl/TransactionMetaStoreHandler.java | 2 ++ .../apache/pulsar/client/impl/TypedMessageBuilderImpl.java | 1 + .../client/impl/transaction/TransactionBuilderImpl.java | 2 ++ .../main/java/org/apache/pulsar/client/util/ObjectCache.java | 2 ++ .../org/apache/bookkeeper/client/PulsarMockBookKeeper.java | 2 ++ .../java/org/apache/bookkeeper/client/TestStatsProvider.java | 2 ++ 22 files changed, 43 insertions(+), 2 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 f16866ac7923a..70890b6d73761 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 @@ -171,6 +171,7 @@ public int getMinimumRolloverTimeMs() { * the time unit */ public void setMinimumRolloverTime(int minimumRolloverTime, TimeUnit unit) { + checkArgument(minimumRolloverTime >= 0); this.minimumRolloverTimeMs = (int) unit.toMillis(minimumRolloverTime); checkArgument(maximumRolloverTimeMs >= minimumRolloverTimeMs, "Minimum rollover time needs to be less than maximum rollover time"); @@ -196,6 +197,7 @@ public long getMaximumRolloverTimeMs() { * the time unit */ public void setMaximumRolloverTime(int maximumRolloverTime, TimeUnit unit) { + checkArgument(maximumRolloverTime >= 0); this.maximumRolloverTimeMs = unit.toMillis(maximumRolloverTime); checkArgument(maximumRolloverTimeMs >= minimumRolloverTimeMs, "Maximum rollover time needs to be greater than minimum rollover time"); @@ -412,7 +414,8 @@ public ManagedLedgerConfig setThrottleMarkDelete(double throttleMarkDelete) { * time unit for retention time */ public ManagedLedgerConfig setRetentionTime(int retentionTime, TimeUnit unit) { - this.retentionTimeMs = unit.toMillis(retentionTime); + checkArgument(retentionTime >= -1, "The retention time should be -1, 0 or value > 0"); + this.retentionTimeMs = retentionTime != -1 ? unit.toMillis(retentionTime) : -1; return this; } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryMBeanImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryMBeanImpl.java index cf3d7142d617e..5a6bc8017b7e0 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryMBeanImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryMBeanImpl.java @@ -18,6 +18,7 @@ */ package org.apache.bookkeeper.mledger.impl; +import static com.google.common.base.Preconditions.checkArgument; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.LongAdder; import org.apache.bookkeeper.mledger.ManagedLedgerFactoryMXBean; @@ -41,6 +42,7 @@ public ManagedLedgerFactoryMBeanImpl(ManagedLedgerFactoryImpl factory) throws Ex } public void refreshStats(long period, TimeUnit unit) { + checkArgument(period >= 0); double seconds = unit.toMillis(period) / 1000.0; if (seconds <= 0.0) { diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java index e057dee99538e..7884add95526c 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerMBeanImpl.java @@ -18,6 +18,7 @@ */ package org.apache.bookkeeper.mledger.impl; +import static com.google.common.base.Preconditions.checkArgument; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.LongAdder; import org.apache.bookkeeper.mledger.ManagedCursor; @@ -63,6 +64,7 @@ public ManagedLedgerMBeanImpl(ManagedLedgerImpl managedLedger) { } public void refreshStats(long period, TimeUnit unit) { + checkArgument(period >= 0); double seconds = unit.toMillis(period) / 1000.0; if (seconds <= 0.0) { // skip refreshing stats diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/DimensionStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/DimensionStats.java index 1b6f981ca4e21..54965e4c783d8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/DimensionStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/DimensionStats.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.stats; +import static com.google.common.base.Preconditions.checkArgument; import static io.prometheus.client.CollectorRegistry.defaultRegistry; import io.prometheus.client.Collector; import io.prometheus.client.Summary; @@ -70,6 +71,7 @@ public DimensionStats(String name, long updateDurationInSec) { } public void recordDimensionTimeValue(long latency, TimeUnit unit) { + checkArgument(latency >= 0); summary.observe(unit.toMillis(latency)); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/LongAdderCounter.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/LongAdderCounter.java index 8ade2bc883f9a..c2816f5a2a013 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/LongAdderCounter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/LongAdderCounter.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.stats.prometheus.metrics; +import static com.google.common.base.Preconditions.checkArgument; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.LongAdder; import org.apache.bookkeeper.stats.Counter; @@ -57,6 +58,7 @@ public void addCount(long delta) { @Override public void addLatency(long eventLatency, TimeUnit unit) { + checkArgument(eventLatency >= 0); long valueMillis = unit.toMillis(eventLatency); counter.add(valueMillis); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionRetentionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionRetentionTest.java index 98bf2b819c2ba..88d923f74e196 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionRetentionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionRetentionTest.java @@ -257,7 +257,9 @@ private void checkTopicRetentionPolicy(String topicName, RetentionPolicies reten ManagedLedgerConfig config = pulsar.getBrokerService() .getManagedLedgerConfig(TopicName.get(topicName)).get(); Assert.assertEquals(config.getRetentionSizeInMB(), retentionPolicies.getRetentionSizeInMB()); - Assert.assertEquals(config.getRetentionTimeMillis(),retentionPolicies.getRetentionTimeInMinutes() * 60000L); + Assert.assertEquals(config.getRetentionTimeMillis(), retentionPolicies.getRetentionTimeInMinutes() < 0 + ? retentionPolicies.getRetentionTimeInMinutes() + : retentionPolicies.getRetentionTimeInMinutes() * 60000L); } private void testCompactionCursorRetention(String topic) throws Exception { diff --git a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ClientConfiguration.java b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ClientConfiguration.java index 3b0efe64cf588..ea2bba166e6c5 100644 --- a/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ClientConfiguration.java +++ b/pulsar-client-1x-base/pulsar-client-1x/src/main/java/org/apache/pulsar/client/api/ClientConfiguration.java @@ -368,6 +368,7 @@ public ClientConfiguration setServiceUrl(String serviceUrl) { * @param unit the time unit in which the duration is defined */ public void setConnectionTimeout(int duration, TimeUnit unit) { + checkArgument(duration >= 0); confData.setConnectionTimeoutMs((int) unit.toMillis(duration)); } 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 index 81956db56f774..f2101b287043c 100644 --- 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 @@ -69,6 +69,7 @@ public long getAckTimeoutMillis() { * @return {@link ConsumerConfiguration} */ public ConsumerConfiguration setAckTimeout(long ackTimeout, TimeUnit timeUnit) { + checkArgument(ackTimeout >= 0); long ackTimeoutMillis = timeUnit.toMillis(ackTimeout); checkArgument(ackTimeoutMillis >= minAckTimeoutMillis, "Ack timeout should be should be greater than " + minAckTimeoutMillis + " ms"); diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java index 009fa67fbaa29..a9d913c016490 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.admin.internal; +import static com.google.common.base.Preconditions.checkArgument; import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -200,18 +201,21 @@ public PulsarAdminBuilder tlsProtocols(Set tlsProtocols) { @Override public PulsarAdminBuilder connectionTimeout(int connectionTimeout, TimeUnit connectionTimeoutUnit) { + checkArgument(connectionTimeout >= 0); this.conf.setConnectionTimeoutMs((int) connectionTimeoutUnit.toMillis(connectionTimeout)); return this; } @Override public PulsarAdminBuilder readTimeout(int readTimeout, TimeUnit readTimeoutUnit) { + checkArgument(readTimeout >= 0); this.conf.setReadTimeoutMs((int) readTimeoutUnit.toMillis(readTimeout)); return this; } @Override public PulsarAdminBuilder requestTimeout(int requestTimeout, TimeUnit requestTimeoutUnit) { + checkArgument(requestTimeout >= 0); this.conf.setRequestTimeoutMs((int) requestTimeoutUnit.toMillis(requestTimeout)); return this; } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TransactionsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TransactionsImpl.java index 2d1dd408ef6c9..046ef3eb370e8 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TransactionsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TransactionsImpl.java @@ -168,6 +168,7 @@ public TransactionPendingAckStats getPendingAckStats(String topic, String subNam @Override public CompletableFuture> getSlowTransactionsByCoordinatorIdAsync( Integer coordinatorId, long timeout, TimeUnit timeUnit) { + checkArgument(timeout >= 0); WebTarget path = adminV3Transactions.path("slowTransactions"); path = path.path(timeUnit.toMillis(timeout) + ""); if (coordinatorId != null) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java index 68b781e67d29c..a1017e66760a5 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java @@ -329,6 +329,7 @@ public AutoClusterFailoverBuilder switchBackDelay(long switchBackDelay, TimeUnit @Override public AutoClusterFailoverBuilder checkInterval(long interval, TimeUnit timeUnit) { + checkArgument(interval >= 0L, "check interval time must not be negative."); this.checkIntervalMs = timeUnit.toMillis(interval); return this; } 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 7677045f0899b..107f15905044c 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 @@ -166,6 +166,7 @@ public ClientBuilder operationTimeout(int operationTimeout, TimeUnit unit) { @Override public ClientBuilder lookupTimeout(int lookupTimeout, TimeUnit unit) { + checkArgument(lookupTimeout >= 0, "lookupTimeout must not be negative"); conf.setLookupTimeoutMs(unit.toMillis(lookupTimeout)); return this; } @@ -331,6 +332,7 @@ public ClientBuilder keepAliveInterval(int keepAliveInterval, TimeUnit unit) { @Override public ClientBuilder connectionTimeout(int duration, TimeUnit unit) { + checkArgument(duration >= 0, "connectionTimeout needs to be >= 0"); conf.setConnectionTimeoutMs((int) unit.toMillis(duration)); return this; } 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 67bddf525c6b4..79cd8db4b99bb 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 @@ -449,6 +449,7 @@ public void acknowledge(Messages messages) throws PulsarClientException { @Override public void reconsumeLater(Message message, long delayTime, TimeUnit unit) throws PulsarClientException { + checkArgument(delayTime >= 0, "The delay time must not be negative."); reconsumeLater(message, null, delayTime, unit); } @@ -563,6 +564,7 @@ public CompletableFuture reconsumeLaterAsync(Message message, long dela @Override public CompletableFuture reconsumeLaterAsync( Message message, Map customProperties, long delayTime, TimeUnit unit) { + checkArgument(delayTime >= 0, "The delay time must not be negative."); if (!conf.isRetryEnable()) { return FutureUtil.failedFuture(new PulsarClientException(RECONSUME_LATER_ERROR_MSG)); } @@ -599,12 +601,14 @@ public CompletableFuture acknowledgeCumulativeAsync(Message message) { @Override public CompletableFuture reconsumeLaterCumulativeAsync(Message message, long delayTime, TimeUnit unit) { + checkArgument(delayTime >= 0, "The delay time must not be negative."); return reconsumeLaterCumulativeAsync(message, null, delayTime, unit); } @Override public CompletableFuture reconsumeLaterCumulativeAsync( Message message, Map customProperties, long delayTime, TimeUnit unit) { + checkArgument(delayTime >= 0, "The delay time must not be negative."); if (!conf.isRetryEnable()) { return FutureUtil.failedFuture(new PulsarClientException(RECONSUME_LATER_ERROR_MSG)); } 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 f644c6a18398f..895273a90c050 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 @@ -497,6 +497,7 @@ public ConsumerBuilder enableBatchIndexAcknowledgment(boolean batchIndexAckno @Override public ConsumerBuilder expireTimeOfIncompleteChunkedMessage(long duration, TimeUnit unit) { + checkArgument(duration >= 0, "expired time of incomplete chunk message must not be negative"); conf.setExpireTimeOfIncompleteChunkedMessageMillis(unit.toMillis(duration)); return this; } 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..9d30108ec7a1d 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 @@ -236,6 +236,7 @@ public ControlledClusterFailoverBuilder urlProviderHeader(Map he @Override public ControlledClusterFailoverBuilder checkInterval(long interval, @NonNull TimeUnit timeUnit) { + checkArgument(interval >= 0, "The check interval time must not be negative."); this.interval = timeUnit.toMillis(interval); return this; } 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 2860cda0ceef1..fd01cef9a216f 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 @@ -274,6 +274,7 @@ public ReaderBuilder autoAckOldestChunkedMessageOnQueueFull(boolean autoAckOl @Override public ReaderBuilder expireTimeOfIncompleteChunkedMessage(long duration, TimeUnit unit) { + checkArgument(duration >= 0, "The expired time must not be negative."); conf.setExpireTimeOfIncompleteChunkedMessageMillis(unit.toMillis(duration)); return this; } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java index 0b5174a015118..4ea6472b9d8b2 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TransactionMetaStoreHandler.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.impl; +import static com.google.common.base.Preconditions.checkArgument; import com.google.common.annotations.VisibleForTesting; import io.netty.buffer.ByteBuf; import io.netty.util.Recycler; @@ -205,6 +206,7 @@ private void failPendingRequest() { } public CompletableFuture newTransactionAsync(long timeout, TimeUnit unit) { + checkArgument(timeout >= 0, "The timeout must not be negative."); if (LOG.isDebugEnabled()) { LOG.debug("New transaction with timeout in ms {}", unit.toMillis(timeout)); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java index 026f8a1e69e0b..895949fdf32cc 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java @@ -212,6 +212,7 @@ public TypedMessageBuilder disableReplication() { @Override public TypedMessageBuilder deliverAfter(long delay, TimeUnit unit) { + checkArgument(delay >= 0, "The delay time must not be negative."); return deliverAt(System.currentTimeMillis() + unit.toMillis(delay)); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/transaction/TransactionBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/transaction/TransactionBuilderImpl.java index c5e9d4781c56f..255dedcbb85e9 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/transaction/TransactionBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/transaction/TransactionBuilderImpl.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.impl.transaction; +import static com.google.common.base.Preconditions.checkArgument; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; @@ -45,6 +46,7 @@ public TransactionBuilderImpl(PulsarClientImpl client, TransactionCoordinatorCli @Override public TransactionBuilder withTransactionTimeout(long txnTimeout, TimeUnit timeoutUnit) { + checkArgument(txnTimeout >= 0, "The txn timeout must not be negative."); this.txnTimeout = txnTimeout; this.timeUnit = timeoutUnit; return this; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/util/ObjectCache.java b/pulsar-client/src/main/java/org/apache/pulsar/client/util/ObjectCache.java index dc057ffe32daf..cf0620edf98df 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/util/ObjectCache.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/util/ObjectCache.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.util; +import static com.google.common.base.Preconditions.checkArgument; import java.time.Clock; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; @@ -33,6 +34,7 @@ public class ObjectCache implements Supplier { public ObjectCache(Supplier supplier, long cacheDuration, TimeUnit unit) { this(supplier, cacheDuration, unit, Clock.systemUTC()); + checkArgument(cacheDuration >= 0, "The cache duration must not be negative."); } ObjectCache(Supplier supplier, long cacheDuration, TimeUnit unit, Clock clock) { 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 f0d279ef25050..998ef73fbd3e9 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java @@ -18,6 +18,7 @@ */ package org.apache.bookkeeper.client; +import static com.google.common.base.Preconditions.checkArgument; import com.google.common.collect.Lists; import java.util.ArrayList; import java.util.Arrays; @@ -364,6 +365,7 @@ public synchronized CompletableFuture promiseAfter(int steps, List= 0, "The delay time must not be negative."); addEntryDelaysMillis.add(unit.toMillis(delay)); } diff --git a/testmocks/src/main/java/org/apache/bookkeeper/client/TestStatsProvider.java b/testmocks/src/main/java/org/apache/bookkeeper/client/TestStatsProvider.java index 4d08a7f80df5b..cf08cc635106e 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/TestStatsProvider.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/TestStatsProvider.java @@ -18,6 +18,7 @@ */ package org.apache.bookkeeper.client; +import static com.google.common.base.Preconditions.checkArgument; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; @@ -65,6 +66,7 @@ public void addCount(long delta) { @Override public void addLatency(long eventLatency, TimeUnit unit) { + checkArgument(eventLatency >= 0, "The event latency must not be negative."); long valueMillis = unit.toMillis(eventLatency); updateMax(val.addAndGet(valueMillis)); } From e881b76053fa85055b76504c541d0527b3991d9a Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 23 Feb 2024 23:13:08 +0800 Subject: [PATCH 288/699] [improve] [broker] Do not try to open ML when the topic meta does not exist and do not expect to create a new one. #21995 (#22004) Co-authored-by: Jiwe Guo (cherry picked from commit d18831f77c8969604b4fd02394a5f07c6e408cd6) --- .../pulsar/broker/service/BrokerService.java | 80 ++++++++++--------- .../broker/TopicEventsListenerTest.java | 33 ++++---- .../pulsar/broker/admin/AdminApi2Test.java | 28 +++++++ .../broker/admin/TopicAutoCreationTest.java | 14 ++-- .../persistent/PersistentTopicTest.java | 3 +- 5 files changed, 99 insertions(+), 59 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 76cd460d3d8f6..a89acc9768f1f 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 @@ -1051,43 +1051,49 @@ public CompletableFuture> getTopic(final TopicName topicName, bo } final boolean isPersistentTopic = topicName.getDomain().equals(TopicDomain.persistent); if (isPersistentTopic) { - final CompletableFuture> topicPoliciesFuture = - getTopicPoliciesBypassSystemTopic(topicName); - return topicPoliciesFuture.exceptionally(ex -> { - final Throwable rc = FutureUtil.unwrapCompletionException(ex); - final String errorInfo = String.format("Topic creation encountered an exception by initialize" - + " topic policies service. topic_name=%s error_message=%s", topicName, rc.getMessage()); - log.error(errorInfo, rc); - throw FutureUtil.wrapToCompletionException(new ServiceUnitNotReadyException(errorInfo)); - }).thenCompose(optionalTopicPolicies -> { - final TopicPolicies topicPolicies = optionalTopicPolicies.orElse(null); - return topics.computeIfAbsent(topicName.toString(), (tpName) -> { - if (topicName.isPartitioned()) { - final TopicName topicNameEntity = TopicName.get(topicName.getPartitionedTopicName()); - return fetchPartitionedTopicMetadataAsync(topicNameEntity) - .thenCompose((metadata) -> { - // Allow crate non-partitioned persistent topic that name includes `partition` - if (metadata.partitions == 0 - || topicName.getPartitionIndex() < metadata.partitions) { - return loadOrCreatePersistentTopic(tpName, createIfMissing, - properties, topicPolicies); - } - final String errorMsg = - String.format("Illegal topic partition name %s with max allowed " - + "%d partitions", topicName, metadata.partitions); - log.warn(errorMsg); - return FutureUtil - .failedFuture(new BrokerServiceException.NotAllowedException(errorMsg)); - }); - } - return loadOrCreatePersistentTopic(tpName, createIfMissing, properties, topicPolicies); - }).thenCompose(optionalTopic -> { - if (!optionalTopic.isPresent() && createIfMissing) { - log.warn("[{}] Try to recreate the topic with createIfMissing=true " - + "but the returned topic is empty", topicName); - return getTopic(topicName, createIfMissing, properties); - } - return CompletableFuture.completedFuture(optionalTopic); + return pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topicName) + .thenCompose(exists -> { + if (!exists && !createIfMissing) { + return CompletableFuture.completedFuture(Optional.empty()); + } + return getTopicPoliciesBypassSystemTopic(topicName).exceptionally(ex -> { + final Throwable rc = FutureUtil.unwrapCompletionException(ex); + final String errorInfo = String.format("Topic creation encountered an exception by initialize" + + " topic policies service. topic_name=%s error_message=%s", topicName, + rc.getMessage()); + log.error(errorInfo, rc); + throw FutureUtil.wrapToCompletionException(new ServiceUnitNotReadyException(errorInfo)); + }).thenCompose(optionalTopicPolicies -> { + final TopicPolicies topicPolicies = optionalTopicPolicies.orElse(null); + return topics.computeIfAbsent(topicName.toString(), (tpName) -> { + if (topicName.isPartitioned()) { + final TopicName topicNameEntity = TopicName.get(topicName.getPartitionedTopicName()); + return fetchPartitionedTopicMetadataAsync(topicNameEntity) + .thenCompose((metadata) -> { + // Allow crate non-partitioned persistent topic that name includes + // `partition` + if (metadata.partitions == 0 + || topicName.getPartitionIndex() < metadata.partitions) { + return loadOrCreatePersistentTopic(tpName, createIfMissing, + properties, topicPolicies); + } + final String errorMsg = + String.format("Illegal topic partition name %s with max allowed " + + "%d partitions", topicName, metadata.partitions); + log.warn(errorMsg); + return FutureUtil.failedFuture( + new BrokerServiceException.NotAllowedException(errorMsg)); + }); + } + return loadOrCreatePersistentTopic(tpName, createIfMissing, properties, topicPolicies); + }).thenCompose(optionalTopic -> { + if (!optionalTopic.isPresent() && createIfMissing) { + log.warn("[{}] Try to recreate the topic with createIfMissing=true " + + "but the returned topic is empty", topicName); + return getTopic(topicName, createIfMissing, properties); + } + return CompletableFuture.completedFuture(optionalTopic); + }); }); }); } else { 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 e6459bbf74c31..ceb3c1d0d9335 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 @@ -126,7 +126,7 @@ public void testEvents(String topicTypePersistence, String topicTypePartitioned, boolean forceDelete) throws Exception { String topicName = topicTypePersistence + "://" + namespace + "/" + "topic-" + UUID.randomUUID(); - createTopicAndVerifyEvents(topicTypePartitioned, topicName); + createTopicAndVerifyEvents(topicTypePersistence, topicTypePartitioned, topicName); events.clear(); if (topicTypePartitioned.equals("partitioned")) { @@ -150,7 +150,7 @@ public void testEventsWithUnload(String topicTypePersistence, String topicTypePa boolean forceDelete) throws Exception { String topicName = topicTypePersistence + "://" + namespace + "/" + "topic-" + UUID.randomUUID(); - createTopicAndVerifyEvents(topicTypePartitioned, topicName); + createTopicAndVerifyEvents(topicTypePersistence, topicTypePartitioned, topicName); events.clear(); admin.topics().unload(topicName); @@ -182,7 +182,7 @@ public void testEventsActiveSub(String topicTypePersistence, String topicTypePar boolean forceDelete) throws Exception { String topicName = topicTypePersistence + "://" + namespace + "/" + "topic-" + UUID.randomUUID(); - createTopicAndVerifyEvents(topicTypePartitioned, topicName); + createTopicAndVerifyEvents(topicTypePersistence, topicTypePartitioned, topicName); Consumer consumer = pulsarClient.newConsumer().topic(topicName).subscriptionName("sub").subscribe(); Producer producer = pulsarClient.newProducer().topic(topicName).create(); @@ -238,7 +238,7 @@ public void testEventsActiveSub(String topicTypePersistence, String topicTypePar public void testTopicAutoGC(String topicTypePersistence, String topicTypePartitioned) throws Exception { String topicName = topicTypePersistence + "://" + namespace + "/" + "topic-" + UUID.randomUUID(); - createTopicAndVerifyEvents(topicTypePartitioned, topicName); + createTopicAndVerifyEvents(topicTypePersistence, topicTypePartitioned, topicName); admin.namespaces().setInactiveTopicPolicies(namespace, new InactiveTopicPolicies(InactiveTopicDeleteMode.delete_when_no_subscriptions, 1, true)); @@ -262,25 +262,21 @@ public void testTopicAutoGC(String topicTypePersistence, String topicTypePartiti ); } - private void createTopicAndVerifyEvents(String topicTypePartitioned, String topicName) throws Exception { + private void createTopicAndVerifyEvents(String topicDomain, String topicTypePartitioned, String topicName) throws Exception { final String[] expectedEvents; - if (topicTypePartitioned.equals("partitioned")) { - topicNameToWatch = topicName + "-partition-1"; - admin.topics().createPartitionedTopic(topicName, 2); - triggerPartitionsCreation(topicName); - + if (topicDomain.equalsIgnoreCase("persistent") || topicTypePartitioned.equals("partitioned")) { expectedEvents = new String[]{ "LOAD__BEFORE", "CREATE__BEFORE", "CREATE__SUCCESS", "LOAD__SUCCESS" }; - } else { - topicNameToWatch = topicName; - admin.topics().createNonPartitionedTopic(topicName); - expectedEvents = new String[]{ + // Before https://github.com/apache/pulsar/pull/21995, Pulsar will skip create topic if the topic + // was already exists, and the action "check topic exists" will try to load Managed ledger, + // the check triggers two exrtra events: [LOAD__BEFORE, LOAD__FAILURE]. + // #21995 fixed this wrong behavior, so remove these two events. "LOAD__BEFORE", "LOAD__FAILURE", "LOAD__BEFORE", @@ -288,7 +284,14 @@ private void createTopicAndVerifyEvents(String topicTypePartitioned, String topi "CREATE__SUCCESS", "LOAD__SUCCESS" }; - + } + if (topicTypePartitioned.equals("partitioned")) { + topicNameToWatch = topicName + "-partition-1"; + admin.topics().createPartitionedTopic(topicName, 2); + triggerPartitionsCreation(topicName); + } else { + topicNameToWatch = topicName; + admin.topics().createNonPartitionedTopic(topicName); } Awaitility.waitAtMost(10, TimeUnit.SECONDS).untilAsserted(() -> 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 fc4aaf10f966c..4d6ac2201225c 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 @@ -3423,4 +3423,32 @@ private void testAnalyzeSubscriptionBacklogNotCauseStuck() throws Exception { producer.close(); admin.topics().delete(topic); } + + @Test + public void testGetStatsIfPartitionNotExists() throws Exception { + // create topic. + final String partitionedTp = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp"); + admin.topics().createPartitionedTopic(partitionedTp, 1); + TopicName partition0 = TopicName.get(partitionedTp).getPartition(0); + boolean topicExists1 = pulsar.getBrokerService().getTopic(partition0.toString(), false).join().isPresent(); + assertTrue(topicExists1); + // Verify topics-stats works. + TopicStats topicStats = admin.topics().getStats(partition0.toString()); + assertNotNull(topicStats); + + // Delete partition and call topic-stats again. + admin.topics().delete(partition0.toString()); + boolean topicExists2 = pulsar.getBrokerService().getTopic(partition0.toString(), false).join().isPresent(); + assertFalse(topicExists2); + // Verify: respond 404. + try { + admin.topics().getStats(partition0.toString()); + fail("Should respond 404 after the partition was deleted"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("Topic partitions were not yet created")); + } + + // cleanup. + admin.topics().deletePartitionedTopic(partitionedTp); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java index c9138beee52d1..a75ae78cef393 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java @@ -149,10 +149,11 @@ public void testPartitionedTopicAutoCreationForbiddenDuringNamespaceDeletion() .sendTimeout(1, TimeUnit.SECONDS) .topic(topic) .create()) { - } catch (PulsarClientException.LookupException expected) { - String msg = "Namespace bundle for topic (%s) not served by this instance"; + } catch (PulsarClientException.TopicDoesNotExistException expected) { + // Since the "policies.deleted" is "true", the value of "isAllowAutoTopicCreationAsync" will be false, + // so the "TopicDoesNotExistException" is expected. log.info("Expected error", expected); - assertTrue(expected.getMessage().contains(String.format(msg, topic)) + assertTrue(expected.getMessage().contains(topic) || expected.getMessage().contains(topicPoliciesServiceInitException)); } @@ -160,10 +161,11 @@ public void testPartitionedTopicAutoCreationForbiddenDuringNamespaceDeletion() .topic(topic) .subscriptionName("test") .subscribe()) { - } catch (PulsarClientException.LookupException expected) { - String msg = "Namespace bundle for topic (%s) not served by this instance"; + } catch (PulsarClientException.TopicDoesNotExistException expected) { + // Since the "policies.deleted" is "true", the value of "isAllowAutoTopicCreationAsync" will be false, + // so the "TopicDoesNotExistException" is expected. log.info("Expected error", expected); - assertTrue(expected.getMessage().contains(String.format(msg, topic)) + assertTrue(expected.getMessage().contains(topic) || expected.getMessage().contains(topicPoliciesServiceInitException)); } 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 6f60a13fd4894..fe84aeb1df792 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 @@ -295,7 +295,8 @@ public void testPersistentPartitionedTopicUnload() throws Exception { assertFalse(pulsar.getBrokerService().getTopics().containsKey(topicName)); pulsar.getBrokerService().getTopicIfExists(topicName).get(); - assertTrue(pulsar.getBrokerService().getTopics().containsKey(topicName)); + // The map topics should only contain partitions, does not contain partitioned topic. + assertFalse(pulsar.getBrokerService().getTopics().containsKey(topicName)); // ref of partitioned-topic name should be empty assertFalse(pulsar.getBrokerService().getTopicReference(topicName).isPresent()); From 64545609030d744670e856247f12b4f07e16de72 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 11 Oct 2023 10:13:26 +0300 Subject: [PATCH 289/699] [fix][ml] Make mlOwnershipChecker asynchronous so that it doesn't block/deadlock threads (#21333) (cherry picked from commit eb9fa63d6bcaa4e1cbc4b87e36ead5a3ff6c44ae) (cherry picked from commit 750547b3e611d4203172f1361dd7611fbcf55e0e) --- .../mledger/ManagedLedgerFactory.java | 2 +- .../mledger/impl/ManagedCursorImpl.java | 57 ++++++++++++------- .../impl/ManagedLedgerFactoryImpl.java | 2 +- .../mledger/impl/ManagedLedgerImpl.java | 4 +- .../mledger/impl/ShadowManagedLedgerImpl.java | 3 +- .../mledger/impl/ManagedLedgerTest.java | 2 +- .../pulsar/broker/service/BrokerService.java | 19 ++++--- .../impl/MLPendingAckStoreProvider.java | 2 +- .../broker/admin/TopicPoliciesTest.java | 2 +- .../OwnerShipCacheForCurrentServerTest.java | 2 +- 10 files changed, 57 insertions(+), 38 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java index b1427bab80b22..e09fd84ea55f2 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerFactory.java @@ -90,7 +90,7 @@ ManagedLedger open(String name, ManagedLedgerConfig config) * opaque context */ void asyncOpen(String name, ManagedLedgerConfig config, OpenLedgerCallback callback, - Supplier mlOwnershipChecker, Object ctx); + Supplier> mlOwnershipChecker, Object ctx); /** * Open a {@link ReadOnlyCursor} positioned to the earliest entry for the specified managed ledger. 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 2abf6ca9d1743..8db8a5714398c 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 @@ -2682,32 +2682,47 @@ public void operationComplete(Void result, Stat stat) { } @Override - public void operationFailed(MetaStoreException e) { - if (e instanceof MetaStoreException.BadVersionException) { + public void operationFailed(MetaStoreException topLevelException) { + if (topLevelException instanceof MetaStoreException.BadVersionException) { log.warn("[{}] Failed to update cursor metadata for {} due to version conflict {}", - ledger.name, name, e.getMessage()); + ledger.name, name, topLevelException.getMessage()); // it means previous owner of the ml might have updated the version incorrectly. So, check // the ownership and refresh the version again. - if (ledger.mlOwnershipChecker != null && ledger.mlOwnershipChecker.get()) { - ledger.getStore().asyncGetCursorInfo(ledger.getName(), name, - new MetaStoreCallback() { - @Override - public void operationComplete(ManagedCursorInfo info, Stat stat) { - updateCursorLedgerStat(info, stat); - } - - @Override - public void operationFailed(MetaStoreException e) { - if (log.isDebugEnabled()) { - log.debug( - "[{}] Failed to refresh cursor metadata-version for {} due " - + "to {}", ledger.name, name, e.getMessage()); - } - } - }); + if (ledger.mlOwnershipChecker != null) { + ledger.mlOwnershipChecker.get().whenComplete((hasOwnership, t) -> { + if (t == null && hasOwnership) { + ledger.getStore().asyncGetCursorInfo(ledger.getName(), name, + new MetaStoreCallback<>() { + @Override + public void operationComplete(ManagedCursorInfo info, Stat stat) { + updateCursorLedgerStat(info, stat); + // fail the top level call so that the caller can retry + callback.operationFailed(topLevelException); + } + + @Override + public void operationFailed(MetaStoreException e) { + if (log.isDebugEnabled()) { + log.debug( + "[{}] Failed to refresh cursor metadata-version " + + "for {} due to {}", ledger.name, name, + e.getMessage()); + } + // fail the top level call so that the caller can retry + callback.operationFailed(topLevelException); + } + }); + } else { + // fail the top level call so that the caller can retry + callback.operationFailed(topLevelException); + } + }); + } else { + callback.operationFailed(topLevelException); } + } else { + callback.operationFailed(topLevelException); } - callback.operationFailed(e); } }); } 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 1bb23912b5e31..42a5216b3ce36 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 @@ -330,7 +330,7 @@ public void asyncOpen(String name, OpenLedgerCallback callback, Object ctx) { @Override public void asyncOpen(final String name, final ManagedLedgerConfig config, final OpenLedgerCallback callback, - Supplier mlOwnershipChecker, final Object ctx) { + Supplier> mlOwnershipChecker, final Object ctx) { if (closed) { callback.openLedgerFailed(new ManagedLedgerException.ManagedLedgerFactoryClosedException(), ctx); return; 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 e2f0739fae1ee..10405134be939 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 @@ -232,7 +232,7 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback { private static final Random random = new Random(System.currentTimeMillis()); private long maximumRolloverTimeMs; - protected final Supplier mlOwnershipChecker; + protected final Supplier> mlOwnershipChecker; volatile PositionImpl lastConfirmedEntry; @@ -336,7 +336,7 @@ public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper } public ManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper, MetaStore store, ManagedLedgerConfig config, OrderedScheduler scheduledExecutor, - final String name, final Supplier mlOwnershipChecker) { + final String name, final Supplier> mlOwnershipChecker) { this.factory = factory; this.bookKeeper = bookKeeper; this.config = config; 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 b33dd87543f77..8b2742d958783 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 @@ -24,6 +24,7 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; import java.util.function.Supplier; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.AsyncCallback; @@ -50,7 +51,7 @@ public class ShadowManagedLedgerImpl extends ManagedLedgerImpl { public ShadowManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper bookKeeper, MetaStore store, ManagedLedgerConfig config, OrderedScheduler scheduledExecutor, - String name, final Supplier mlOwnershipChecker) { + String name, final Supplier> mlOwnershipChecker) { super(factory, bookKeeper, store, config, scheduledExecutor, name, mlOwnershipChecker); this.sourceMLName = config.getShadowSourceName(); } 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 6c4f21c3af29d..34f65dfd00ee8 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 @@ -3399,7 +3399,7 @@ public void openCursorFailed(ManagedLedgerException exception, Object ctx) { @Override public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { } - }, checkOwnershipFlag ? () -> true : null, null); + }, checkOwnershipFlag ? () -> CompletableFuture.completedFuture(true) : null, null); latch.await(); } 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 a89acc9768f1f..6f61616548b30 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 @@ -1797,7 +1797,7 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { topicFuture.completeExceptionally(new PersistenceException(exception)); } } - }, () -> isTopicNsOwnedByBroker(topicName), null); + }, () -> isTopicNsOwnedByBrokerAsync(topicName), null); }).exceptionally((exception) -> { log.warn("[{}] Failed to get topic configuration: {}", topic, exception.getMessage(), exception); @@ -2193,13 +2193,16 @@ public void monitorBacklogQuota() { }); } - public boolean isTopicNsOwnedByBroker(TopicName topicName) { - try { - return pulsar.getNamespaceService().isServiceUnitOwned(topicName); - } catch (Exception e) { - log.warn("Failed to check the ownership of the topic: {}, {}", topicName, e.getMessage()); - } - return false; + public CompletableFuture isTopicNsOwnedByBrokerAsync(TopicName topicName) { + return pulsar.getNamespaceService().isServiceUnitOwnedAsync(topicName) + .handle((hasOwnership, t) -> { + if (t == null) { + return hasOwnership; + } else { + log.warn("Failed to check the ownership of the topic: {}, {}", topicName, t.getMessage()); + return false; + } + }); } public CompletableFuture checkTopicNsOwnership(final String topic) { 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 ecc6599ce52b5..5308648b80c1d 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 @@ -159,7 +159,7 @@ public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { , originPersistentTopic.getName(), subscription.getName(), exception); pendingAckStoreFuture.completeExceptionally(exception); } - }, () -> true, null); + }, () -> CompletableFuture.completedFuture(true), null); }).exceptionally(e -> { Throwable t = FutureUtil.unwrapCompletionException(e); log.error("[{}] [{}] Failed to get managedLedger config when init pending ack store!", diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java index fe0901bf2d53b..c5111b47d2082 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesTest.java @@ -183,7 +183,7 @@ public void testTopicPolicyInitialValueWithNamespaceAlreadyLoaded() throws Excep //load the nameserver, but topic is not init. log.info("lookup:{}",admin.lookups().lookupTopic(topic)); - assertTrue(pulsar.getBrokerService().isTopicNsOwnedByBroker(topicName)); + assertTrue(pulsar.getBrokerService().isTopicNsOwnedByBrokerAsync(topicName).join()); assertFalse(pulsar.getBrokerService().getTopics().containsKey(topic)); //make sure namespace policy reader is fully started. Awaitility.await().untilAsserted(()-> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipCacheForCurrentServerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipCacheForCurrentServerTest.java index e53d5c25bd2b5..22fa2c32b5655 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipCacheForCurrentServerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipCacheForCurrentServerTest.java @@ -76,7 +76,7 @@ public void testCreateTopicWithNotTopicNsOwnedBroker() { int verifiedBrokerNum = 0; for (PulsarService pulsarService : this.getPulsarServiceList()) { BrokerService bs = pulsarService.getBrokerService(); - if (bs.isTopicNsOwnedByBroker(TopicName.get(topicName))) { + if (bs.isTopicNsOwnedByBrokerAsync(TopicName.get(topicName)).join()) { continue; } verifiedBrokerNum ++; From 4daa01df00541f0a2e73a3b877babae7a9e808e2 Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Sun, 18 Feb 2024 12:29:51 +0800 Subject: [PATCH 290/699] [improve][broker] Add an error log to troubleshoot the failure of starting broker registry. (#22065) (cherry picked from commit baddda56e27486ea3b803bf7569768164444a738) (cherry picked from commit 71022f5ba9801ebbcd9f0bdcb52ef804c1d70db9) --- .../loadbalance/extensions/ExtensibleLoadManagerImpl.java | 2 ++ 1 file changed, 2 insertions(+) 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 5fd675d7df897..baf3e7b156335 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 @@ -390,6 +390,8 @@ public void start() throws PulsarServerException { this.initWaiter.countDown(); this.started = true; } catch (Exception ex) { + log.error("Failed to start the extensible load balance and close broker registry {}.", + this.brokerRegistry, ex); if (this.brokerRegistry != null) { brokerRegistry.close(); } From 17d6ff43444fc416d485c4a11713a4aad244ad24 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Wed, 28 Feb 2024 17:52:37 -0800 Subject: [PATCH 291/699] [fix][broker][branch-3.0] Set ServiceUnitStateChannel topic compaction threshold explicitly, improve getOwnerAsync, and fix other bugs (#22064) (#22154) (cherry picked from commit 6df02655a3c307d29a86b7f3e0ea46c16ad18aea) (cherry picked from commit 6d2ce8966afd1a46df231ccc8d8b54ea7d2496a0) --- .../extensions/ExtensibleLoadManagerImpl.java | 56 ++++++-- .../channel/ServiceUnitStateChannelImpl.java | 101 ++++++++++--- .../store/LoadDataStoreFactory.java | 7 +- .../store/TableViewLoadDataStoreImpl.java | 79 ++++++++-- .../impl/RawBatchMessageContainerImpl.java | 1 + .../ExtensibleLoadManagerImplTest.java | 136 ++++++++++-------- .../channel/ServiceUnitStateChannelTest.java | 77 +++++----- .../extensions/store/LoadDataStoreTest.java | 41 ++---- .../pulsar/client/impl/TableViewImpl.java | 8 +- .../ExtensibleLoadManagerTest.java | 1 - 10 files changed, 325 insertions(+), 182 deletions(-) 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 baf3e7b156335..409bb55075be0 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 @@ -21,6 +21,7 @@ import static java.lang.String.format; import static org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl.Role.Follower; import static org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl.Role.Leader; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl.TOPIC; import static org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision.Label.Success; import static org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision.Reason.Admin; import static org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared.getNamespaceBundle; @@ -117,6 +118,8 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager { private static final long MONITOR_INTERVAL_IN_MILLIS = 120_000; + public static final long COMPACTION_THRESHOLD = 5 * 1024 * 1024; + private static final String ELECTION_ROOT = "/loadbalance/extension/leader"; private PulsarService pulsar; @@ -173,6 +176,8 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager { private volatile boolean started = false; + private boolean configuredSystemTopics = false; + private final AssignCounter assignCounter = new AssignCounter(); @Getter private final UnloadCounter unloadCounter = new UnloadCounter(); @@ -262,6 +267,10 @@ public static boolean isLoadManagerExtensionEnabled(ServiceConfiguration conf) { return ExtensibleLoadManagerImpl.class.getName().equals(conf.getLoadManagerClassName()); } + public static boolean isLoadManagerExtensionEnabled(PulsarService pulsar) { + return pulsar.getLoadManager().get() instanceof ExtensibleLoadManagerWrapper; + } + public static ExtensibleLoadManagerImpl get(LoadManager loadManager) { if (!(loadManager instanceof ExtensibleLoadManagerWrapper loadManagerWrapper)) { throw new IllegalArgumentException("The load manager should be 'ExtensibleLoadManagerWrapper'."); @@ -291,6 +300,27 @@ private static void createSystemTopics(PulsarService pulsar) throws PulsarServer createSystemTopic(pulsar, TOP_BUNDLES_LOAD_DATA_STORE_TOPIC); } + private static boolean configureSystemTopics(PulsarService pulsar) { + try { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar) + && (pulsar.getConfiguration().isSystemTopicEnabled() + && pulsar.getConfiguration().isTopicLevelPoliciesEnabled())) { + Long threshold = pulsar.getAdminClient().topicPolicies().getCompactionThreshold(TOPIC); + if (threshold == null || COMPACTION_THRESHOLD != threshold.longValue()) { + pulsar.getAdminClient().topicPolicies().setCompactionThreshold(TOPIC, COMPACTION_THRESHOLD); + log.info("Set compaction threshold: {} bytes for system topic {}.", COMPACTION_THRESHOLD, TOPIC); + } + } else { + log.warn("System topic or topic level policies is disabled. " + + "{} compaction threshold follows the broker or namespace policies.", TOPIC); + } + return true; + } catch (Exception e) { + log.error("Failed to set compaction threshold for system topic:{}", TOPIC, e); + } + return false; + } + @Override public void start() throws PulsarServerException { if (this.started) { @@ -329,9 +359,9 @@ public void start() throws PulsarServerException { try { this.brokerLoadDataStore = LoadDataStoreFactory - .create(pulsar.getClient(), BROKER_LOAD_DATA_STORE_TOPIC, BrokerLoadData.class); + .create(pulsar, BROKER_LOAD_DATA_STORE_TOPIC, BrokerLoadData.class); this.topBundlesLoadDataStore = LoadDataStoreFactory - .create(pulsar.getClient(), TOP_BUNDLES_LOAD_DATA_STORE_TOPIC, TopBundlesLoadData.class); + .create(pulsar, TOP_BUNDLES_LOAD_DATA_STORE_TOPIC, TopBundlesLoadData.class); } catch (LoadDataStoreException e) { throw new PulsarServerException(e); } @@ -389,6 +419,7 @@ public void start() throws PulsarServerException { this.splitScheduler.start(); this.initWaiter.countDown(); this.started = true; + log.info("Started load manager."); } catch (Exception ex) { log.error("Failed to start the extensible load balance and close broker registry {}.", this.brokerRegistry, ex); @@ -523,7 +554,7 @@ private CompletableFuture> dedupeLookupRequest( if (ex != null) { assignCounter.incrementFailure(); } - lookupRequests.remove(key, newFutureCreated.getValue()); + lookupRequests.remove(key); }); } } @@ -736,13 +767,13 @@ public void close() throws PulsarServerException { } public static boolean isInternalTopic(String topic) { - return topic.startsWith(ServiceUnitStateChannelImpl.TOPIC) + return topic.startsWith(TOPIC) || topic.startsWith(BROKER_LOAD_DATA_STORE_TOPIC) || topic.startsWith(TOP_BUNDLES_LOAD_DATA_STORE_TOPIC); } @VisibleForTesting - void playLeader() { + synchronized void playLeader() { log.info("This broker:{} is setting the role from {} to {}", pulsar.getBrokerId(), role, Leader); int retry = 0; @@ -760,7 +791,7 @@ void playLeader() { serviceUnitStateChannel.scheduleOwnershipMonitor(); break; } catch (Throwable e) { - log.error("The broker:{} failed to set the role. Retrying {} th ...", + log.warn("The broker:{} failed to set the role. Retrying {} th ...", pulsar.getBrokerId(), ++retry, e); try { Thread.sleep(Math.min(retry * 10, MAX_ROLE_CHANGE_RETRY_DELAY_IN_MILLIS)); @@ -780,7 +811,7 @@ void playLeader() { } @VisibleForTesting - void playFollower() { + synchronized void playFollower() { log.info("This broker:{} is setting the role from {} to {}", pulsar.getBrokerId(), role, Follower); int retry = 0; @@ -794,7 +825,7 @@ void playFollower() { topBundlesLoadDataStore.startProducer(); break; } catch (Throwable e) { - log.error("The broker:{} failed to set the role. Retrying {} th ...", + log.warn("The broker:{} failed to set the role. Retrying {} th ...", pulsar.getBrokerId(), ++retry, e); try { Thread.sleep(Math.min(retry * 10, MAX_ROLE_CHANGE_RETRY_DELAY_IN_MILLIS)); @@ -834,7 +865,9 @@ public List getMetrics() { return metricsCollection; } - private void monitor() { + + @VisibleForTesting + protected void monitor() { try { initWaiter.await(); @@ -842,6 +875,11 @@ private void monitor() { // Periodically check the role in case ZK watcher fails. var isChannelOwner = serviceUnitStateChannel.isChannelOwner(); if (isChannelOwner) { + // System topic config might fail due to the race condition + // with topic policy init(Topic policies cache have not init). + if (!configuredSystemTopics) { + configuredSystemTopics = configureSystemTopics(pulsar); + } if (role != Leader) { log.warn("Current role:{} does not match with the channel ownership:{}. " + "Playing the leader role.", role, isChannelOwner); 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 08d21f81b0e07..a94ce7446bd01 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 @@ -492,21 +492,28 @@ private CompletableFuture> getActiveOwnerAsync( String serviceUnit, ServiceUnitState state, Optional owner) { - CompletableFuture> activeOwner = owner.isPresent() - ? brokerRegistry.lookupAsync(owner.get()).thenApply(lookupData -> lookupData.flatMap(__ -> owner)) - : CompletableFuture.completedFuture(Optional.empty()); - - return activeOwner - .thenCompose(broker -> broker - .map(__ -> activeOwner) - .orElseGet(() -> deferGetOwnerRequest(serviceUnit).thenApply(Optional::ofNullable))) - .whenComplete((__, e) -> { + return deferGetOwnerRequest(serviceUnit) + .thenCompose(newOwner -> { + if (newOwner == null) { + return CompletableFuture.completedFuture(null); + } + + return brokerRegistry.lookupAsync(newOwner) + .thenApply(lookupData -> { + if (lookupData.isPresent()) { + return newOwner; + } else { + throw new IllegalStateException( + "The new owner " + newOwner + " is inactive."); + } + }); + }).whenComplete((__, e) -> { if (e != null) { - log.error("Failed to get active owner broker. serviceUnit:{}, state:{}, owner:{}", - serviceUnit, state, owner, e); + log.error("{} failed to get active owner broker. serviceUnit:{}, state:{}, owner:{}", + brokerId, serviceUnit, state, owner, e); ownerLookUpCounters.get(state).getFailure().incrementAndGet(); } - }); + }).thenApply(Optional::ofNullable); } public CompletableFuture> getOwnerAsync(String serviceUnit) { @@ -544,6 +551,25 @@ public CompletableFuture> getOwnerAsync(String serviceUnit) { } } + private Optional getOwner(String serviceUnit) { + ServiceUnitStateData data = tableview.get(serviceUnit); + ServiceUnitState state = state(data); + switch (state) { + case Owned -> { + return Optional.of(data.dstBroker()); + } + case Splitting -> { + return Optional.of(data.sourceBroker()); + } + case Init, Free -> { + return Optional.empty(); + } + default -> { + return null; + } + } + } + private long getNextVersionId(String serviceUnit) { var data = tableview.get(serviceUnit); return getNextVersionId(data); @@ -697,7 +723,7 @@ private AtomicLong getHandlerCounter(ServiceUnitStateData data, boolean total) { private void log(Throwable e, String serviceUnit, ServiceUnitStateData data, ServiceUnitStateData next) { if (e == null) { - if (log.isDebugEnabled() || isTransferCommand(data)) { + if (debug() || isTransferCommand(data)) { long handlerTotalCount = getHandlerTotalCounter(data).get(); long handlerFailureCount = getHandlerFailureCounter(data).get(); log.info("{} handled {} event for serviceUnit:{}, cur:{}, next:{}, " @@ -736,6 +762,9 @@ private void handleSkippedEvent(String serviceUnit) { private void handleOwnEvent(String serviceUnit, ServiceUnitStateData data) { var getOwnerRequest = getOwnerRequests.remove(serviceUnit); if (getOwnerRequest != null) { + if (debug()) { + log.info("Returned owner request for serviceUnit:{}", serviceUnit); + } getOwnerRequest.complete(data.dstBroker()); } stateChangeListeners.notify(serviceUnit, data, null); @@ -848,26 +877,52 @@ private boolean isTargetBroker(String broker) { } private CompletableFuture deferGetOwnerRequest(String serviceUnit) { + var requested = new MutableObject>(); try { return getOwnerRequests .computeIfAbsent(serviceUnit, k -> { - CompletableFuture future = new CompletableFuture<>(); + var ownerBefore = getOwner(serviceUnit); + if (ownerBefore != null && ownerBefore.isPresent()) { + // Here, we do a quick active check first with the computeIfAbsent lock + brokerRegistry.lookupAsync(ownerBefore.get()).getNow(Optional.empty()) + .ifPresent(__ -> requested.setValue( + CompletableFuture.completedFuture(ownerBefore.get()))); + + if (requested.getValue() != null) { + return requested.getValue(); + } + } + + + CompletableFuture future = + new CompletableFuture().orTimeout(inFlightStateWaitingTimeInMillis, + TimeUnit.MILLISECONDS) + .exceptionally(e -> { + var ownerAfter = getOwner(serviceUnit); + log.warn("{} failed to wait for owner for serviceUnit:{}; Trying to " + + "return the current owner:{}", + brokerId, serviceUnit, ownerAfter, e); + if (ownerAfter == null) { + throw new IllegalStateException(e); + } + return ownerAfter.orElse(null); + }); + if (debug()) { + log.info("{} is waiting for owner for serviceUnit:{}", brokerId, serviceUnit); + } requested.setValue(future); return future; }); } finally { var future = requested.getValue(); if (future != null) { - future.orTimeout(inFlightStateWaitingTimeInMillis + 5 * 1000, TimeUnit.MILLISECONDS) - .whenComplete((v, e) -> { - if (e != null) { - getOwnerRequests.remove(serviceUnit, future); - log.warn("Failed to getOwner for serviceUnit:{}", - serviceUnit, e); - } - } - ); + future.whenComplete((__, e) -> { + getOwnerRequests.remove(serviceUnit); + if (e != null) { + log.warn("{} failed to getOwner for serviceUnit:{}", brokerId, serviceUnit, e); + } + }); } } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreFactory.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreFactory.java index 18f39abd76b76..bcb2657c67f05 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreFactory.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/store/LoadDataStoreFactory.java @@ -18,15 +18,16 @@ */ package org.apache.pulsar.broker.loadbalance.extensions.store; -import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.broker.PulsarService; /** * The load data store factory, use to create the load data store. */ public class LoadDataStoreFactory { - public static LoadDataStore create(PulsarClient client, String name, Class clazz) + public static LoadDataStore create(PulsarService pulsar, String name, + Class clazz) throws LoadDataStoreException { - return new TableViewLoadDataStoreImpl<>(client, name, clazz); + return new TableViewLoadDataStoreImpl<>(pulsar, name, clazz); } } 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 56afbef04565c..d916e91716223 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 @@ -23,34 +23,46 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; 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.util.FutureUtil; /** * The load data store, base on {@link TableView }. * * @param Load data type. */ +@Slf4j public class TableViewLoadDataStoreImpl implements LoadDataStore { + private static final long LOAD_DATA_REPORT_UPDATE_MAX_INTERVAL_MULTIPLIER_BEFORE_RESTART = 2; + private volatile TableView tableView; + private volatile long tableViewLastUpdateTimestamp; private volatile Producer producer; + private final ServiceConfiguration conf; + private final PulsarClient client; private final String topic; private final Class clazz; - public TableViewLoadDataStoreImpl(PulsarClient client, String topic, Class clazz) throws LoadDataStoreException { + public TableViewLoadDataStoreImpl(PulsarService pulsar, String topic, Class clazz) + throws LoadDataStoreException { try { - this.client = client; + this.conf = pulsar.getConfiguration(); + this.client = pulsar.getClient(); this.topic = topic; this.clazz = clazz; } catch (Exception e) { @@ -60,40 +72,36 @@ public TableViewLoadDataStoreImpl(PulsarClient client, String topic, Class cl @Override public synchronized CompletableFuture pushAsync(String key, T loadData) { - if (producer == null) { - return FutureUtil.failedFuture(new IllegalStateException("producer has not been started")); - } + validateProducer(); return producer.newMessage().key(key).value(loadData).sendAsync().thenAccept(__ -> {}); } @Override public synchronized CompletableFuture removeAsync(String key) { - if (producer == null) { - return FutureUtil.failedFuture(new IllegalStateException("producer has not been started")); - } + validateProducer(); return producer.newMessage().key(key).value(null).sendAsync().thenAccept(__ -> {}); } @Override public synchronized Optional get(String key) { - validateTableViewStart(); + validateTableView(); return Optional.ofNullable(tableView.get(key)); } @Override public synchronized void forEach(BiConsumer action) { - validateTableViewStart(); + validateTableView(); tableView.forEach(action); } public synchronized Set> entrySet() { - validateTableViewStart(); + validateTableView(); return tableView.entrySet(); } @Override public synchronized int size() { - validateTableViewStart(); + validateTableView(); return tableView.size(); } @@ -116,6 +124,8 @@ public synchronized void startTableView() throws LoadDataStoreException { if (tableView == null) { try { tableView = client.newTableViewBuilder(Schema.JSON(clazz)).topic(topic).create(); + tableView.forEachAndListen((k, v) -> + tableViewLastUpdateTimestamp = System.currentTimeMillis()); } catch (PulsarClientException e) { tableView = null; throw new LoadDataStoreException(e); @@ -150,9 +160,48 @@ public synchronized void init() throws IOException { start(); } - private synchronized void validateTableViewStart() { + private void validateProducer() { + if (producer == null || !producer.isConnected()) { + try { + if (producer != null) { + producer.close(); + } + producer = null; + startProducer(); + log.info("Restarted producer on {}", topic); + } catch (Exception e) { + log.error("Failed to restart producer on {}", topic, e); + throw new RuntimeException(e); + } + } + } + + private void validateTableView() { + String restartReason = null; + if (tableView == null) { - throw new IllegalStateException("table view has not been started"); + restartReason = "table view is null"; + } else { + long inactiveDuration = System.currentTimeMillis() - tableViewLastUpdateTimestamp; + long threshold = TimeUnit.MINUTES.toMillis(conf.getLoadBalancerReportUpdateMaxIntervalMinutes()) + * LOAD_DATA_REPORT_UPDATE_MAX_INTERVAL_MULTIPLIER_BEFORE_RESTART; + if (inactiveDuration > threshold) { + restartReason = String.format("inactiveDuration=%d secs > threshold = %d secs", + TimeUnit.MILLISECONDS.toSeconds(inactiveDuration), + TimeUnit.MILLISECONDS.toSeconds(threshold)); + } + } + + if (StringUtils.isNotBlank(restartReason)) { + tableViewLastUpdateTimestamp = 0; + try { + closeTableView(); + startTableView(); + log.info("Restarted tableview on {}, {}", topic, restartReason); + } catch (Exception e) { + log.error("Failed to restart tableview on {}", topic, e); + throw new RuntimeException(e); + } } } } 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 ba8d3db7178d9..374f1e30c0a89 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 @@ -187,6 +187,7 @@ public ByteBuf toByteBuf() { idData.writeTo(buf); buf.writeInt(metadataAndPayload.readableBytes()); buf.writeBytes(metadataAndPayload); + metadataAndPayload.release(); encryptedPayload.release(); clear(); return buf; 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 86b820a9f7cac..76bc714f9aaf2 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 @@ -116,6 +116,7 @@ import org.apache.pulsar.policies.data.loadbalancer.SystemResourceUsage; import org.awaitility.Awaitility; import org.mockito.MockedStatic; +import org.testng.AssertJUnit; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; @@ -142,64 +143,53 @@ public class ExtensibleLoadManagerImplTest extends MockedPulsarServiceBaseTest { private final String defaultTestNamespace = "public/test"; + private static void initConfig(ServiceConfiguration conf){ + conf.setForceDeleteNamespaceAllowed(true); + conf.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); + conf.setAllowAutoTopicCreation(true); + conf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); + conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); + conf.setLoadBalancerSheddingEnabled(false); + conf.setLoadBalancerDebugModeEnabled(true); + conf.setTopicLevelPoliciesEnabled(true); + } + @BeforeClass @Override public void setup() throws Exception { - try (MockedStatic channelMockedStatic = - mockStatic(ServiceUnitStateChannelImpl.class)) { - channelMockedStatic.when(() -> ServiceUnitStateChannelImpl.newInstance(isA(PulsarService.class))) - .thenAnswer(invocation -> { - PulsarService pulsarService = invocation.getArgument(0); - // Set the inflight state waiting time and ownership monitor delay time to 5 seconds to avoid - // stuck when doing unload. - return new ServiceUnitStateChannelImpl(pulsarService, 5 * 1000, 1); - }); - conf.setForceDeleteNamespaceAllowed(true); - conf.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); - conf.setAllowAutoTopicCreation(true); - conf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); - conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); - conf.setLoadBalancerSheddingEnabled(false); - conf.setLoadBalancerDebugModeEnabled(true); - conf.setTopicLevelPoliciesEnabled(true); - super.internalSetup(conf); - pulsar1 = pulsar; - ServiceConfiguration defaultConf = getDefaultConf(); - defaultConf.setAllowAutoTopicCreation(true); - defaultConf.setForceDeleteNamespaceAllowed(true); - defaultConf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); - defaultConf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); - defaultConf.setLoadBalancerSheddingEnabled(false); - defaultConf.setTopicLevelPoliciesEnabled(true); - additionalPulsarTestContext = createAdditionalPulsarTestContext(defaultConf); - pulsar2 = additionalPulsarTestContext.getPulsarService(); - - setPrimaryLoadManager(); - - setSecondaryLoadManager(); - - admin.clusters().createCluster(this.conf.getClusterName(), - ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); - admin.tenants().createTenant("public", - new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), - Sets.newHashSet(this.conf.getClusterName()))); - admin.namespaces().createNamespace("public/default"); - admin.namespaces().setNamespaceReplicationClusters("public/default", - Sets.newHashSet(this.conf.getClusterName())); - - admin.namespaces().createNamespace(defaultTestNamespace); - admin.namespaces().setNamespaceReplicationClusters(defaultTestNamespace, - Sets.newHashSet(this.conf.getClusterName())); - } + // Set the inflight state waiting time and ownership monitor delay time to 5 seconds to avoid + // stuck when doing unload. + initConfig(conf); + super.internalSetup(conf); + pulsar1 = pulsar; + ServiceConfiguration defaultConf = getDefaultConf(); + initConfig(defaultConf); + additionalPulsarTestContext = createAdditionalPulsarTestContext(defaultConf); + pulsar2 = additionalPulsarTestContext.getPulsarService(); + + setPrimaryLoadManager(); + + setSecondaryLoadManager(); + + admin.clusters().createCluster(this.conf.getClusterName(), + ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); + admin.tenants().createTenant("public", + new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), + Sets.newHashSet(this.conf.getClusterName()))); + admin.namespaces().createNamespace("public/default"); + admin.namespaces().setNamespaceReplicationClusters("public/default", + Sets.newHashSet(this.conf.getClusterName())); + + admin.namespaces().createNamespace(defaultTestNamespace); + admin.namespaces().setNamespaceReplicationClusters(defaultTestNamespace, + Sets.newHashSet(this.conf.getClusterName())); } @Override @AfterClass(alwaysRun = true) protected void cleanup() throws Exception { - pulsar1 = null; - pulsar2.close(); - super.internalCleanup(); this.additionalPulsarTestContext.close(); + super.internalCleanup(); } @BeforeMethod(alwaysRun = true) @@ -237,9 +227,6 @@ public void testAssign() throws Exception { Optional brokerLookupData1 = secondaryLoadManager.assign(Optional.empty(), bundle).get(); assertEquals(brokerLookupData, brokerLookupData1); - verify(primaryLoadManager, times(1)).getBrokerSelectionStrategy(); - verify(secondaryLoadManager, times(0)).getBrokerSelectionStrategy(); - Optional lookupResult = pulsar2.getNamespaceService() .getBrokerServiceUrlAsync(topicName, null).get(); assertTrue(lookupResult.isPresent()); @@ -481,7 +468,8 @@ public void testSplitBundleWithSpecificPositionAdminAPI() throws Exception { "specified_positions_divide", List.of(bundleRanges.get(0), bundleRanges.get(1), splitPosition)); BundlesData bundlesData = admin.namespaces().getBundles(namespace); - assertEquals(bundlesData.getNumBundles(), numBundles + 1); + Awaitility.waitAtMost(15, TimeUnit.SECONDS) + .untilAsserted(() -> assertEquals(bundlesData.getNumBundles(), numBundles + 1)); String lowBundle = String.format("0x%08x", bundleRanges.get(0)); String midBundle = String.format("0x%08x", splitPosition); String highBundle = String.format("0x%08x", bundleRanges.get(1)); @@ -495,15 +483,26 @@ public void testDeleteNamespaceBundle() throws Exception { final String namespace = "public/testDeleteNamespaceBundle"; admin.namespaces().createNamespace(namespace, 3); TopicName topicName = TopicName.get(namespace + "/test-delete-namespace-bundle"); - NamespaceBundle bundle = getBundleAsync(pulsar1, topicName).get(); - String broker = admin.lookups().lookupTopic(topicName.toString()); - log.info("Assign the bundle {} to {}", bundle, broker); - - checkOwnershipState(broker, bundle); - admin.namespaces().deleteNamespaceBundle(topicName.getNamespace(), bundle.getBundleRange()); - assertFalse(primaryLoadManager.checkOwnershipAsync(Optional.empty(), bundle).get()); + Awaitility.await() + .atMost(15, TimeUnit.SECONDS) + .ignoreExceptions() + .untilAsserted(() -> { + NamespaceBundle bundle = getBundleAsync(pulsar1, topicName).get(); + String broker = admin.lookups().lookupTopic(topicName.toString()); + log.info("Assign the bundle {} to {}", bundle, broker); + checkOwnershipState(broker, bundle); + admin.namespaces().deleteNamespaceBundle(topicName.getNamespace(), bundle.getBundleRange(), true); + // this could fail if the system topic lookup asynchronously happens before this. + // we will retry if it fails. + assertFalse(primaryLoadManager.checkOwnershipAsync(Optional.empty(), bundle).get()); + }); + + Awaitility.await() + .atMost(15, TimeUnit.SECONDS) + .ignoreExceptions() + .untilAsserted(() -> admin.namespaces().deleteNamespace(namespace, true)); } @Test(timeOut = 30 * 1000) @@ -727,7 +726,7 @@ private void assertLookupSLANamespaceOwner(PulsarService pulsar, assertEquals(result, expectedBrokerServiceUrl); } - @Test + @Test(priority = 10) public void testTopBundlesLoadDataStoreTableViewFromChannelOwner() throws Exception { var topBundlesLoadDataStorePrimary = (LoadDataStore) FieldUtils.readDeclaredField(primaryLoadManager, "topBundlesLoadDataStore", true); @@ -1238,6 +1237,21 @@ public void testHealthcheck() throws PulsarAdminException { admin.brokers().healthcheck(TopicVersion.V2); } + @Test(timeOut = 30 * 1000) + public void compactionScheduleTest() { + Awaitility.await() + .pollInterval(200, TimeUnit.MILLISECONDS) + .atMost(30, TimeUnit.SECONDS) + .ignoreExceptions() + .untilAsserted(() -> { // wait until true + primaryLoadManager.monitor(); + secondaryLoadManager.monitor(); + var threshold = admin.topicPolicies() + .getCompactionThreshold(ServiceUnitStateChannelImpl.TOPIC, false); + AssertJUnit.assertEquals(5 * 1024 * 1024, threshold == null ? 0 : threshold.longValue()); + }); + } + 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 26bc948a975f4..0cdf6e977ffb1 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 @@ -86,7 +86,6 @@ import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.client.impl.TableViewImpl; import org.apache.pulsar.common.policies.data.TopicType; @@ -331,23 +330,6 @@ private int validateChannelStart(ServiceUnitStateChannelImpl channel) return errorCnt; } - @Test(priority = 1) - public void compactionScheduleTest() { - - Awaitility.await() - .pollInterval(200, TimeUnit.MILLISECONDS) - .atMost(5, TimeUnit.SECONDS) - .untilAsserted(() -> { // wait until true - try { - var threshold = admin.topicPolicies() - .getCompactionThreshold(ServiceUnitStateChannelImpl.TOPIC, false).longValue(); - assertEquals(5 * 1024 * 1024, threshold); - } catch (Exception e) { - ; - } - }); - } - @Test(priority = 2) public void assignmentTest() throws ExecutionException, InterruptedException, IllegalAccessException, TimeoutException { @@ -927,8 +909,7 @@ public void handleBrokerDeletionEventTest() } @Test(priority = 10) - public void conflictAndCompactionTest() throws ExecutionException, InterruptedException, TimeoutException, - IllegalAccessException, PulsarClientException, PulsarServerException { + public void conflictAndCompactionTest() throws Exception { String bundle = String.format("%s/%s", "public/default", "0x0000000a_0xffffffff"); var owner1 = channel1.getOwnerAsync(bundle); var owner2 = channel2.getOwnerAsync(bundle); @@ -961,26 +942,41 @@ public void conflictAndCompactionTest() throws ExecutionException, InterruptedEx Field strategicCompactorField = FieldUtils.getDeclaredField(PulsarService.class, "strategicCompactor", true); FieldUtils.writeField(strategicCompactorField, pulsar1, compactor, true); FieldUtils.writeField(strategicCompactorField, pulsar2, compactor, true); - Awaitility.await() - .pollInterval(200, TimeUnit.MILLISECONDS) - .atMost(140, TimeUnit.SECONDS) - .untilAsserted(() -> { - channel1.publishAssignEventAsync(bundle, brokerId1); - verify(compactor, times(1)) - .compact(eq(ServiceUnitStateChannelImpl.TOPIC), any()); - }); + + var threshold = admin.topicPolicies() + .getCompactionThreshold(ServiceUnitStateChannelImpl.TOPIC); + admin.topicPolicies() + .setCompactionThreshold(ServiceUnitStateChannelImpl.TOPIC, 0); + + try { + Awaitility.await() + .pollInterval(200, TimeUnit.MILLISECONDS) + .atMost(140, TimeUnit.SECONDS) + .untilAsserted(() -> { + channel1.publishAssignEventAsync(bundle, brokerId1); + verify(compactor, times(1)) + .compact(eq(ServiceUnitStateChannelImpl.TOPIC), any()); + }); + + + var channel3 = createChannel(pulsar); + channel3.start(); + Awaitility.await() + .pollInterval(200, TimeUnit.MILLISECONDS) + .atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> assertEquals( + channel3.getOwnerAsync(bundle).get(), Optional.of(brokerId1))); + channel3.close(); + } finally { + FieldUtils.writeDeclaredField(channel2, + "inFlightStateWaitingTimeInMillis", 30 * 1000, true); + if (threshold != null) { + admin.topicPolicies() + .setCompactionThreshold(ServiceUnitStateChannelImpl.TOPIC, threshold); + } + } - var channel3 = createChannel(pulsar); - channel3.start(); - Awaitility.await() - .pollInterval(200, TimeUnit.MILLISECONDS) - .atMost(5, TimeUnit.SECONDS) - .untilAsserted(() -> assertEquals( - channel3.getOwnerAsync(bundle).get(), Optional.of(brokerId1))); - channel3.close(); - FieldUtils.writeDeclaredField(channel2, - "inFlightStateWaitingTimeInMillis", 30 * 1000, true); } @Test(priority = 11) @@ -1583,7 +1579,7 @@ public void testActiveGetOwner() throws Exception { // verify getOwnerAsync times out because the owner is inactive now. long start = System.currentTimeMillis(); var ex = expectThrows(ExecutionException.class, () -> channel1.getOwnerAsync(bundle).get()); - assertTrue(ex.getCause() instanceof TimeoutException); + assertTrue(ex.getCause() instanceof IllegalStateException); assertTrue(System.currentTimeMillis() - start >= 1000); // simulate ownership cleanup(no selected owner) by the leader channel @@ -1783,6 +1779,8 @@ private static void overrideTableView(ServiceUnitStateChannel channel, String se throws IllegalAccessException { var tv = (TableViewImpl) FieldUtils.readField(channel, "tableview", true); + var getOwnerRequests = (Map>) + FieldUtils.readField(channel, "getOwnerRequests", true); var cache = (ConcurrentMap) FieldUtils.readField(tv, "data", true); if(val == null){ @@ -1790,6 +1788,7 @@ private static void overrideTableView(ServiceUnitStateChannel channel, String se } else { cache.put(serviceUnit, val); } + getOwnerRequests.clear(); } private static void cleanOpsCounters(ServiceUnitStateChannel channel) 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 f486370400c92..d25cba2bd1bdd 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 @@ -20,8 +20,6 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertNotNull; -import static org.testng.Assert.fail; import static org.testng.AssertJUnit.assertTrue; import com.google.common.collect.Sets; @@ -29,6 +27,7 @@ import lombok.Cleanup; import lombok.Data; import lombok.NoArgsConstructor; +import org.apache.commons.lang.reflect.FieldUtils; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicDomain; @@ -40,7 +39,6 @@ import java.util.HashMap; import java.util.Map; import java.util.UUID; -import java.util.concurrent.ExecutionException; @Test(groups = "broker") public class LoadDataStoreTest extends MockedPulsarServiceBaseTest { @@ -76,7 +74,7 @@ public void testPushGetAndRemove() throws Exception { @Cleanup LoadDataStore loadDataStore = - LoadDataStoreFactory.create(pulsar.getClient(), topic, MyClass.class); + LoadDataStoreFactory.create(pulsar, topic, MyClass.class); loadDataStore.startProducer(); loadDataStore.startTableView(); MyClass myClass1 = new MyClass("1", 1); @@ -110,7 +108,7 @@ public void testForEach() throws Exception { @Cleanup LoadDataStore loadDataStore = - LoadDataStoreFactory.create(pulsar.getClient(), topic, Integer.class); + LoadDataStoreFactory.create(pulsar, topic, Integer.class); loadDataStore.startProducer(); loadDataStore.startTableView(); @@ -135,7 +133,7 @@ public void testForEach() throws Exception { public void testTableViewRestart() throws Exception { String topic = TopicDomain.persistent + "://" + NamespaceName.SYSTEM_NAMESPACE + "/" + UUID.randomUUID(); LoadDataStore loadDataStore = - LoadDataStoreFactory.create(pulsar.getClient(), topic, Integer.class); + LoadDataStoreFactory.create(pulsar, topic, Integer.class); loadDataStore.startProducer(); loadDataStore.startTableView(); @@ -145,43 +143,26 @@ public void testTableViewRestart() throws Exception { loadDataStore.closeTableView(); loadDataStore.pushAsync("1", 2).get(); - Exception ex = null; - try { - loadDataStore.get("1"); - } catch (IllegalStateException e) { - ex = e; - } - assertNotNull(ex); - loadDataStore.startTableView(); Awaitility.await().untilAsserted(() -> assertEquals(loadDataStore.get("1").get(), 2)); + + loadDataStore.pushAsync("1", 3).get(); + FieldUtils.writeField(loadDataStore, "tableViewLastUpdateTimestamp", 0 , true); + Awaitility.await().untilAsserted(() -> assertEquals(loadDataStore.get("1").get(), 3)); } @Test public void testProducerStop() throws Exception { String topic = TopicDomain.persistent + "://" + NamespaceName.SYSTEM_NAMESPACE + "/" + UUID.randomUUID(); LoadDataStore loadDataStore = - LoadDataStoreFactory.create(pulsar.getClient(), topic, Integer.class); + LoadDataStoreFactory.create(pulsar, topic, Integer.class); loadDataStore.startProducer(); loadDataStore.pushAsync("1", 1).get(); loadDataStore.removeAsync("1").get(); loadDataStore.close(); - try { - loadDataStore.pushAsync("2", 2).get(); - fail(); - } catch (ExecutionException ex) { - assertTrue(ex.getCause() instanceof IllegalStateException); - } - try { - loadDataStore.removeAsync("2").get(); - fail(); - } catch (ExecutionException ex) { - assertTrue(ex.getCause() instanceof IllegalStateException); - } - loadDataStore.startProducer(); - loadDataStore.pushAsync("3", 3).get(); - loadDataStore.removeAsync("3").get(); + loadDataStore.pushAsync("2", 2).get(); + loadDataStore.removeAsync("2").get(); } } 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 560636f94622b..d46f7fa140887 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 @@ -284,8 +284,14 @@ private void readTailMessages(Reader reader) { log.error("Reader {} was closed while reading tail messages.", reader.getTopic(), ex); } else { + // Retrying on the other exceptions such as NotConnectedException + try { + Thread.sleep(50); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } log.warn("Reader {} was interrupted while reading tail messages. " - + "Retrying..", reader.getTopic(), ex); + + "Retrying..", reader.getTopic(), ex); readTailMessages(reader); } return null; diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java index e262b27fe2306..b9707ea76c352 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java @@ -91,7 +91,6 @@ public void setup() throws Exception { "org.apache.pulsar.broker.loadbalance.extensions.scheduler.TransferShedder"); brokerEnvs.put("forceDeleteNamespaceAllowed", "true"); brokerEnvs.put("loadBalancerDebugModeEnabled", "true"); - brokerEnvs.put("topicLevelPoliciesEnabled", "false"); brokerEnvs.put("PULSAR_MEM", "-Xmx512M"); spec.brokerEnvs(brokerEnvs); pulsarCluster = PulsarCluster.forSpec(spec); From 1206f46bde7aa8cb31da92f96279ecca979fb7c3 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Wed, 28 Feb 2024 18:18:22 -0800 Subject: [PATCH 292/699] [fix][broker][branch-3.0] Return getOwnerAsync without waiting on source broker upon Assigning and Releasing and handle role change during role init (#22112) (#22156) (cherry picked from commit b3b1bfb3e2a29674cc9d6144baeef1a3f0058c07) (cherry picked from commit 58a45aa2d2abf1014675cd367132c22000907a3f) --- .../extensions/ExtensibleLoadManagerImpl.java | 24 ++++ .../channel/ServiceUnitStateChannelImpl.java | 11 +- .../ExtensibleLoadManagerImplTest.java | 128 ++++++++++++++---- .../channel/ServiceUnitStateChannelTest.java | 77 +++++++++-- .../ExtensibleLoadManagerTest.java | 35 +++-- 5 files changed, 224 insertions(+), 51 deletions(-) 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 409bb55075be0..6a0e677c66268 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 @@ -777,8 +777,13 @@ synchronized void playLeader() { log.info("This broker:{} is setting the role from {} to {}", pulsar.getBrokerId(), role, Leader); int retry = 0; + boolean becameFollower = false; while (!Thread.currentThread().isInterrupted()) { try { + if (!serviceUnitStateChannel.isChannelOwner()) { + becameFollower = true; + break; + } initWaiter.await(); // 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 @@ -802,6 +807,13 @@ synchronized void playLeader() { } } } + + if (becameFollower) { + log.warn("The broker:{} became follower while initializing leader role.", pulsar.getBrokerId()); + playFollower(); + return; + } + role = Leader; log.info("This broker:{} plays the leader now.", pulsar.getBrokerId()); @@ -815,8 +827,13 @@ synchronized void playFollower() { log.info("This broker:{} is setting the role from {} to {}", pulsar.getBrokerId(), role, Follower); int retry = 0; + boolean becameLeader = false; while (!Thread.currentThread().isInterrupted()) { try { + if (serviceUnitStateChannel.isChannelOwner()) { + becameLeader = true; + break; + } initWaiter.await(); unloadScheduler.close(); serviceUnitStateChannel.cancelOwnershipMonitor(); @@ -836,6 +853,13 @@ synchronized void playFollower() { } } } + + if (becameLeader) { + log.warn("This broker:{} became leader while initializing follower role.", pulsar.getBrokerId()); + playLeader(); + return; + } + role = Follower; log.info("This broker:{} plays a follower now.", pulsar.getBrokerId()); 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 a94ce7446bd01..1471d4a75c175 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 @@ -533,7 +533,16 @@ public CompletableFuture> getOwnerAsync(String serviceUnit) { return getActiveOwnerAsync(serviceUnit, state, Optional.of(data.sourceBroker())); } case Assigning, Releasing -> { - return getActiveOwnerAsync(serviceUnit, state, Optional.empty()); + if (isTargetBroker(data.dstBroker())) { + return getActiveOwnerAsync(serviceUnit, state, Optional.of(data.dstBroker())); + } + // If this broker is not the dst broker, return the dst broker as the owner(or empty). + // Clients need to connect(redirect) to the dst broker anyway + // and wait for the dst broker to receive `Owned`. + // This is also required to return getOwnerAsync on the src broker immediately during unloading. + // Otherwise, topic creation(getOwnerAsync) could block unloading bundles, + // if the topic creation(getOwnerAsync) happens during unloading on the src broker. + return CompletableFuture.completedFuture(Optional.ofNullable(data.dstBroker())); } case Init, Free -> { return CompletableFuture.completedFuture(Optional.empty()); 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 76bc714f9aaf2..fca84f30210ce 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 @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker.loadbalance.extensions; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Releasing; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelTest.overrideTableView; import static org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision.Reason.Admin; import static org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision.Reason.Bandwidth; import static org.apache.pulsar.broker.loadbalance.extensions.models.SplitDecision.Reason.MsgRate; @@ -70,6 +72,7 @@ import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; @@ -78,6 +81,7 @@ import org.apache.pulsar.broker.loadbalance.LeaderElectionService; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateChannelImpl; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLoadData; import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.loadbalance.extensions.data.TopBundlesLoadData; @@ -96,12 +100,14 @@ import org.apache.pulsar.broker.namespace.NamespaceBundleSplitListener; import org.apache.pulsar.broker.namespace.NamespaceEphemeralData; 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.PulsarAdminException; import org.apache.pulsar.client.impl.TableViewImpl; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.ServiceUnitId; +import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.naming.TopicVersion; import org.apache.pulsar.common.policies.data.BrokerAssignment; @@ -794,7 +800,6 @@ public void testRoleChange() throws Exception { reset(); return null; }).when(topBundlesLoadDataStorePrimarySpy).closeTableView(); - FieldUtils.writeDeclaredField(primaryLoadManager, "topBundlesLoadDataStore", topBundlesLoadDataStorePrimarySpy, true); var topBundlesLoadDataStoreSecondary = (LoadDataStore) FieldUtils.readDeclaredField(secondaryLoadManager, "topBundlesLoadDataStore", true); @@ -817,36 +822,65 @@ public void testRoleChange() throws Exception { reset(); return null; }).when(topBundlesLoadDataStoreSecondarySpy).closeTableView(); - FieldUtils.writeDeclaredField(secondaryLoadManager, "topBundlesLoadDataStore", topBundlesLoadDataStoreSecondarySpy, true); - if (channel1.isChannelOwnerAsync().get(5, TimeUnit.SECONDS)) { - primaryLoadManager.playFollower(); // close 3 times - primaryLoadManager.playFollower(); // close 1 time - secondaryLoadManager.playLeader(); - secondaryLoadManager.playLeader(); - primaryLoadManager.playLeader(); // close 3 times and open 3 times - primaryLoadManager.playLeader(); // close 1 time and open 1 time, - secondaryLoadManager.playFollower(); - secondaryLoadManager.playFollower(); - } else { - primaryLoadManager.playLeader(); - primaryLoadManager.playLeader(); - secondaryLoadManager.playFollower(); - secondaryLoadManager.playFollower(); - primaryLoadManager.playFollower(); + try { + FieldUtils.writeDeclaredField(primaryLoadManager, "topBundlesLoadDataStore", + topBundlesLoadDataStorePrimarySpy, true); + FieldUtils.writeDeclaredField(secondaryLoadManager, "topBundlesLoadDataStore", + topBundlesLoadDataStoreSecondarySpy, true); + + + if (channel1.isChannelOwnerAsync().get(5, TimeUnit.SECONDS)) { + primaryLoadManager.playLeader(); + secondaryLoadManager.playFollower(); + verify(topBundlesLoadDataStorePrimarySpy, times(3)).startTableView(); + verify(topBundlesLoadDataStorePrimarySpy, times(5)).closeTableView(); + verify(topBundlesLoadDataStoreSecondarySpy, times(0)).startTableView(); + verify(topBundlesLoadDataStoreSecondarySpy, times(3)).closeTableView(); + } else { + primaryLoadManager.playFollower(); + secondaryLoadManager.playLeader(); + verify(topBundlesLoadDataStoreSecondarySpy, times(3)).startTableView(); + verify(topBundlesLoadDataStoreSecondarySpy, times(5)).closeTableView(); + verify(topBundlesLoadDataStorePrimarySpy, times(0)).startTableView(); + verify(topBundlesLoadDataStorePrimarySpy, times(3)).closeTableView(); + } + primaryLoadManager.playFollower(); - secondaryLoadManager.playLeader(); - secondaryLoadManager.playLeader(); - } + secondaryLoadManager.playFollower(); + if (channel1.isChannelOwnerAsync().get(5, TimeUnit.SECONDS)) { + assertEquals(ExtensibleLoadManagerImpl.Role.Leader, + FieldUtils.readDeclaredField(primaryLoadManager, "role", true)); + assertEquals(ExtensibleLoadManagerImpl.Role.Follower, + FieldUtils.readDeclaredField(secondaryLoadManager, "role", true)); + } else { + assertEquals(ExtensibleLoadManagerImpl.Role.Follower, + FieldUtils.readDeclaredField(primaryLoadManager, "role", true)); + assertEquals(ExtensibleLoadManagerImpl.Role.Leader, + FieldUtils.readDeclaredField(secondaryLoadManager, "role", true)); + } - verify(topBundlesLoadDataStorePrimarySpy, times(4)).startTableView(); - verify(topBundlesLoadDataStorePrimarySpy, times(8)).closeTableView(); - verify(topBundlesLoadDataStoreSecondarySpy, times(4)).startTableView(); - verify(topBundlesLoadDataStoreSecondarySpy, times(8)).closeTableView(); + primaryLoadManager.playLeader(); + secondaryLoadManager.playLeader(); - FieldUtils.writeDeclaredField(primaryLoadManager, "topBundlesLoadDataStore", topBundlesLoadDataStorePrimary, true); - FieldUtils.writeDeclaredField(secondaryLoadManager, "topBundlesLoadDataStore", topBundlesLoadDataStoreSecondary, true); + if (channel1.isChannelOwnerAsync().get(5, TimeUnit.SECONDS)) { + assertEquals(ExtensibleLoadManagerImpl.Role.Leader, + FieldUtils.readDeclaredField(primaryLoadManager, "role", true)); + assertEquals(ExtensibleLoadManagerImpl.Role.Follower, + FieldUtils.readDeclaredField(secondaryLoadManager, "role", true)); + } else { + assertEquals(ExtensibleLoadManagerImpl.Role.Follower, + FieldUtils.readDeclaredField(primaryLoadManager, "role", true)); + assertEquals(ExtensibleLoadManagerImpl.Role.Leader, + FieldUtils.readDeclaredField(secondaryLoadManager, "role", true)); + } + } finally { + FieldUtils.writeDeclaredField(primaryLoadManager, "topBundlesLoadDataStore", + topBundlesLoadDataStorePrimary, true); + FieldUtils.writeDeclaredField(secondaryLoadManager, "topBundlesLoadDataStore", + topBundlesLoadDataStoreSecondary, true); + } } @Test @@ -1252,6 +1286,32 @@ public void compactionScheduleTest() { }); } + @Test(timeOut = 10 * 1000) + public void unloadTimeoutCheckTest() + throws Exception { + Pair topicAndBundle = getBundleIsNotOwnByChangeEventTopic("unload-timeout"); + String topic = topicAndBundle.getLeft().toString(); + var bundle = topicAndBundle.getRight().toString(); + var releasing = new ServiceUnitStateData(Releasing, pulsar2.getBrokerId(), pulsar1.getBrokerId(), 1); + overrideTableView(channel1, bundle, releasing); + var topicFuture = pulsar1.getBrokerService().getOrCreateTopic(topic); + + + try { + topicFuture.get(1, TimeUnit.SECONDS); + } catch (Exception e) { + log.info("getOrCreateTopic failed", e); + if (!(e.getCause() instanceof BrokerServiceException.ServiceUnitNotReadyException && e.getMessage() + .contains("Please redo the lookup"))) { + fail(); + } + } + + pulsar1.getBrokerService() + .unloadServiceUnit(topicAndBundle.getRight(), true, 5, + TimeUnit.SECONDS).get(2, TimeUnit.SECONDS); + } + private static abstract class MockBrokerFilter implements BrokerFilter { @Override @@ -1284,4 +1344,20 @@ private void setSecondaryLoadManager() throws IllegalAccessException { private CompletableFuture getBundleAsync(PulsarService pulsar, TopicName topic) { return pulsar.getNamespaceService().getBundleAsync(topic); } + + private Pair getBundleIsNotOwnByChangeEventTopic(String topicNamePrefix) + throws Exception { + TopicName changeEventsTopicName = + TopicName.get(defaultTestNamespace + "/" + SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME); + NamespaceBundle changeEventsBundle = getBundleAsync(pulsar1, changeEventsTopicName).get(); + int i = 0; + while (true) { + TopicName topicName = TopicName.get(defaultTestNamespace + "/" + topicNamePrefix + "-" + i); + NamespaceBundle bundle = getBundleAsync(pulsar1, topicName).get(); + if (!bundle.equals(changeEventsBundle)) { + return Pair.of(topicName, bundle); + } + i++; + } + } } 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 0cdf6e977ffb1..860e892ed6199 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 @@ -484,19 +484,17 @@ public void transferTestWhenDestBrokerFails() var owner1 = channel1.getOwnerAsync(bundle); var owner2 = channel2.getOwnerAsync(bundle); - assertFalse(owner1.isDone()); + assertTrue(owner1.isDone()); + assertEquals(brokerId2, owner1.get().get()); assertFalse(owner2.isDone()); - assertEquals(1, getOwnerRequests1.size()); + assertEquals(0, getOwnerRequests1.size()); assertEquals(1, getOwnerRequests2.size()); // In 10 secs, the getOwnerAsync requests(lookup requests) should time out. - Awaitility.await().atMost(10, TimeUnit.SECONDS) - .untilAsserted(() -> assertTrue(owner1.isCompletedExceptionally())); Awaitility.await().atMost(10, TimeUnit.SECONDS) .untilAsserted(() -> assertTrue(owner2.isCompletedExceptionally())); - assertEquals(0, getOwnerRequests1.size()); assertEquals(0, getOwnerRequests2.size()); // recovered, check the monitor update state : Assigned -> Owned @@ -1133,12 +1131,10 @@ public void assignTestWhenDestBrokerProducerFails() var owner1 = channel1.getOwnerAsync(bundle); var owner2 = channel2.getOwnerAsync(bundle); - assertFalse(owner1.isDone()); + assertTrue(owner1.isDone()); assertFalse(owner2.isDone()); // In 10 secs, the getOwnerAsync requests(lookup requests) should time out. - Awaitility.await().atMost(10, TimeUnit.SECONDS) - .untilAsserted(() -> assertTrue(owner1.isCompletedExceptionally())); Awaitility.await().atMost(10, TimeUnit.SECONDS) .untilAsserted(() -> assertTrue(owner2.isCompletedExceptionally())); @@ -1317,6 +1313,68 @@ public void testIsOwner() throws IllegalAccessException { assertFalse(channel1.isOwner(bundle)); } + @Test(priority = 15) + public void testGetOwnerAsync() throws Exception { + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Owned, brokerId1, 1)); + var owner = channel1.getOwnerAsync(bundle); + assertTrue(owner.isDone()); + assertEquals(brokerId1, owner.get().get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Owned, brokerId2, 1)); + owner = channel1.getOwnerAsync(bundle); + assertTrue(owner.isDone()); + assertEquals(brokerId2, owner.get().get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigning, brokerId1, 1)); + owner = channel1.getOwnerAsync(bundle); + assertTrue(!owner.isDone()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Assigning, brokerId2, 1)); + owner = channel1.getOwnerAsync(bundle); + assertTrue(owner.isDone()); + assertEquals(brokerId2, owner.get().get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, brokerId1, 1)); + owner = channel1.getOwnerAsync(bundle); + assertTrue(!owner.isDone()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, brokerId2, 1)); + owner = channel1.getOwnerAsync(bundle); + assertTrue(owner.isDone()); + assertEquals(brokerId2, owner.get().get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Releasing, null, brokerId1, 1)); + owner = channel1.getOwnerAsync(bundle); + assertTrue(owner.isDone()); + assertEquals(Optional.empty(), owner.get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, null, brokerId1, 1)); + owner = channel1.getOwnerAsync(bundle); + assertTrue(owner.isDone()); + assertEquals(brokerId1, owner.get().get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Splitting, null, brokerId2, 1)); + owner = channel1.getOwnerAsync(bundle); + assertTrue(owner.isDone()); + assertEquals(brokerId2, owner.get().get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Free, null, brokerId1, 1)); + owner = channel1.getOwnerAsync(bundle); + assertTrue(owner.isDone()); + assertEquals(Optional.empty(), owner.get()); + + overrideTableView(channel1, bundle, new ServiceUnitStateData(Deleted, null, brokerId1, 1)); + owner = channel1.getOwnerAsync(bundle); + assertTrue(owner.isDone()); + assertTrue(owner.isCompletedExceptionally()); + + overrideTableView(channel1, bundle, null); + owner = channel1.getOwnerAsync(bundle); + assertTrue(owner.isDone()); + assertEquals(Optional.empty(), owner.get()); + } + @Test(priority = 16) public void splitAndRetryFailureTest() throws Exception { channel1.publishAssignEventAsync(bundle3, brokerId1); @@ -1775,7 +1833,8 @@ private void overrideTableViews(String serviceUnit, ServiceUnitStateData val) th overrideTableView(channel2, serviceUnit, val); } - private static void overrideTableView(ServiceUnitStateChannel channel, String serviceUnit, ServiceUnitStateData val) + @Test(enabled = false) + public static void overrideTableView(ServiceUnitStateChannel channel, String serviceUnit, ServiceUnitStateData val) throws IllegalAccessException { var tv = (TableViewImpl) FieldUtils.readField(channel, "tableview", true); diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java index b9707ea76c352..af14ef97f85c3 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java @@ -324,8 +324,8 @@ public void testAntiaffinityPolicy() throws PulsarAdminException { assertEquals(result.size(), NUM_BROKERS); } - @Test(timeOut = 40 * 1000) - public void testIsolationPolicy() throws PulsarAdminException { + @Test(timeOut = 300 * 1000) + public void testIsolationPolicy() throws Exception { final String namespaceIsolationPolicyName = "my-isolation-policy"; final String isolationEnabledNameSpace = DEFAULT_TENANT + "/my-isolation-policy" + nsSuffix; Map parameters1 = new HashMap<>(); @@ -334,7 +334,8 @@ public void testIsolationPolicy() throws PulsarAdminException { Awaitility.await().atMost(10, TimeUnit.SECONDS).ignoreExceptions().untilAsserted( () -> { - List activeBrokers = admin.brokers().getActiveBrokers(); + List activeBrokers = admin.brokers().getActiveBrokersAsync() + .get(5, TimeUnit.SECONDS); assertEquals(activeBrokers.size(), NUM_BROKERS); } ); @@ -377,15 +378,16 @@ public void testIsolationPolicy() throws PulsarAdminException { } } - Awaitility.await().atMost(30, TimeUnit.SECONDS).ignoreExceptions().untilAsserted( + Awaitility.await().atMost(60, TimeUnit.SECONDS).ignoreExceptions().untilAsserted( () -> { - List activeBrokers = admin.brokers().getActiveBrokers(); + List activeBrokers = admin.brokers().getActiveBrokersAsync() + .get(5, TimeUnit.SECONDS); assertEquals(activeBrokers.size(), 2); } ); Awaitility.await().atMost(60, TimeUnit.SECONDS).ignoreExceptions().untilAsserted(() -> { - String ownerBroker = admin.lookups().lookupTopic(topic); + String ownerBroker = admin.lookups().lookupTopicAsync(topic).get(5, TimeUnit.SECONDS); assertEquals(extractBrokerIndex(ownerBroker), 1); }); @@ -396,20 +398,23 @@ public void testIsolationPolicy() throws PulsarAdminException { } } - Awaitility.await().atMost(30, TimeUnit.SECONDS).ignoreExceptions().untilAsserted( + Awaitility.await().atMost(60, TimeUnit.SECONDS).ignoreExceptions().untilAsserted( () -> { - List activeBrokers = admin.brokers().getActiveBrokers(); + List activeBrokers = admin.brokers().getActiveBrokersAsync().get(5, TimeUnit.SECONDS); assertEquals(activeBrokers.size(), 1); } ); - try { - admin.lookups().lookupTopic(topic); - fail(); - } catch (Exception ex) { - log.error("Failed to lookup topic: ", ex); - assertThat(ex.getMessage()).contains("Failed to select the new owner broker for bundle"); - } + Awaitility.await().atMost(60, TimeUnit.SECONDS).ignoreExceptions().untilAsserted( + () -> { + try { + admin.lookups().lookupTopicAsync(topic).get(5, TimeUnit.SECONDS); + } catch (Exception ex) { + log.error("Failed to lookup topic: ", ex); + assertThat(ex.getMessage()).contains("Failed to select the new owner broker for bundle"); + } + } + ); } private String getBrokerUrl(int index) { From c5dcd7001d03e6eb94be0b841ad2418ec752d0d8 Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Fri, 1 Mar 2024 11:16:38 +0530 Subject: [PATCH 293/699] Fix for import statements --- .../service/SystemTopicBasedTopicPoliciesServiceTest.java | 2 ++ .../org/apache/pulsar/broker/transaction/TransactionTest.java | 1 + 2 files changed, 3 insertions(+) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java index cde41ffae6835..bf2698d400872 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/SystemTopicBasedTopicPoliciesServiceTest.java @@ -37,6 +37,8 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; + +import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; 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 3f0ef0ea74208..12f126b20bd7b 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 @@ -106,6 +106,7 @@ import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore; import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStoreProvider; import org.apache.pulsar.broker.transaction.pendingack.impl.PendingAckHandleImpl; +import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; From 0dc60c6aa0d49b4651c8a8def43edc8fa721ab99 Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Fri, 1 Mar 2024 12:12:17 +0530 Subject: [PATCH 294/699] Fix for LICENSE --- .../server/src/assemble/LICENSE.bin.txt | 44 +++++++++---------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 9b7074a6299a0..014fbed905326 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -585,28 +585,28 @@ Bouncy Castle License - org.bouncycastle-bcutil-jdk18on-1.75.jar Datastax - - com.datastax.oss-managed-ledger-3.1.0.2.jar - - com.datastax.oss-pulsar-broker-3.1.0.2.jar - - com.datastax.oss-pulsar-broker-auth-oidc-3.1.0.2.jar - - com.datastax.oss-pulsar-broker-auth-sasl-3.1.0.2.jar - - com.datastax.oss-pulsar-broker-common-3.1.0.2.jar - - com.datastax.oss-pulsar-config-validation-3.1.0.2.jar - - com.datastax.oss-pulsar-docs-tools-3.1.0.2.jar - - com.datastax.oss-pulsar-functions-api-3.1.0.2.jar - - com.datastax.oss-pulsar-functions-instance-3.1.0.2.jar - - com.datastax.oss-pulsar-functions-local-runner-original-3.1.0.2.jar - - com.datastax.oss-pulsar-functions-proto-3.1.0.2.jar - - com.datastax.oss-pulsar-functions-runtime-3.1.0.2.jar - - com.datastax.oss-pulsar-functions-secrets-3.1.0.2.jar - - com.datastax.oss-pulsar-functions-utils-3.1.0.2.jar - - com.datastax.oss-pulsar-functions-worker-3.1.0.2.jar - - com.datastax.oss-pulsar-io-core-3.1.0.2.jar - - com.datastax.oss-pulsar-metadata-3.1.0.2.jar - - com.datastax.oss-pulsar-proxy-3.1.0.2.jar - - com.datastax.oss-pulsar-testclient-3.1.0.2.jar - - com.datastax.oss-pulsar-transaction-common-3.1.0.2.jar - - com.datastax.oss-pulsar-transaction-coordinator-3.1.0.2.jar - - com.datastax.oss-testmocks-3.1.0.2.jar + - com.datastax.oss-managed-ledger-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-pulsar-broker-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-pulsar-broker-auth-oidc-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-pulsar-broker-auth-sasl-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-pulsar-broker-common-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-pulsar-config-validation-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-pulsar-docs-tools-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-pulsar-functions-api-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-pulsar-functions-instance-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-pulsar-functions-local-runner-original-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-pulsar-functions-proto-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-pulsar-functions-runtime-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-pulsar-functions-secrets-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-pulsar-functions-utils-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-pulsar-functions-worker-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-pulsar-io-core-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-pulsar-metadata-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-pulsar-proxy-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-pulsar-testclient-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-pulsar-transaction-common-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-pulsar-transaction-coordinator-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-testmocks-3.1.0.3-SNAPSHOT.jar ------------------------ From 956f42d9ca41701909905b24d56ba675f4aa5336 Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Thu, 29 Feb 2024 18:57:03 +0800 Subject: [PATCH 295/699] [improve][broker] Add fine-grain authorization to retention admin API (#22163) (cherry picked from commit 6ec473ed6458cf30e1fc7062057a50bfefada6cf) (cherry picked from commit 4f2e4c81ab7cecd1bf020acae1b9223698fb0363) --- .../broker/admin/v2/PersistentTopics.java | 9 +- .../broker/admin/TopicPoliciesAuthZTest.java | 175 ++++++++++++++++++ .../security/MockedPulsarStandalone.java | 155 ++++++++++++++++ 3 files changed, 336 insertions(+), 3 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/security/MockedPulsarStandalone.java 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 b8457339213c0..1fc46f9c872b9 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 @@ -2414,7 +2414,8 @@ public void getRetention(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.RETENTION, PolicyOperation.READ) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalGetRetention(applied, isGlobal)) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { @@ -2441,7 +2442,8 @@ public void setRetention(@Suspended final AsyncResponse asyncResponse, @QueryParam("isGlobal") @DefaultValue("false") boolean isGlobal, @ApiParam(value = "Retention policies for the specified topic") RetentionPolicies retention) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.RETENTION, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalSetRetention(retention, isGlobal)) .thenRun(() -> { try { @@ -2477,7 +2479,8 @@ public void removeRetention(@Suspended final AsyncResponse asyncResponse, @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - preValidation(authoritative) + validateTopicPolicyOperationAsync(topicName, PolicyName.RETENTION, PolicyOperation.WRITE) + .thenCompose(__ -> preValidation(authoritative)) .thenCompose(__ -> internalRemoveRetention(isGlobal)) .thenRun(() -> { log.info("[{}] Successfully remove retention: namespace={}, topic={}", diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.java new file mode 100644 index 0000000000000..f07b9a6c2aabf --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicPoliciesAuthZTest.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.broker.admin; + +import io.jsonwebtoken.Jwts; +import java.util.Set; +import java.util.UUID; +import lombok.Cleanup; +import lombok.SneakyThrows; +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.AuthAction; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.security.MockedPulsarStandalone; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static org.awaitility.Awaitility.await; + + +public final class TopicPoliciesAuthZTest extends MockedPulsarStandalone { + + private PulsarAdmin superUserAdmin; + + private PulsarAdmin tenantManagerAdmin; + + private static final String TENANT_ADMIN_SUBJECT = UUID.randomUUID().toString(); + private static final String TENANT_ADMIN_TOKEN = Jwts.builder() + .claim("sub", TENANT_ADMIN_SUBJECT).signWith(SECRET_KEY).compact(); + + @SneakyThrows + @BeforeClass + public void before() { + configureTokenAuthentication(); + configureDefaultAuthorization(); + start(); + this.superUserAdmin =PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(SUPER_USER_TOKEN)) + .build(); + final TenantInfo tenantInfo = superUserAdmin.tenants().getTenantInfo("public"); + tenantInfo.getAdminRoles().add(TENANT_ADMIN_SUBJECT); + superUserAdmin.tenants().updateTenant("public", tenantInfo); + this.tenantManagerAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(TENANT_ADMIN_TOKEN)) + .build(); + } + + + @SneakyThrows + @AfterClass + public void after() { + close(); + } + + + @SneakyThrows + @Test + public void testRetention() { + final String random = UUID.randomUUID().toString(); + final String topic = "persistent://public/default/" + random; + final String subject = UUID.randomUUID().toString(); + final String token = Jwts.builder() + .claim("sub", subject).signWith(SECRET_KEY).compact(); + superUserAdmin.topics().createNonPartitionedTopic(topic); + + @Cleanup + final PulsarAdmin subAdmin = PulsarAdmin.builder() + .serviceHttpUrl(getPulsarService().getWebServiceAddress()) + .authentication(new AuthenticationToken(token)) + .build(); + final RetentionPolicies definedRetentionPolicy = new RetentionPolicies(1, 1); + // test superuser + superUserAdmin.topicPolicies().setRetention(topic, definedRetentionPolicy); + + // because the topic policies is eventual consistency, we should wait here + await().untilAsserted(() -> { + final RetentionPolicies receivedRetentionPolicy = superUserAdmin.topicPolicies().getRetention(topic); + Assert.assertEquals(receivedRetentionPolicy, definedRetentionPolicy); + }); + superUserAdmin.topicPolicies().removeRetention(topic); + + await().untilAsserted(() -> { + final RetentionPolicies retention = superUserAdmin.topicPolicies().getRetention(topic); + Assert.assertNull(retention); + }); + + // test tenant manager + + tenantManagerAdmin.topicPolicies().setRetention(topic, definedRetentionPolicy); + await().untilAsserted(() -> { + final RetentionPolicies receivedRetentionPolicy = tenantManagerAdmin.topicPolicies().getRetention(topic); + Assert.assertEquals(receivedRetentionPolicy, definedRetentionPolicy); + }); + tenantManagerAdmin.topicPolicies().removeRetention(topic); + await().untilAsserted(() -> { + final RetentionPolicies retention = tenantManagerAdmin.topicPolicies().getRetention(topic); + Assert.assertNull(retention); + }); + + // test nobody + + try { + subAdmin.topicPolicies().getRetention(topic); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + + try { + + subAdmin.topicPolicies().setRetention(topic, definedRetentionPolicy); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + + try { + subAdmin.topicPolicies().removeRetention(topic); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + + // test sub user with permissions + for (AuthAction action : AuthAction.values()) { + superUserAdmin.namespaces().grantPermissionOnNamespace("public/default", + subject, Set.of(action)); + try { + subAdmin.topicPolicies().getRetention(topic); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + + try { + + subAdmin.topicPolicies().setRetention(topic, definedRetentionPolicy); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + + try { + subAdmin.topicPolicies().removeRetention(topic); + Assert.fail("unexpected behaviour"); + } catch (PulsarAdminException ex) { + Assert.assertTrue(ex instanceof PulsarAdminException.NotAuthorizedException); + } + superUserAdmin.namespaces().revokePermissionsOnNamespace("public/default", subject); + } + } + +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/security/MockedPulsarStandalone.java b/pulsar-broker/src/test/java/org/apache/pulsar/security/MockedPulsarStandalone.java new file mode 100644 index 0000000000000..20dd2e1066a87 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/security/MockedPulsarStandalone.java @@ -0,0 +1,155 @@ +/* + * 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.security; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Sets; +import io.jsonwebtoken.Jwts; +import io.jsonwebtoken.SignatureAlgorithm; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; +import javax.crypto.SecretKey; +import lombok.Getter; +import lombok.SneakyThrows; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; +import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils; +import org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider; +import org.apache.pulsar.broker.testcontext.PulsarTestContext; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.impl.auth.AuthenticationToken; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.util.ObjectMapperFactory; + + +public abstract class MockedPulsarStandalone implements AutoCloseable { + + @Getter + private final ServiceConfiguration serviceConfiguration = new ServiceConfiguration(); + private PulsarTestContext pulsarTestContext; + + @Getter + private PulsarService pulsarService; + private PulsarAdmin serviceInternalAdmin; + + + { + serviceConfiguration.setClusterName(TEST_CLUSTER_NAME); + serviceConfiguration.setBrokerShutdownTimeoutMs(0L); + serviceConfiguration.setBrokerServicePort(Optional.of(0)); + serviceConfiguration.setBrokerServicePortTls(Optional.of(0)); + serviceConfiguration.setAdvertisedAddress("localhost"); + serviceConfiguration.setWebServicePort(Optional.of(0)); + serviceConfiguration.setWebServicePortTls(Optional.of(0)); + serviceConfiguration.setNumExecutorThreadPoolSize(5); + serviceConfiguration.setExposeBundlesMetricsInPrometheus(true); + } + + + protected static final SecretKey SECRET_KEY = AuthTokenUtils.createSecretKey(SignatureAlgorithm.HS256); + + private static final String BROKER_INTERNAL_CLIENT_SUBJECT = "broker_internal"; + private static final String BROKER_INTERNAL_CLIENT_TOKEN = Jwts.builder() + .claim("sub", BROKER_INTERNAL_CLIENT_SUBJECT).signWith(SECRET_KEY).compact(); + protected static final String SUPER_USER_SUBJECT = "super-user"; + protected static final String SUPER_USER_TOKEN = Jwts.builder() + .claim("sub", SUPER_USER_SUBJECT).signWith(SECRET_KEY).compact(); + protected static final String NOBODY_SUBJECT = "nobody"; + protected static final String NOBODY_TOKEN = Jwts.builder() + .claim("sub", NOBODY_SUBJECT).signWith(SECRET_KEY).compact(); + + + @SneakyThrows + protected void configureTokenAuthentication() { + serviceConfiguration.setAuthenticationEnabled(true); + serviceConfiguration.setAuthenticationProviders(Set.of(AuthenticationProviderToken.class.getName())); + // internal client + serviceConfiguration.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName()); + final Map brokerClientAuthParams = new HashMap<>(); + brokerClientAuthParams.put("token", BROKER_INTERNAL_CLIENT_TOKEN); + final String brokerClientAuthParamStr = MAPPER.writeValueAsString(brokerClientAuthParams); + serviceConfiguration.setBrokerClientAuthenticationParameters(brokerClientAuthParamStr); + + Properties properties = serviceConfiguration.getProperties(); + if (properties == null) { + properties = new Properties(); + serviceConfiguration.setProperties(properties); + } + properties.put("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(SECRET_KEY)); + + } + + + + protected void configureDefaultAuthorization() { + serviceConfiguration.setAuthorizationEnabled(true); + serviceConfiguration.setAuthorizationProvider(PulsarAuthorizationProvider.class.getName()); + serviceConfiguration.setSuperUserRoles(Set.of(SUPER_USER_SUBJECT, BROKER_INTERNAL_CLIENT_SUBJECT)); + } + + + @SneakyThrows + protected void start() { + this.pulsarTestContext = PulsarTestContext.builder() + .spyByDefault() + .config(serviceConfiguration) + .withMockZookeeper(false) + .build(); + this.pulsarService = pulsarTestContext.getPulsarService(); + this.serviceInternalAdmin = pulsarService.getAdminClient(); + setupDefaultTenantAndNamespace(); + } + + private void setupDefaultTenantAndNamespace() throws Exception { + if (!serviceInternalAdmin.clusters().getClusters().contains(TEST_CLUSTER_NAME)) { + serviceInternalAdmin.clusters().createCluster(TEST_CLUSTER_NAME, + ClusterData.builder().serviceUrl(pulsarService.getWebServiceAddress()).build()); + } + if (!serviceInternalAdmin.tenants().getTenants().contains(DEFAULT_TENANT)) { + serviceInternalAdmin.tenants().createTenant(DEFAULT_TENANT, TenantInfo.builder().allowedClusters( + Sets.newHashSet(TEST_CLUSTER_NAME)).build()); + } + if (!serviceInternalAdmin.namespaces().getNamespaces(DEFAULT_TENANT).contains(DEFAULT_NAMESPACE)) { + serviceInternalAdmin.namespaces().createNamespace(DEFAULT_NAMESPACE); + } + } + + + @Override + public void close() throws Exception { + if (pulsarTestContext != null) { + pulsarTestContext.close(); + } + } + + // Utils + protected static final ObjectMapper mapper = new ObjectMapper(); + + // Static name + private static final String DEFAULT_TENANT = "public"; + private static final String DEFAULT_NAMESPACE = "public/default"; + private static final String TEST_CLUSTER_NAME = "test-standalone"; + + private static final ObjectMapper MAPPER = ObjectMapperFactory.getMapper().getObjectMapper(); +} From 1aa48ab68818b02f0a4af02291c50e6b9afbc89b Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Mon, 4 Mar 2024 11:40:36 +0530 Subject: [PATCH 296/699] Fix for Github Broker Group Tests --- .../pulsar/broker/auth/MockedPulsarServiceBaseTest.java | 4 ---- 1 file changed, 4 deletions(-) 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 972885bb31275..28d66af8b265b 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 @@ -233,10 +233,8 @@ protected void doInitConf() throws Exception { this.conf.setBrokerShutdownTimeoutMs(0L); this.conf.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); this.conf.setBrokerServicePort(Optional.of(0)); - this.conf.setBrokerServicePortTls(Optional.of(0)); this.conf.setAdvertisedAddress("localhost"); this.conf.setWebServicePort(Optional.of(0)); - this.conf.setWebServicePortTls(Optional.of(0)); this.conf.setNumExecutorThreadPoolSize(5); this.conf.setExposeBundlesMetricsInPrometheus(true); } @@ -492,9 +490,7 @@ protected ServiceConfiguration getDefaultConf() { configuration.setBrokerShutdownTimeoutMs(0L); configuration.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); configuration.setBrokerServicePort(Optional.of(0)); - configuration.setBrokerServicePortTls(Optional.of(0)); configuration.setWebServicePort(Optional.of(0)); - configuration.setWebServicePortTls(Optional.of(0)); configuration.setBookkeeperClientExposeStatsToPrometheus(true); configuration.setNumExecutorThreadPoolSize(5); configuration.setBrokerMaxConnections(0); From 6d10ef3445f980f6b3acdbb418809c1e9c6869df Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Mon, 4 Mar 2024 12:25:37 +0530 Subject: [PATCH 297/699] Fix for ClientAuthenticationTlsTest & ServiceUnitStateChannelTest --- .../extensions/channel/ServiceUnitStateChannelTest.java | 4 ++-- .../apache/pulsar/client/api/ClientAuthenticationTlsTest.java | 4 +++- 2 files changed, 5 insertions(+), 3 deletions(-) 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 860e892ed6199..ceb58e8d9647c 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 @@ -617,7 +617,7 @@ public void splitAndRetryTest() throws Exception { var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; validateMonitorCounters(leader, 0, - 3, + 1, 0, 0, 0, @@ -1461,7 +1461,7 @@ public void splitAndRetryFailureTest() throws Exception { validateMonitorCounters(leader, 0, - 3, + 1, 1, 0, 0, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientAuthenticationTlsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientAuthenticationTlsTest.java index c9b243257c4e1..d716d5a806392 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientAuthenticationTlsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ClientAuthenticationTlsTest.java @@ -22,6 +22,7 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.expectThrows; import java.util.HashSet; +import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; @@ -50,7 +51,8 @@ protected void doInitConf() throws Exception { Set providers = new HashSet<>(); providers.add(AuthenticationProviderTls.class.getName()); conf.setAuthenticationProviders(providers); - + conf.setWebServicePortTls(Optional.of(0)); + conf.setBrokerServicePortTls(Optional.of(0)); conf.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); conf.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); conf.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); From 1c861a1fade3f22b2e46c2a878ba1b3fe157cf60 Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Mon, 4 Mar 2024 13:39:02 +0530 Subject: [PATCH 298/699] Fix for PulsarServiceTest --- .../test/java/org/apache/pulsar/broker/PulsarServiceTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java index 37a7310ae17ca..3e0887646e119 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java @@ -54,6 +54,8 @@ protected void cleanup() throws Exception { @Override protected void doInitConf() throws Exception { super.doInitConf(); + conf.setBrokerServicePortTls(Optional.of(0)); + conf.setWebServicePortTls(Optional.of(0)); if (useStaticPorts) { conf.setBrokerServicePortTls(Optional.of(6651)); conf.setBrokerServicePort(Optional.of(6660)); From 24110fad2c4358039a4634ce9151bd5bfd99ed10 Mon Sep 17 00:00:00 2001 From: feynmanlin <315157973@qq.com> Date: Thu, 22 Feb 2024 12:09:24 +0800 Subject: [PATCH 299/699] [fix] [broker] Expire messages according to ledger close time to avoid client clock skew (#21940) (cherry picked from commit 861618a8120901a4042e99243d8fa539449d7f60) (cherry picked from commit fee3a33541646844fdbc9e6d06a921fdfd1a9044) --- .../PersistentMessageExpiryMonitor.java | 36 ++++++++++++++++++- .../service/PersistentMessageFinderTest.java | 28 ++++++++++++++- 2 files changed, 62 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java index 978cd3f886f16..ea45e9536a731 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java @@ -20,6 +20,7 @@ import java.util.Objects; import java.util.Optional; +import java.util.SortedMap; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.concurrent.atomic.LongAdder; import javax.annotation.Nullable; @@ -30,8 +31,10 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.LedgerNotExistException; import org.apache.bookkeeper.mledger.ManagedLedgerException.NonRecoverableLedgerException; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.pulsar.broker.service.MessageExpirer; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType; @@ -78,7 +81,9 @@ public boolean expireMessages(int messageTTLInSeconds) { if (expirationCheckInProgressUpdater.compareAndSet(this, FALSE, TRUE)) { log.info("[{}][{}] Starting message expiry check, ttl= {} seconds", topicName, subName, messageTTLInSeconds); - + // First filter the entire Ledger reached TTL based on the Ledger closing time to avoid client clock skew + checkExpiryByLedgerClosureTime(cursor, messageTTLInSeconds); + // Some part of entries in active Ledger may have reached TTL, so we need to continue searching. cursor.asyncFindNewestMatching(ManagedCursor.FindPositionConstraint.SearchActiveEntries, entry -> { try { long entryTimestamp = Commands.getEntryTimestamp(entry.getDataBuffer()); @@ -100,6 +105,35 @@ public boolean expireMessages(int messageTTLInSeconds) { } } + private void checkExpiryByLedgerClosureTime(ManagedCursor cursor, int messageTTLInSeconds) { + if (messageTTLInSeconds <= 0) { + return; + } + if (cursor instanceof ManagedCursorImpl managedCursor) { + ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) managedCursor.getManagedLedger(); + Position deletedPosition = managedCursor.getMarkDeletedPosition(); + SortedMap ledgerInfoSortedMap = + managedLedger.getLedgersInfo().subMap(deletedPosition.getLedgerId(), true, + managedLedger.getLedgersInfo().lastKey(), true); + MLDataFormats.ManagedLedgerInfo.LedgerInfo info = null; + for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo : ledgerInfoSortedMap.values()) { + if (!ledgerInfo.hasTimestamp() || !MessageImpl.isEntryExpired(messageTTLInSeconds, + ledgerInfo.getTimestamp())) { + break; + } + info = ledgerInfo; + } + if (info != null && info.getLedgerId() > -1) { + PositionImpl position = PositionImpl.get(info.getLedgerId(), info.getEntries() - 1); + if (((PositionImpl) managedLedger.getLastConfirmedEntry()).compareTo(position) < 0) { + findEntryComplete(managedLedger.getLastConfirmedEntry(), null); + } else { + findEntryComplete(position, null); + } + } + } + } + @Override public boolean expireMessages(Position messagePosition) { // If it's beyond last position of this topic, do nothing. 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 f0e2e6eafcdfb..ace552a55a72a 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 @@ -81,8 +81,11 @@ public class PersistentMessageFinderTest extends MockedBookKeeperTestCase { public static byte[] createMessageWrittenToLedger(String msg) { + return createMessageWrittenToLedger(msg, System.currentTimeMillis()); + } + public static byte[] createMessageWrittenToLedger(String msg, long messageTimestamp) { MessageMetadata messageMetadata = new MessageMetadata() - .setPublishTime(System.currentTimeMillis()) + .setPublishTime(messageTimestamp) .setProducerName("createMessageWrittenToLedger") .setSequenceId(1); ByteBuf data = UnpooledByteBufAllocator.DEFAULT.heapBuffer().writeBytes(msg.getBytes()); @@ -437,6 +440,29 @@ void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { } + @Test + public void testIncorrectClientClock() throws Exception { + final String ledgerAndCursorName = "testIncorrectClientClock"; + int maxTTLSeconds = 1; + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setMaxEntriesPerLedger(1); + ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open(ledgerAndCursorName, config); + ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor(ledgerAndCursorName); + // set client clock to 10 days later + long incorrectPublishTimestamp = System.currentTimeMillis() + TimeUnit.DAYS.toMillis(10); + for (int i = 0; i < 10; i++) { + ledger.addEntry(createMessageWrittenToLedger("msg" + i, incorrectPublishTimestamp)); + } + assertEquals(ledger.getLedgersInfoAsList().size(), 10); + PersistentTopic mock = mock(PersistentTopic.class); + when(mock.getName()).thenReturn("topicname"); + when(mock.getLastPosition()).thenReturn(PositionImpl.EARLIEST); + PersistentMessageExpiryMonitor monitor = new PersistentMessageExpiryMonitor(mock, c1.getName(), c1, null); + Thread.sleep(TimeUnit.SECONDS.toMillis(maxTTLSeconds)); + monitor.expireMessages(maxTTLSeconds); + assertEquals(c1.getNumberOfEntriesInBacklog(true), 0); + } + @Test void testMessageExpiryWithPosition() throws Exception { final String ledgerAndCursorName = "testPersistentMessageExpiryWithPositionNonRecoverableLedgers"; From 4efd40a3f4767ccfaeb9dc50502c9b99f5418108 Mon Sep 17 00:00:00 2001 From: jiangpengcheng Date: Wed, 31 Jan 2024 22:18:47 +0800 Subject: [PATCH 300/699] [fix][fn] Use unified PackageManagement service to download packages (#21955) (cherry picked from commit d3b7ca5e964216bc52c436ebeb88357a0aed5948) (cherry picked from commit d58d5983da3f506c6ef186b684558dd41c22cba9) --- .../apache/pulsar/functions/worker/rest/api/ComponentImpl.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java index f80bc31e7f96c..5e105f7057e33 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java @@ -1817,6 +1817,8 @@ protected File getPackageFile(String functionPkgUrl, String existingPackagePath, + "when getting %s package from %s", e.getMessage(), ComponentTypeUtils.toString(componentType), functionPkgUrl)); } + } else if (Utils.hasPackageTypePrefix(existingPackagePath)) { + componentPackageFile = getPackageFile(existingPackagePath); } else if (uploadedInputStream != null) { componentPackageFile = WorkerUtils.dumpToTmpFile(uploadedInputStream); } else if (!existingPackagePath.startsWith(Utils.BUILTIN)) { From c9c4f65f3d3696365f5a47b90dbf2fc898bf8b5e Mon Sep 17 00:00:00 2001 From: atomchen <492672043@qq.com> Date: Sun, 18 Feb 2024 15:51:49 +0800 Subject: [PATCH 301/699] =?UTF-8?q?[fix][broker]Support=20setting=20`autoS?= =?UTF-8?q?kipNonRecoverableData`=20dynamically=20in=20expiryMon=E2=80=A6?= =?UTF-8?q?=20(#21991)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: atomchchen (cherry picked from commit 220a3d601602d67f5f44516c5d9895dfaa270380) (cherry picked from commit 09cb54148c1db3b3f18dd8991c59b788c975d6b3) --- .../PersistentMessageExpiryMonitor.java | 10 ++++--- .../persistent/PersistentTopicTest.java | 26 +++++++++++++++++++ 2 files changed, 33 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java index ea45e9536a731..ac391c1050340 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentMessageExpiryMonitor.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service.persistent; +import com.google.common.annotations.VisibleForTesting; import java.util.Objects; import java.util.Optional; import java.util.SortedMap; @@ -51,7 +52,6 @@ public class PersistentMessageExpiryMonitor implements FindEntryCallback, Messag private final String topicName; private final Rate msgExpired; private final LongAdder totalMsgExpired; - private final boolean autoSkipNonRecoverableData; private final PersistentSubscription subscription; private static final int FALSE = 0; @@ -71,8 +71,12 @@ public PersistentMessageExpiryMonitor(PersistentTopic topic, String subscription this.subscription = subscription; this.msgExpired = new Rate(); this.totalMsgExpired = new LongAdder(); + } + + @VisibleForTesting + public boolean isAutoSkipNonRecoverableData() { // check to avoid test failures - this.autoSkipNonRecoverableData = this.cursor.getManagedLedger() != null + return this.cursor.getManagedLedger() != null && this.cursor.getManagedLedger().getConfig().isAutoSkipNonRecoverableData(); } @@ -230,7 +234,7 @@ public void findEntryFailed(ManagedLedgerException exception, Optional if (log.isDebugEnabled()) { log.debug("[{}][{}] Finding expired entry operation failed", topicName, subName, exception); } - if (autoSkipNonRecoverableData && failedReadPosition.isPresent() + if (isAutoSkipNonRecoverableData() && failedReadPosition.isPresent() && (exception instanceof NonRecoverableLedgerException)) { log.warn("[{}][{}] read failed from ledger at position:{} : {}", topicName, subName, failedReadPosition, exception.getMessage()); 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 fe84aeb1df792..717dfc28ac884 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 @@ -635,4 +635,30 @@ public void testCheckPersistencePolicies() throws Exception { assertEquals(persistentTopic.getManagedLedger().getConfig().getRetentionSizeInMB(), 1L); assertEquals(persistentTopic.getManagedLedger().getConfig().getRetentionTimeMillis(), TimeUnit.MINUTES.toMillis(1)); } + + @Test + public void testDynamicConfigurationAutoSkipNonRecoverableData() throws Exception { + pulsar.getConfiguration().setAutoSkipNonRecoverableData(false); + final String topicName = "persistent://prop/ns-abc/testAutoSkipNonRecoverableData"; + final String subName = "test_sub"; + + Consumer subscribe = pulsarClient.newConsumer().topic(topicName).subscriptionName(subName).subscribe(); + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).join().get(); + PersistentSubscription subscription = persistentTopic.getSubscription(subName); + + assertFalse(persistentTopic.ledger.getConfig().isAutoSkipNonRecoverableData()); + assertFalse(subscription.getExpiryMonitor().isAutoSkipNonRecoverableData()); + + String key = "autoSkipNonRecoverableData"; + admin.brokers().updateDynamicConfiguration(key, "true"); + Awaitility.await() + .untilAsserted(() -> assertEquals(admin.brokers().getAllDynamicConfigurations().get(key), "true")); + + assertTrue(persistentTopic.ledger.getConfig().isAutoSkipNonRecoverableData()); + assertTrue(subscription.getExpiryMonitor().isAutoSkipNonRecoverableData()); + + subscribe.close(); + admin.topics().delete(topicName); + } } From 27d4275bb3e4f94309250c8a830e9d6cdc1147ad Mon Sep 17 00:00:00 2001 From: zifengmo <38554710+zifengmo@users.noreply.github.com> Date: Mon, 26 Feb 2024 18:06:28 +0800 Subject: [PATCH 302/699] [fix] [client] Do no retrying for error subscription not found when disabled allowAutoSubscriptionCreation (#22078) (cherry picked from commit bbf6ddf9244c1dc02cc157a014c153a09f45be1f) (cherry picked from commit 7eec36e743fd3d70addce777a36f17f9b45fa5e9) --- .../service/BrokerServiceException.java | 2 ++ .../client/api/MultiTopicsConsumerTest.java | 32 +++++++++++++++++++ .../client/api/PulsarClientException.java | 17 ++++++++++ .../apache/pulsar/client/impl/ClientCnx.java | 2 ++ 4 files changed, 53 insertions(+) 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 3e77588b2459f..831d6068e2097 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 @@ -258,6 +258,8 @@ private static ServerError getClientErrorCode(Throwable t, boolean checkCauseIfU return ServerError.ServiceNotReady; } else if (t instanceof TopicNotFoundException) { return ServerError.TopicNotFound; + } else if (t instanceof SubscriptionNotFoundException) { + return ServerError.SubscriptionNotFound; } else if (t instanceof IncompatibleSchemaException || t instanceof InvalidSchemaDataException) { // for backward compatible with old clients, invalid schema data 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 315ce378d6953..bb8bab29ad9ef 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 @@ -371,4 +371,36 @@ public void testMultipleIOThreads() throws PulsarAdminException, PulsarClientExc assertTrue(consumer instanceof MultiTopicsConsumerImpl); assertTrue(consumer.isConnected()); } + + @Test(timeOut = 30000) + public void testSubscriptionNotFound() throws PulsarAdminException, PulsarClientException { + final var topic1 = newTopicName(); + final var topic2 = newTopicName(); + + pulsar.getConfiguration().setAllowAutoSubscriptionCreation(false); + + try { + final var singleTopicConsumer = pulsarClient.newConsumer() + .topic(topic1) + .subscriptionName("sub-1") + .isAckReceiptEnabled(true) + .subscribe(); + assertTrue(singleTopicConsumer instanceof ConsumerImpl); + } catch (Throwable t) { + assertTrue(t.getCause().getCause() instanceof PulsarClientException.SubscriptionNotFoundException); + } + + try { + final var multiTopicsConsumer = pulsarClient.newConsumer() + .topics(List.of(topic1, topic2)) + .subscriptionName("sub-2") + .isAckReceiptEnabled(true) + .subscribe(); + assertTrue(multiTopicsConsumer instanceof MultiTopicsConsumerImpl); + } catch (Throwable t) { + assertTrue(t.getCause().getCause() instanceof PulsarClientException.SubscriptionNotFoundException); + } + + 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 9409eefe2e0f0..22a97571e532e 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 @@ -344,6 +344,22 @@ public TopicDoesNotExistException(String msg) { } } + /** + * Not found subscription that cannot be created. + */ + public static class SubscriptionNotFoundException extends PulsarClientException { + /** + * Constructs an {@code SubscriptionNotFoundException} with the specified detail message. + * + * @param msg + * The detail message (which is saved for later retrieval + * by the {@link #getMessage()} method) + */ + public SubscriptionNotFoundException(String msg) { + super(msg); + } + } + /** * Lookup exception thrown by Pulsar client. */ @@ -1159,6 +1175,7 @@ public static boolean isRetriableError(Throwable t) { || t instanceof NotFoundException || t instanceof IncompatibleSchemaException || t instanceof TopicDoesNotExistException + || t instanceof SubscriptionNotFoundException || t instanceof UnsupportedAuthenticationException || t instanceof InvalidMessageException || t instanceof InvalidTopicNameException 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 115c71307c4f2..e1b374acbbb54 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 @@ -1291,6 +1291,8 @@ public static PulsarClientException getPulsarClientException(ServerError error, return new PulsarClientException.IncompatibleSchemaException(errorMsg); case TopicNotFound: return new PulsarClientException.TopicDoesNotExistException(errorMsg); + case SubscriptionNotFound: + return new PulsarClientException.SubscriptionNotFoundException(errorMsg); case ConsumerAssignError: return new PulsarClientException.ConsumerAssignException(errorMsg); case NotAllowedError: From ca998b8b2c401a80307a24df4b9a66ca3b9736cb Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Tue, 27 Feb 2024 22:22:10 +0800 Subject: [PATCH 303/699] [fix][test] Fix test testAsyncFunctionMaxPending (#22121) (cherry picked from commit 91de98ad45675c23b79c18e49602e9c49ec880b3) (cherry picked from commit c9ecd5221f494ac858d44871c670dbd6aad1d039) --- .../pulsar/functions/instance/JavaInstanceTest.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) 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 efe80922dfa8c..5a3332042938d 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 @@ -24,6 +24,7 @@ import static org.testng.Assert.assertNull; import static org.testng.Assert.assertSame; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import lombok.Cleanup; @@ -185,6 +186,7 @@ public void testUserExceptionThrowingAsyncFunction() throws Exception { @Test public void testAsyncFunctionMaxPending() throws Exception { + CountDownLatch count = new CountDownLatch(1); InstanceConfig instanceConfig = new InstanceConfig(); int pendingQueueSize = 3; instanceConfig.setMaxPendingAsyncRequests(pendingQueueSize); @@ -196,7 +198,7 @@ public void testAsyncFunctionMaxPending() throws Exception { CompletableFuture result = new CompletableFuture<>(); executor.submit(() -> { try { - Thread.sleep(500); + count.await(); result.complete(String.format("%s-lambda", input)); } catch (Exception e) { result.completeExceptionally(e); @@ -222,8 +224,13 @@ public void testAsyncFunctionMaxPending() throws Exception { // no space left assertEquals(0, instance.getPendingAsyncRequests().remainingCapacity()); + AsyncFuncRequest[] asyncFuncRequests = new AsyncFuncRequest[3]; for (int i = 0; i < 3; i++) { - AsyncFuncRequest request = instance.getPendingAsyncRequests().poll(); + asyncFuncRequests[i] = instance.getPendingAsyncRequests().poll(); + } + + count.countDown(); + for (AsyncFuncRequest request : asyncFuncRequests) { Assert.assertEquals(request.getProcessResult().get(), testString + "-lambda"); } From a8f7008b8deb5fd13cce9ffe3aff479c82ece756 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 28 Feb 2024 11:40:21 +0200 Subject: [PATCH 304/699] [fix][sec] Upgrade Jetty to 9.4.54.v20240208 to address CVE-2024-22201 (#22144) (cherry picked from commit e3a081e4c5ea380eb505751193bc71dd0ae39281) (cherry picked from commit 93a44574fca5e562173b2b0e7e60f568f7e5a934) --- .../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 014fbed905326..715cbec18bbc4 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -382,25 +382,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.53.v20231009.jar - - org.eclipse.jetty-jetty-continuation-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-http-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-io-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-proxy-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-security-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-server-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-servlet-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-servlets-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-util-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-util-ajax-9.4.53.v20231009.jar - - org.eclipse.jetty.websocket-javax-websocket-client-impl-9.4.53.v20231009.jar - - org.eclipse.jetty.websocket-websocket-api-9.4.53.v20231009.jar - - org.eclipse.jetty.websocket-websocket-client-9.4.53.v20231009.jar - - org.eclipse.jetty.websocket-websocket-common-9.4.53.v20231009.jar - - org.eclipse.jetty.websocket-websocket-server-9.4.53.v20231009.jar - - org.eclipse.jetty.websocket-websocket-servlet-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-alpn-conscrypt-server-9.4.53.v20231009.jar - - org.eclipse.jetty-jetty-alpn-server-9.4.53.v20231009.jar + - 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 * 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.5.1.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 31e0e884caa16..5d880a018c53f 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -395,14 +395,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.53.v20231009.jar - - jetty-http-9.4.53.v20231009.jar - - jetty-io-9.4.53.v20231009.jar - - jetty-util-9.4.53.v20231009.jar - - javax-websocket-client-impl-9.4.53.v20231009.jar - - websocket-api-9.4.53.v20231009.jar - - websocket-client-9.4.53.v20231009.jar - - websocket-common-9.4.53.v20231009.jar + - 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 * SnakeYaml -- snakeyaml-2.0.jar * Google Error Prone Annotations - error_prone_annotations-2.5.1.jar * Javassist -- javassist-3.25.0-GA.jar diff --git a/pom.xml b/pom.xml index f721c7542286b..1db2ca3e23f22 100644 --- a/pom.xml +++ b/pom.xml @@ -142,7 +142,7 @@ flexible messaging model and an intuitive client API. 5.1.0 4.1.100.Final 0.0.21.Final - 9.4.53.v20231009 + 9.4.54.v20240208 2.5.2 2.34 1.10.50 From 0224e0da5d230a0892ffb57da573fb4d92283cb6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Thu, 29 Feb 2024 21:22:03 +0800 Subject: [PATCH 305/699] [fix][txn]Fix TopicTransactionBuffer potential thread safety issue (#22149) (cherry picked from commit 74be3fd4917a2327f2da9b5b55cc572b3c1f4e84) (cherry picked from commit c6b4887124bb5dee9edc7a85448dc341c3ff41a6) --- .../buffer/impl/TopicTransactionBuffer.java | 30 +++++++++++-------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java index 5392e473947e6..a36216bd6258b 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 @@ -170,13 +170,15 @@ public void handleTxnEntry(Entry entry) { if (msgMetadata != null && msgMetadata.hasTxnidMostBits() && msgMetadata.hasTxnidLeastBits()) { TxnID txnID = new TxnID(msgMetadata.getTxnidMostBits(), msgMetadata.getTxnidLeastBits()); PositionImpl position = PositionImpl.get(entry.getLedgerId(), entry.getEntryId()); - if (Markers.isTxnMarker(msgMetadata)) { - if (Markers.isTxnAbortMarker(msgMetadata)) { - snapshotAbortedTxnProcessor.putAbortedTxnAndPosition(txnID, position); + synchronized (TopicTransactionBuffer.this) { + if (Markers.isTxnMarker(msgMetadata)) { + if (Markers.isTxnAbortMarker(msgMetadata)) { + snapshotAbortedTxnProcessor.putAbortedTxnAndPosition(txnID, position); + } + updateMaxReadPosition(txnID); + } else { + handleTransactionMessage(txnID, position); } - updateMaxReadPosition(txnID); - } else { - handleTransactionMessage(txnID, position); } } } @@ -362,10 +364,10 @@ public void addComplete(Position position, ByteBuf entryData, Object ctx) { updateMaxReadPosition(txnID); snapshotAbortedTxnProcessor.trimExpiredAbortedTxns(); takeSnapshotByChangeTimes(); + txnAbortedCounter.increment(); + completableFuture.complete(null); + handleLowWaterMark(txnID, lowWaterMark); } - txnAbortedCounter.increment(); - completableFuture.complete(null); - handleLowWaterMark(txnID, lowWaterMark); } @Override @@ -473,7 +475,7 @@ public CompletableFuture closeAsync() { } @Override - public boolean isTxnAborted(TxnID txnID, PositionImpl readPosition) { + public synchronized boolean isTxnAborted(TxnID txnID, PositionImpl readPosition) { return snapshotAbortedTxnProcessor.checkAbortedTransaction(txnID); } @@ -510,9 +512,11 @@ public PositionImpl getMaxReadPosition() { @Override public TransactionInBufferStats getTransactionInBufferStats(TxnID txnID) { TransactionInBufferStats transactionInBufferStats = new TransactionInBufferStats(); - transactionInBufferStats.aborted = isTxnAborted(txnID, null); - if (ongoingTxns.containsKey(txnID)) { - transactionInBufferStats.startPosition = ongoingTxns.get(txnID).toString(); + synchronized (this) { + transactionInBufferStats.aborted = isTxnAborted(txnID, null); + if (ongoingTxns.containsKey(txnID)) { + transactionInBufferStats.startPosition = ongoingTxns.get(txnID).toString(); + } } return transactionInBufferStats; } From 80da36e9530ec2102e5a33f6db459ead8d99db09 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 29 Feb 2024 12:47:54 +0800 Subject: [PATCH 306/699] [fix] [broker] print non log when delete partitioned topic failed (#22153) (cherry picked from commit 72cedb7020c75ada0d26b8120e55e0bec4467f13) (cherry picked from commit bbbd1ef5314049b15957737808a29e61160aac0d) --- .../pulsar/broker/service/persistent/PersistentTopic.java | 6 ++++-- 1 file changed, 4 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 12f0f738bb816..2752f24785324 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 @@ -2688,7 +2688,7 @@ public void checkGC() { replCloseFuture.thenCompose(v -> delete(deleteMode == InactiveTopicDeleteMode.delete_when_no_subscriptions, deleteMode == InactiveTopicDeleteMode.delete_when_subscriptions_caught_up, false)) - .thenApply((res) -> tryToDeletePartitionedMetadata()) + .thenCompose((res) -> tryToDeletePartitionedMetadata()) .thenRun(() -> log.info("[{}] Topic deleted successfully due to inactivity", topic)) .exceptionally(e -> { if (e.getCause() instanceof TopicBusyException) { @@ -2696,6 +2696,8 @@ public void checkGC() { if (log.isDebugEnabled()) { log.debug("[{}] Did not delete busy topic: {}", topic, e.getCause().getMessage()); } + } else if (e.getCause() instanceof UnsupportedOperationException) { + log.info("[{}] Skip to delete partitioned topic: {}", topic, e.getCause().getMessage()); } else { log.warn("[{}] Inactive topic deletion failed", topic, e); } @@ -2740,7 +2742,7 @@ private CompletableFuture tryToDeletePartitionedMetadata() { .filter(topicExist -> topicExist) .findAny(); if (anyExistPartition.isPresent()) { - log.error("[{}] Delete topic metadata failed because" + log.info("[{}] Delete topic metadata failed because" + " another partition exist.", topicName); throw new UnsupportedOperationException( String.format("Another partition exists for [%s].", From 127a6f060a27662e1b87c0ad416a3d9ad73b4440 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Thu, 29 Feb 2024 21:03:47 +0800 Subject: [PATCH 307/699] [fix][offload] Fix Offload readHandle cannot close multi times. (#22162) (cherry picked from commit e25c7f045753b949c5ecd492bd7b7a77440c6937) (cherry picked from commit 62de4a5639becfb2e0076c86ab569bdb8d03fbcb) --- .../impl/FileStoreBackedReadHandleImpl.java | 36 ++++++++++++++----- .../impl/BlobStoreBackedReadHandleImpl.java | 32 ++++++++++------- .../impl/BlobStoreBackedReadHandleImplV2.java | 14 ++++++-- 3 files changed, 59 insertions(+), 23 deletions(-) diff --git a/tiered-storage/file-system/src/main/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileStoreBackedReadHandleImpl.java b/tiered-storage/file-system/src/main/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileStoreBackedReadHandleImpl.java index 49b2071f5db2c..91e7e902eab8a 100644 --- a/tiered-storage/file-system/src/main/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileStoreBackedReadHandleImpl.java +++ b/tiered-storage/file-system/src/main/java/org/apache/bookkeeper/mledger/offload/filesystem/impl/FileStoreBackedReadHandleImpl.java @@ -27,6 +27,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.api.LastConfirmedAndEntry; import org.apache.bookkeeper.client.api.LedgerEntries; @@ -36,6 +37,7 @@ import org.apache.bookkeeper.client.impl.LedgerEntriesImpl; import org.apache.bookkeeper.client.impl.LedgerEntryImpl; import org.apache.bookkeeper.mledger.LedgerOffloaderStats; +import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.MapFile; @@ -53,6 +55,12 @@ public class FileStoreBackedReadHandleImpl implements ReadHandle { private final LedgerOffloaderStats offloaderStats; private final String managedLedgerName; private final String topicName; + enum State { + Opened, + Closed + } + private volatile State state; + private final AtomicReference> closeFuture = new AtomicReference<>(); private FileStoreBackedReadHandleImpl(ExecutorService executor, MapFile.Reader reader, long ledgerId, LedgerOffloaderStats offloaderStats, @@ -72,6 +80,7 @@ private FileStoreBackedReadHandleImpl(ExecutorService executor, MapFile.Reader r offloaderStats.recordReadOffloadIndexLatency(topicName, System.nanoTime() - startReadIndexTime, TimeUnit.NANOSECONDS); this.ledgerMetadata = parseLedgerMetadata(ledgerId, value.copyBytes()); + state = State.Opened; } catch (IOException e) { log.error("Fail to read LedgerMetadata for ledgerId {}", ledgerId); @@ -92,15 +101,20 @@ public LedgerMetadata getLedgerMetadata() { @Override public CompletableFuture closeAsync() { - CompletableFuture promise = new CompletableFuture<>(); + if (closeFuture.get() != null || !closeFuture.compareAndSet(null, new CompletableFuture<>())) { + return closeFuture.get(); + } + + CompletableFuture promise = closeFuture.get(); executor.execute(() -> { - try { - reader.close(); - promise.complete(null); - } catch (IOException t) { - promise.completeExceptionally(t); - } - }); + try { + reader.close(); + state = State.Closed; + promise.complete(null); + } catch (IOException t) { + promise.completeExceptionally(t); + } + }); return promise; } @@ -111,6 +125,12 @@ public CompletableFuture readAsync(long firstEntry, long lastEntr } CompletableFuture promise = new CompletableFuture<>(); executor.execute(() -> { + if (state == State.Closed) { + log.warn("Reading a closed read handler. Ledger ID: {}, Read range: {}-{}", + ledgerId, firstEntry, lastEntry); + promise.completeExceptionally(new ManagedLedgerException.OffloadReadHandleClosedException()); + return; + } if (firstEntry > lastEntry || firstEntry < 0 || lastEntry > getLastAddConfirmed()) { diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java index 46bfc8d5bdc1f..ef39f7f50d07e 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java @@ -18,6 +18,7 @@ */ package org.apache.bookkeeper.mledger.offload.jcloud.impl; +import com.google.common.annotations.VisibleForTesting; import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; import io.netty.buffer.ByteBuf; @@ -30,6 +31,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.api.LastConfirmedAndEntry; import org.apache.bookkeeper.client.api.LedgerEntries; @@ -67,13 +69,14 @@ public class BlobStoreBackedReadHandleImpl implements ReadHandle, OffloadedLedge .newBuilder() .expireAfterAccess(CACHE_TTL_SECONDS, TimeUnit.SECONDS) .build(); + private final AtomicReference> closeFuture = new AtomicReference<>(); enum State { Opened, Closed } - private State state = null; + private volatile State state = null; private long lastAccessTimestamp = System.currentTimeMillis(); @@ -99,18 +102,22 @@ public LedgerMetadata getLedgerMetadata() { @Override public CompletableFuture closeAsync() { - CompletableFuture promise = new CompletableFuture<>(); + if (closeFuture.get() != null || !closeFuture.compareAndSet(null, new CompletableFuture<>())) { + return closeFuture.get(); + } + + CompletableFuture promise = closeFuture.get(); executor.execute(() -> { - try { - index.close(); - inputStream.close(); - entryOffsets.invalidateAll(); - state = State.Closed; - promise.complete(null); - } catch (IOException t) { - promise.completeExceptionally(t); - } - }); + try { + index.close(); + inputStream.close(); + entryOffsets.invalidateAll(); + state = State.Closed; + promise.complete(null); + } catch (IOException t) { + promise.completeExceptionally(t); + } + }); return promise; } @@ -303,6 +310,7 @@ public static ReadHandle open(ScheduledExecutorService executor, } // for testing + @VisibleForTesting State getState() { return this.state; } diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImplV2.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImplV2.java index e40a0a3834c85..53d96e08abf5e 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImplV2.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImplV2.java @@ -30,6 +30,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import lombok.val; import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.api.LastConfirmedAndEntry; @@ -60,7 +61,8 @@ public class BlobStoreBackedReadHandleImplV2 implements ReadHandle { private final List inputStreams; private final List dataStreams; private final ExecutorService executor; - private State state = null; + private volatile State state = null; + private final AtomicReference> closeFuture = new AtomicReference<>(); enum State { Opened, @@ -123,7 +125,11 @@ public LedgerMetadata getLedgerMetadata() { @Override public CompletableFuture closeAsync() { - CompletableFuture promise = new CompletableFuture<>(); + if (closeFuture.get() != null || !closeFuture.compareAndSet(null, new CompletableFuture<>())) { + return closeFuture.get(); + } + + CompletableFuture promise = closeFuture.get(); executor.execute(() -> { try { for (OffloadIndexBlockV2 indexBlock : indices) { @@ -143,7 +149,9 @@ public CompletableFuture closeAsync() { @Override public CompletableFuture readAsync(long firstEntry, long lastEntry) { - log.debug("Ledger {}: reading {} - {}", getId(), firstEntry, lastEntry); + if (log.isDebugEnabled()) { + log.debug("Ledger {}: reading {} - {}", getId(), firstEntry, lastEntry); + } CompletableFuture promise = new CompletableFuture<>(); executor.execute(() -> { if (state == State.Closed) { From 65e8cacf89f990530569aba183fa1c81831363fd Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Mon, 4 Mar 2024 17:43:57 +0530 Subject: [PATCH 308/699] Fix for ClusterMigrationTest & ProxyServiceTlsStarterTest --- .../pulsar/broker/service/ClusterMigrationTest.java | 9 +++++++++ .../pulsar/proxy/server/ProxyServiceTlsStarterTest.java | 1 + 2 files changed, 10 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 c4f2cd047aff6..a2dcf3c9c0b4d 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 @@ -28,6 +28,7 @@ import com.google.common.collect.Sets; import java.lang.reflect.Method; import java.net.URL; +import java.util.Optional; import java.util.concurrent.TimeUnit; import lombok.Cleanup; import org.apache.pulsar.broker.BrokerTestUtil; @@ -477,6 +478,14 @@ protected void setup() throws Exception { super.setupWithClusterName(clusterName); } + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + this.conf.setWebServicePortTls(Optional.of(0)); + this.conf.setBrokerServicePortTls(Optional.of(0)); + } + + public PulsarService getPulsarService() { return pulsar; } 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 01c06fbf52f4e..6247c2a66e874 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 @@ -75,6 +75,7 @@ protected void setup() throws Exception { protected void doInitConf() throws Exception { super.doInitConf(); + this.conf.setBrokerServicePortTls(Optional.of(0)); this.conf.setTlsCertificateFilePath(PROXY_CERT_FILE_PATH); this.conf.setTlsKeyFilePath(PROXY_KEY_FILE_PATH); } From 5a2f2cbdd1855734bb1faa27fc3cba07b651b890 Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Mon, 4 Mar 2024 18:14:11 +0530 Subject: [PATCH 309/699] Fix for ProxyRefreshAuthTest --- .../org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java index bde989fc432f9..2f36cc679f1f2 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java @@ -69,6 +69,7 @@ protected void doInitConf() throws Exception { conf.setAdvertisedAddress(null); conf.setAuthenticateOriginalAuthData(true); conf.setBrokerServicePort(Optional.of(0)); + conf.setWebServicePortTls(Optional.of(0)); conf.setWebServicePort(Optional.of(0)); Set superUserRoles = new HashSet<>(); From 116e5edc85c92dfcc513e454b23c609a02ff6928 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 4 Mar 2024 18:12:08 +0200 Subject: [PATCH 310/699] [improve][fn] Add configuration for connector & functions package url sources (#22184) (cherry picked from commit 207335a449f2bc9cdf6782c67f93f8c2fb267271) (cherry picked from commit b183492bb03c023983ea7a5354c53c6c97bf62fd) --- conf/functions_worker.yml | 8 ++ .../worker/PulsarFunctionE2ESecurityTest.java | 6 ++ .../worker/PulsarFunctionPublishTest.java | 4 + .../worker/PulsarFunctionTlsTest.java | 9 +- .../pulsar/io/AbstractPulsarE2ETest.java | 6 ++ .../pulsar/functions/worker/WorkerConfig.java | 24 +++++ .../functions/worker/FunctionActioner.java | 16 ++- .../worker/FunctionRuntimeManager.java | 3 +- .../functions/worker/PackageUrlValidator.java | 101 ++++++++++++++++++ .../functions/worker/PulsarWorkerService.java | 4 +- .../worker/rest/api/ComponentImpl.java | 32 ++++-- .../worker/rest/api/FunctionsImpl.java | 3 +- .../functions/worker/rest/api/SinksImpl.java | 9 +- .../worker/rest/api/SourcesImpl.java | 3 +- .../worker/FunctionActionerTest.java | 17 ++- .../worker/FunctionRuntimeManagerTest.java | 2 +- .../v3/AbstractFunctionApiResourceTest.java | 7 ++ .../api/v3/AbstractFunctionsResourceTest.java | 18 +++- .../rest/api/v3/SinkApiV3ResourceTest.java | 56 +++++----- .../rest/api/v3/SourceApiV3ResourceTest.java | 9 ++ 20 files changed, 282 insertions(+), 55 deletions(-) create mode 100644 pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PackageUrlValidator.java diff --git a/conf/functions_worker.yml b/conf/functions_worker.yml index 8c62536971990..3871c74a88778 100644 --- a/conf/functions_worker.yml +++ b/conf/functions_worker.yml @@ -408,7 +408,15 @@ saslJaasServerRoleTokenSignerSecretPath: ######################## connectorsDirectory: ./connectors +# Whether to enable referencing connectors directory files by file url in connector (sink/source) creation +enableReferencingConnectorDirectoryFiles: true +# Regex patterns for enabling creation of connectors by referencing packages in matching http/https urls +additionalEnabledConnectorUrlPatterns: [] functionsDirectory: ./functions +# Whether to enable referencing functions directory files by file url in functions creation +enableReferencingFunctionsDirectoryFiles: true +# Regex patterns for enabling creation of functions by referencing packages in matching http/https urls +additionalEnabledFunctionsUrlPatterns: [] # Enables extended validation for connector config with fine-grain annotation based validation # during submission. Classloading with either enableClassloadingOfExternalFiles or diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java index 107aedd076691..cbf2f28b0b50b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionE2ESecurityTest.java @@ -34,6 +34,7 @@ import java.net.URL; import java.util.Collections; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Properties; @@ -267,6 +268,11 @@ private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration conf workerConfig.setAuthorizationEnabled(config.isAuthorizationEnabled()); workerConfig.setAuthorizationProvider(config.getAuthorizationProvider()); + List urlPatterns = + List.of(getPulsarApiExamplesJar().getParentFile().toURI() + ".*", "http://127\\.0\\.0\\.1:.*"); + workerConfig.setAdditionalEnabledConnectorUrlPatterns(urlPatterns); + workerConfig.setAdditionalEnabledFunctionsUrlPatterns(urlPatterns); + PulsarWorkerService workerService = new PulsarWorkerService(); return workerService; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java index 7bcf1dec871e0..72e6a3766aeee 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionPublishTest.java @@ -268,6 +268,10 @@ private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration conf workerConfig.setAuthenticationEnabled(true); workerConfig.setAuthorizationEnabled(true); + List urlPatterns = List.of(getPulsarApiExamplesJar().getParentFile().toURI() + ".*"); + workerConfig.setAdditionalEnabledConnectorUrlPatterns(urlPatterns); + workerConfig.setAdditionalEnabledFunctionsUrlPatterns(urlPatterns); + PulsarWorkerService workerService = new PulsarWorkerService(); return workerService; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java index 9882b15450e40..3508cf0bfc7e6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/functions/worker/PulsarFunctionTlsTest.java @@ -20,8 +20,8 @@ import static org.apache.pulsar.common.util.PortManager.nextLockedFreePort; import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertTrue; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Sets; @@ -32,6 +32,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -153,6 +154,12 @@ void setup() throws Exception { workerConfig.setUseTls(true); workerConfig.setTlsEnableHostnameVerification(true); workerConfig.setTlsAllowInsecureConnection(false); + File packagePath = new File( + PulsarSink.class.getProtectionDomain().getCodeSource().getLocation().getPath()).getParentFile(); + List urlPatterns = + List.of(packagePath.toURI() + ".*"); + workerConfig.setAdditionalEnabledConnectorUrlPatterns(urlPatterns); + workerConfig.setAdditionalEnabledFunctionsUrlPatterns(urlPatterns); fnWorkerServices[i] = WorkerServiceLoader.load(workerConfig); configurations[i] = config; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java b/pulsar-broker/src/test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java index f968315a7124c..3c0dd0822b7dc 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/io/AbstractPulsarE2ETest.java @@ -35,6 +35,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -306,6 +307,11 @@ private PulsarWorkerService createPulsarFunctionWorker(ServiceConfiguration conf workerConfig.setAuthenticationEnabled(true); workerConfig.setAuthorizationEnabled(true); + List urlPatterns = + List.of(getPulsarApiExamplesJar().getParentFile().toURI() + ".*", "http://127\\.0\\.0\\.1:.*"); + workerConfig.setAdditionalEnabledConnectorUrlPatterns(urlPatterns); + workerConfig.setAdditionalEnabledFunctionsUrlPatterns(urlPatterns); + PulsarWorkerService workerService = new PulsarWorkerService(); return workerService; } diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java index 2d9698103fa0f..ec0e620d0ae8b 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java @@ -264,6 +264,18 @@ public class WorkerConfig implements Serializable, PulsarConfiguration { doc = "The directory where nar packages are extractors" ) private String narExtractionDirectory = NarClassLoader.DEFAULT_NAR_EXTRACTION_DIR; + @FieldContext( + category = CATEGORY_CONNECTORS, + doc = "Whether to enable referencing connectors directory files by file url in connector (sink/source) " + + "creation. Default is true." + ) + private Boolean enableReferencingConnectorDirectoryFiles = true; + @FieldContext( + category = CATEGORY_FUNCTIONS, + doc = "Regex patterns for enabling creation of connectors by referencing packages in matching http/https " + + "urls." + ) + private List additionalEnabledConnectorUrlPatterns = new ArrayList<>(); @FieldContext( category = CATEGORY_CONNECTORS, doc = "Enables extended validation for connector config with fine-grain annotation based validation " @@ -282,6 +294,18 @@ public class WorkerConfig implements Serializable, PulsarConfiguration { doc = "The path to the location to locate builtin functions" ) private String functionsDirectory = "./functions"; + @FieldContext( + category = CATEGORY_FUNCTIONS, + doc = "Whether to enable referencing functions directory files by file url in functions creation. " + + "Default is true." + ) + private Boolean enableReferencingFunctionsDirectoryFiles = true; + @FieldContext( + category = CATEGORY_FUNCTIONS, + doc = "Regex patterns for enabling creation of functions by referencing packages in matching http/https " + + "urls." + ) + private List additionalEnabledFunctionsUrlPatterns = new ArrayList<>(); @FieldContext( category = CATEGORY_FUNC_METADATA_MNG, doc = "The Pulsar topic used for storing function metadata" diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java index 250a7cc4c7bd4..389051fce4217 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionActioner.java @@ -83,18 +83,21 @@ public class FunctionActioner { private final ConnectorsManager connectorsManager; private final FunctionsManager functionsManager; private final PulsarAdmin pulsarAdmin; + private final PackageUrlValidator packageUrlValidator; public FunctionActioner(WorkerConfig workerConfig, RuntimeFactory runtimeFactory, Namespace dlogNamespace, ConnectorsManager connectorsManager, - FunctionsManager functionsManager, PulsarAdmin pulsarAdmin) { + FunctionsManager functionsManager, PulsarAdmin pulsarAdmin, + PackageUrlValidator packageUrlValidator) { this.workerConfig = workerConfig; this.runtimeFactory = runtimeFactory; this.dlogNamespace = dlogNamespace; this.connectorsManager = connectorsManager; this.functionsManager = functionsManager; this.pulsarAdmin = pulsarAdmin; + this.packageUrlValidator = packageUrlValidator; } @@ -152,6 +155,9 @@ private String getPackageFile(FunctionMetaData functionMetaData, FunctionDetails boolean isPkgUrlProvided = isFunctionPackageUrlSupported(packagePath); String packageFile; if (isPkgUrlProvided && packagePath.startsWith(FILE)) { + if (!packageUrlValidator.isValidPackageUrl(componentType, packagePath)) { + throw new IllegalArgumentException("Package URL " + packagePath + " is not valid"); + } URL url = new URL(packagePath); File pkgFile = new File(url.toURI()); packageFile = pkgFile.getAbsolutePath(); @@ -168,7 +174,7 @@ private String getPackageFile(FunctionMetaData functionMetaData, FunctionDetails pkgDir, new File(getDownloadFileName(functionMetaData.getFunctionDetails(), pkgLocation)).getName()); - downloadFile(pkgFile, isPkgUrlProvided, functionMetaData, instanceId, pkgLocation); + downloadFile(pkgFile, isPkgUrlProvided, functionMetaData, instanceId, pkgLocation, componentType); packageFile = pkgFile.getAbsolutePath(); } return packageFile; @@ -227,7 +233,8 @@ InstanceConfig createInstanceConfig(FunctionDetails functionDetails, Function.Fu } private void downloadFile(File pkgFile, boolean isPkgUrlProvided, FunctionMetaData functionMetaData, - int instanceId, Function.PackageLocationMetaData pkgLocation) + int instanceId, Function.PackageLocationMetaData pkgLocation, + FunctionDetails.ComponentType componentType) throws IOException, PulsarAdminException { FunctionDetails details = functionMetaData.getFunctionDetails(); @@ -252,6 +259,9 @@ private void downloadFile(File pkgFile, boolean isPkgUrlProvided, FunctionMetaDa downloadFromHttp ? pkgLocationPath : pkgLocation); if (downloadFromHttp) { + if (!packageUrlValidator.isValidPackageUrl(componentType, pkgLocationPath)) { + throw new IllegalArgumentException("Package URL " + pkgLocationPath + " is not valid"); + } FunctionCommon.downloadFromHttpUrl(pkgLocationPath, tempPkgFile); } else if (downloadFromPackageManagementService) { getPulsarAdmin().packages().download(pkgLocationPath, tempPkgFile.getPath()); diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionRuntimeManager.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionRuntimeManager.java index 8e6725e93af91..b6e2bbb1ca0f8 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionRuntimeManager.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/FunctionRuntimeManager.java @@ -220,7 +220,8 @@ public FunctionRuntimeManager(WorkerConfig workerConfig, PulsarWorkerService wor functionAuthProvider, runtimeCustomizer); this.functionActioner = new FunctionActioner(this.workerConfig, runtimeFactory, - dlogNamespace, connectorsManager, functionsManager, workerService.getBrokerAdmin()); + dlogNamespace, connectorsManager, functionsManager, workerService.getBrokerAdmin(), + workerService.getPackageUrlValidator()); this.membershipManager = membershipManager; this.functionMetaDataManager = functionMetaDataManager; diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PackageUrlValidator.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PackageUrlValidator.java new file mode 100644 index 0000000000000..2a8fe8dddb153 --- /dev/null +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PackageUrlValidator.java @@ -0,0 +1,101 @@ +/* + * 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.worker; + +import java.net.URI; +import java.nio.file.Path; +import java.util.Collections; +import java.util.List; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import org.apache.pulsar.functions.proto.Function; + +/** + * Validates package URLs for functions and connectors. + * Validates that the package URL is either a file in the connectors or functions directory + * when referencing connector or function files is enabled, or matches one of the additional url patterns. + */ +public class PackageUrlValidator { + private final Path connectionsDirectory; + private final Path functionsDirectory; + private final List additionalConnectionsPatterns; + private final List additionalFunctionsPatterns; + + public PackageUrlValidator(WorkerConfig workerConfig) { + this.connectionsDirectory = resolveDirectory(workerConfig.getEnableReferencingConnectorDirectoryFiles(), + workerConfig.getConnectorsDirectory()); + this.functionsDirectory = resolveDirectory(workerConfig.getEnableReferencingFunctionsDirectoryFiles(), + workerConfig.getFunctionsDirectory()); + this.additionalConnectionsPatterns = + compilePatterns(workerConfig.getAdditionalEnabledConnectorUrlPatterns()); + this.additionalFunctionsPatterns = + compilePatterns(workerConfig.getAdditionalEnabledFunctionsUrlPatterns()); + } + + private static Path resolveDirectory(Boolean enabled, String directory) { + return enabled != null && enabled + ? Path.of(directory).normalize().toAbsolutePath() : null; + } + + private static List compilePatterns(List additionalPatterns) { + return additionalPatterns != null ? additionalPatterns.stream().map(Pattern::compile).collect( + Collectors.toList()) : Collections.emptyList(); + } + + boolean isValidFunctionsPackageUrl(URI functionPkgUrl) { + return doesMatch(functionPkgUrl, functionsDirectory, additionalFunctionsPatterns); + } + + boolean isValidConnectionsPackageUrl(URI functionPkgUrl) { + return doesMatch(functionPkgUrl, connectionsDirectory, additionalConnectionsPatterns); + } + + private boolean doesMatch(URI functionPkgUrl, Path directory, List patterns) { + if (directory != null && "file".equals(functionPkgUrl.getScheme())) { + Path filePath = Path.of(functionPkgUrl.getPath()).normalize().toAbsolutePath(); + if (filePath.startsWith(directory)) { + return true; + } + } + String functionPkgUrlString = functionPkgUrl.normalize().toString(); + for (Pattern pattern : patterns) { + if (pattern.matcher(functionPkgUrlString).matches()) { + return true; + } + } + return false; + } + + public boolean isValidPackageUrl(Function.FunctionDetails.ComponentType componentType, String functionPkgUrl) { + URI uri = URI.create(functionPkgUrl); + if (componentType == null) { + // if component type is not specified, we need to check both functions and connections + return isValidFunctionsPackageUrl(uri) || isValidConnectionsPackageUrl(uri); + } + switch (componentType) { + case FUNCTION: + return isValidFunctionsPackageUrl(uri); + case SINK: + case SOURCE: + return isValidConnectionsPackageUrl(uri); + default: + throw new IllegalArgumentException("Unknown component type: " + componentType); + } + } +} diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java index f9f2738828be7..84b943e5671ac 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/PulsarWorkerService.java @@ -119,7 +119,8 @@ public interface PulsarClientCreator { private Sinks sinks; private Sources sources; private Workers workers; - + @Getter + private PackageUrlValidator packageUrlValidator; private final PulsarClientCreator clientCreator; public PulsarWorkerService() { @@ -196,6 +197,7 @@ public void init(WorkerConfig workerConfig, this.sinks = new SinksImpl(() -> PulsarWorkerService.this); this.sources = new SourcesImpl(() -> PulsarWorkerService.this); this.workers = new WorkerImpl(() -> PulsarWorkerService.this); + this.packageUrlValidator = new PackageUrlValidator(workerConfig); } @Override diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java index 5e105f7057e33..6d07e5870917a 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/ComponentImpl.java @@ -1376,11 +1376,17 @@ private StreamingOutput getStreamingOutput(String pkgPath) { private StreamingOutput getStreamingOutput(String pkgPath, FunctionDetails.ComponentType componentType) { return output -> { if (pkgPath.startsWith(Utils.HTTP)) { + if (!worker().getPackageUrlValidator().isValidPackageUrl(componentType, pkgPath)) { + throw new IllegalArgumentException("Invalid package url: " + pkgPath); + } URL url = URI.create(pkgPath).toURL(); try (InputStream inputStream = url.openStream()) { IOUtils.copy(inputStream, output); } } else if (pkgPath.startsWith(Utils.FILE)) { + if (!worker().getPackageUrlValidator().isValidPackageUrl(componentType, pkgPath)) { + throw new IllegalArgumentException("Invalid package url: " + pkgPath); + } URI url = URI.create(pkgPath); File file = new File(url.getPath()); Files.copy(file.toPath(), output); @@ -1804,12 +1810,17 @@ static File downloadPackageFile(PulsarWorkerService worker, String packageName) return file; } - protected File getPackageFile(String functionPkgUrl, String existingPackagePath, InputStream uploadedInputStream) + protected File getPackageFile(FunctionDetails.ComponentType componentType, String functionPkgUrl, + String existingPackagePath, InputStream uploadedInputStream) throws IOException, PulsarAdminException { File componentPackageFile = null; if (isNotBlank(functionPkgUrl)) { - componentPackageFile = getPackageFile(functionPkgUrl); + componentPackageFile = getPackageFile(componentType, functionPkgUrl); } else if (existingPackagePath.startsWith(Utils.FILE) || existingPackagePath.startsWith(Utils.HTTP)) { + if (!worker().getPackageUrlValidator().isValidPackageUrl(componentType, functionPkgUrl)) { + throw new IllegalArgumentException("Function Package url is not valid." + + "supported url (http/https/file)"); + } try { componentPackageFile = FunctionCommon.extractFileFromPkgURL(existingPackagePath); } catch (Exception e) { @@ -1818,7 +1829,7 @@ protected File getPackageFile(String functionPkgUrl, String existingPackagePath, ComponentTypeUtils.toString(componentType), functionPkgUrl)); } } else if (Utils.hasPackageTypePrefix(existingPackagePath)) { - componentPackageFile = getPackageFile(existingPackagePath); + componentPackageFile = getPackageFile(componentType, existingPackagePath); } else if (uploadedInputStream != null) { componentPackageFile = WorkerUtils.dumpToTmpFile(uploadedInputStream); } else if (!existingPackagePath.startsWith(Utils.BUILTIN)) { @@ -1836,15 +1847,16 @@ protected File getPackageFile(String functionPkgUrl, String existingPackagePath, return componentPackageFile; } - protected File downloadPackageFile(String packageName) throws IOException, PulsarAdminException { - return downloadPackageFile(worker(), packageName); - } - - protected File getPackageFile(String functionPkgUrl) throws IOException, PulsarAdminException { + protected File getPackageFile(FunctionDetails.ComponentType componentType, String functionPkgUrl) + throws IOException, PulsarAdminException { if (Utils.hasPackageTypePrefix(functionPkgUrl)) { - return downloadPackageFile(functionPkgUrl); + if (!worker().getWorkerConfig().isFunctionsWorkerEnablePackageManagement()) { + throw new IllegalStateException("Function Package management service is disabled. " + + "Please enable it to use " + functionPkgUrl); + } + return downloadPackageFile(worker(), functionPkgUrl); } else { - if (!Utils.isFunctionPackageUrlSupported(functionPkgUrl)) { + if (!worker().getPackageUrlValidator().isValidPackageUrl(componentType, functionPkgUrl)) { throw new IllegalArgumentException("Function Package url is not valid." + "supported url (http/https/file)"); } diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java index 6b81d2c4918a6..a075d3e18a0b3 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/FunctionsImpl.java @@ -144,7 +144,7 @@ public void registerFunction(final String tenant, // validate parameters try { if (isNotBlank(functionPkgUrl)) { - componentPackageFile = getPackageFile(functionPkgUrl); + componentPackageFile = getPackageFile(componentType, functionPkgUrl); functionDetails = validateUpdateRequestParams(tenant, namespace, functionName, functionConfig, componentPackageFile); } else { @@ -305,6 +305,7 @@ public void updateFunction(final String tenant, // validate parameters try { componentPackageFile = getPackageFile( + componentType, functionPkgUrl, existingComponent.getPackageLocation().getPackagePath(), uploadedInputStream); diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java index 51d1333a79c36..6b8b41e5a8e5b 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SinksImpl.java @@ -143,7 +143,7 @@ public void registerSink(final String tenant, // validate parameters try { if (isNotBlank(sinkPkgUrl)) { - componentPackageFile = getPackageFile(sinkPkgUrl); + componentPackageFile = getPackageFile(componentType, sinkPkgUrl); functionDetails = validateUpdateRequestParams(tenant, namespace, sinkName, sinkConfig, componentPackageFile); } else { @@ -310,6 +310,7 @@ public void updateSink(final String tenant, // validate parameters try { componentPackageFile = getPackageFile( + componentType, sinkPkgUrl, existingComponent.getPackageLocation().getPackagePath(), uploadedInputStream); @@ -421,7 +422,8 @@ private void setTransformFunctionPackageLocation(Function.FunctionMetaData.Build try { String builtin = functionDetails.getBuiltin(); if (isBlank(builtin)) { - functionPackageFile = getPackageFile(transformFunction); + functionPackageFile = + getPackageFile(Function.FunctionDetails.ComponentType.FUNCTION, transformFunction); } Function.PackageLocationMetaData.Builder functionPackageLocation = getFunctionPackageLocation(functionMetaDataBuilder.build(), @@ -744,7 +746,8 @@ private Function.FunctionDetails validateUpdateRequestParams(final String tenant transformFunctionPackage = getBuiltinFunctionPackage(sinkConfig.getTransformFunction()); if (transformFunctionPackage == null) { - File functionPackageFile = getPackageFile(sinkConfig.getTransformFunction()); + File functionPackageFile = getPackageFile(Function.FunctionDetails.ComponentType.FUNCTION, + sinkConfig.getTransformFunction()); transformFunctionPackage = new FunctionFilePackage(functionPackageFile, workerConfig.getNarExtractionDirectory(), workerConfig.getEnableClassloadingOfExternalFiles(), ConnectorDefinition.class); diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java index dea69698dd28d..5191306146951 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/api/SourcesImpl.java @@ -143,7 +143,7 @@ public void registerSource(final String tenant, // validate parameters try { if (isPkgUrlProvided) { - componentPackageFile = getPackageFile(sourcePkgUrl); + componentPackageFile = getPackageFile(componentType, sourcePkgUrl); functionDetails = validateUpdateRequestParams(tenant, namespace, sourceName, sourceConfig, componentPackageFile); } else { @@ -304,6 +304,7 @@ public void updateSource(final String tenant, // validate parameters try { componentPackageFile = getPackageFile( + componentType, sourcePkgUrl, existingComponent.getPackageLocation().getPackagePath(), uploadedInputStream); diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionActionerTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionActionerTest.java index 4e4c3d2f234aa..ac5ca617ea43b 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionActionerTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionActionerTest.java @@ -30,6 +30,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; import static org.testng.AssertJUnit.fail; +import java.util.List; import java.util.Map; import java.util.Optional; import org.apache.distributedlog.api.namespace.Namespace; @@ -77,7 +78,8 @@ public void testStartFunctionWithDLNamespace() throws Exception { @SuppressWarnings("resource") FunctionActioner actioner = new FunctionActioner(workerConfig, factory, dlogNamespace, - new ConnectorsManager(workerConfig), new FunctionsManager(workerConfig), mock(PulsarAdmin.class)); + new ConnectorsManager(workerConfig), new FunctionsManager(workerConfig), mock(PulsarAdmin.class), + mock(PackageUrlValidator.class)); Function.FunctionMetaData function1 = Function.FunctionMetaData.newBuilder() .setFunctionDetails(Function.FunctionDetails.newBuilder().setTenant("test-tenant") .setNamespace("test-namespace").setName("func-1")) @@ -109,6 +111,8 @@ public void testStartFunctionWithPkgUrl() throws Exception { workerConfig.setPulsarServiceUrl("pulsar://localhost:6650"); workerConfig.setStateStorageServiceUrl("foo"); workerConfig.setFunctionAssignmentTopicName("assignments"); + workerConfig.setAdditionalEnabledFunctionsUrlPatterns(List.of("file:///user/.*", "http://invalid/.*")); + workerConfig.setAdditionalEnabledConnectorUrlPatterns(List.of("file:///user/.*", "http://invalid/.*")); String downloadDir = this.getClass().getProtectionDomain().getCodeSource().getLocation().getPath(); workerConfig.setDownloadDirectory(downloadDir); @@ -122,11 +126,12 @@ public void testStartFunctionWithPkgUrl() throws Exception { @SuppressWarnings("resource") FunctionActioner actioner = new FunctionActioner(workerConfig, factory, dlogNamespace, - new ConnectorsManager(workerConfig), new FunctionsManager(workerConfig), mock(PulsarAdmin.class)); + new ConnectorsManager(workerConfig), new FunctionsManager(workerConfig), mock(PulsarAdmin.class), + new PackageUrlValidator(workerConfig)); // (1) test with file url. functionActioner should be able to consider file-url and it should be able to call // RuntimeSpawner - String pkgPathLocation = FILE + ":/user/my-file.jar"; + String pkgPathLocation = FILE + ":///user/my-file.jar"; startFunction(actioner, pkgPathLocation, pkgPathLocation); verify(runtime, times(1)).start(); @@ -194,7 +199,8 @@ public void testFunctionAuthDisabled() throws Exception { @SuppressWarnings("resource") FunctionActioner actioner = new FunctionActioner(workerConfig, factory, dlogNamespace, - new ConnectorsManager(workerConfig), new FunctionsManager(workerConfig), mock(PulsarAdmin.class)); + new ConnectorsManager(workerConfig), new FunctionsManager(workerConfig), mock(PulsarAdmin.class), + mock(PackageUrlValidator.class)); String pkgPathLocation = "http://invalid/my-file.jar"; @@ -257,7 +263,8 @@ public void testStartFunctionWithPackageUrl() throws Exception { @SuppressWarnings("resource") FunctionActioner actioner = new FunctionActioner(workerConfig, factory, dlogNamespace, - new ConnectorsManager(workerConfig), new FunctionsManager(workerConfig), pulsarAdmin); + new ConnectorsManager(workerConfig), new FunctionsManager(workerConfig), pulsarAdmin, + mock(PackageUrlValidator.class)); // (1) test with file url. functionActioner should be able to consider file-url and it should be able to call // RuntimeSpawner diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionRuntimeManagerTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionRuntimeManagerTest.java index bc56b1766d39d..7b57a1c896539 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionRuntimeManagerTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/FunctionRuntimeManagerTest.java @@ -720,7 +720,7 @@ public void testExternallyManagedRuntimeUpdate() throws Exception { FunctionActioner functionActioner = spy(new FunctionActioner( workerConfig, - kubernetesRuntimeFactory, null, null, null, null)); + kubernetesRuntimeFactory, null, null, null, null, workerService.getPackageUrlValidator())); try (final MockedStatic runtimeFactoryMockedStatic = Mockito .mockStatic(RuntimeFactory.class);) { diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionApiResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionApiResourceTest.java index 5845ff3afd9ac..388331ce6f241 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionApiResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionApiResourceTest.java @@ -29,6 +29,7 @@ import java.io.File; import java.io.IOException; import java.io.InputStream; +import java.lang.reflect.Method; import java.util.HashMap; import java.util.LinkedList; import java.util.List; @@ -59,6 +60,12 @@ import org.testng.annotations.Test; public abstract class AbstractFunctionApiResourceTest extends AbstractFunctionsResourceTest { + @Override + protected void customizeWorkerConfig(WorkerConfig workerConfig, Method method) { + if (method.getName().contains("Upload")) { + workerConfig.setFunctionsWorkerEnablePackageManagement(false); + } + } @Test public void testListFunctionsSuccess() { diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionsResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionsResourceTest.java index 4cc4ed0b09819..51ca4c83f9e02 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionsResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/AbstractFunctionsResourceTest.java @@ -28,6 +28,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.UncheckedIOException; +import java.lang.reflect.Method; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; @@ -63,6 +64,7 @@ import org.apache.pulsar.functions.worker.FunctionRuntimeManager; import org.apache.pulsar.functions.worker.FunctionsManager; import org.apache.pulsar.functions.worker.LeaderService; +import org.apache.pulsar.functions.worker.PackageUrlValidator; import org.apache.pulsar.functions.worker.PulsarWorkerService; import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerUtils; @@ -141,7 +143,7 @@ public static File getPulsarApiExamplesNar() { } @BeforeMethod - public final void setup() throws Exception { + public final void setup(Method method) throws Exception { this.mockedManager = mock(FunctionMetaDataManager.class); this.mockedFunctionRunTimeManager = mock(FunctionRuntimeManager.class); this.mockedRuntimeFactory = mock(RuntimeFactory.class); @@ -181,21 +183,33 @@ public final void setup() throws Exception { }).when(mockedPackages).download(any(), any()); // worker config + List urlPatterns = + List.of("http://localhost.*", "file:.*", "https://repo1.maven.org/maven2/org/apache/pulsar/.*"); WorkerConfig workerConfig = new WorkerConfig() .setWorkerId("test") .setWorkerPort(8080) .setFunctionMetadataTopicName("pulsar/functions") .setNumFunctionPackageReplicas(3) - .setPulsarServiceUrl("pulsar://localhost:6650/"); + .setPulsarServiceUrl("pulsar://localhost:6650/") + .setAdditionalEnabledFunctionsUrlPatterns(urlPatterns) + .setAdditionalEnabledConnectorUrlPatterns(urlPatterns) + .setFunctionsWorkerEnablePackageManagement(true); + customizeWorkerConfig(workerConfig, method); tempDirectory = PulsarFunctionTestTemporaryDirectory.create(getClass().getSimpleName()); tempDirectory.useTemporaryDirectoriesForWorkerConfig(workerConfig); when(mockedWorkerService.getWorkerConfig()).thenReturn(workerConfig); when(mockedWorkerService.getFunctionsManager()).thenReturn(functionsManager); when(mockedWorkerService.getConnectorsManager()).thenReturn(connectorsManager); + PackageUrlValidator packageUrlValidator = new PackageUrlValidator(workerConfig); + when(mockedWorkerService.getPackageUrlValidator()).thenReturn(packageUrlValidator); doSetup(); } + protected void customizeWorkerConfig(WorkerConfig workerConfig, Method method) { + + } + protected File getDefaultNarFile() { return getPulsarIOTwitterNar(); } diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java index b9833380d7087..c6c6303e48007 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SinkApiV3ResourceTest.java @@ -36,6 +36,7 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; +import java.lang.reflect.Method; import java.util.Collections; import java.util.HashMap; import java.util.LinkedList; @@ -58,6 +59,7 @@ import org.apache.pulsar.functions.proto.Function.FunctionDetails; import org.apache.pulsar.functions.proto.Function.FunctionMetaData; import org.apache.pulsar.functions.utils.SinkConfigUtils; +import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.functions.worker.rest.api.SinksImpl; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; @@ -80,6 +82,12 @@ protected void doSetup() { this.resource = spy(new SinksImpl(() -> mockedWorkerService)); } + @Override + protected void customizeWorkerConfig(WorkerConfig workerConfig, Method method) { + if (method.getName().contains("Upload") || method.getName().contains("BKPackage")) { + workerConfig.setFunctionsWorkerEnablePackageManagement(false); + } + } @Override protected Function.FunctionDetails.ComponentType getComponentType() { return Function.FunctionDetails.ComponentType.SINK; @@ -1486,17 +1494,15 @@ public void testRegisterSinkSuccessK8sNoUpload() throws Exception { SinkConfig sinkConfig = createDefaultSinkConfig(); sinkConfig.setArchive("builtin://cassandra"); - try (FileInputStream inputStream = new FileInputStream(getPulsarIOCassandraNar())) { - resource.registerSink( - tenant, - namespace, - sink, - inputStream, - mockedFormData, - null, - sinkConfig, - null); - } + resource.registerSink( + tenant, + namespace, + sink, + null, + mockedFormData, + null, + sinkConfig, + null); } /* @@ -1526,21 +1532,19 @@ public void testRegisterSinkSuccessK8sWithUpload() throws Exception { SinkConfig sinkConfig = createDefaultSinkConfig(); sinkConfig.setArchive("builtin://cassandra"); - try (FileInputStream inputStream = new FileInputStream(getPulsarIOCassandraNar())) { - try { - resource.registerSink( - tenant, - namespace, - sink, - inputStream, - mockedFormData, - null, - sinkConfig, - null); - Assert.fail(); - } catch (RuntimeException e) { - Assert.assertEquals(e.getMessage(), injectedErrMsg); - } + try { + resource.registerSink( + tenant, + namespace, + sink, + null, + mockedFormData, + null, + sinkConfig, + null); + Assert.fail(); + } catch (RuntimeException e) { + Assert.assertEquals(e.getMessage(), injectedErrMsg); } } diff --git a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SourceApiV3ResourceTest.java b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SourceApiV3ResourceTest.java index c7e69484d3019..f02acbd3663bf 100644 --- a/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SourceApiV3ResourceTest.java +++ b/pulsar-functions/worker/src/test/java/org/apache/pulsar/functions/worker/rest/api/v3/SourceApiV3ResourceTest.java @@ -34,6 +34,7 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; +import java.lang.reflect.Method; import java.util.LinkedList; import java.util.List; import javax.ws.rs.core.Response; @@ -57,6 +58,7 @@ import org.apache.pulsar.functions.source.TopicSchema; import org.apache.pulsar.functions.utils.SourceConfigUtils; import org.apache.pulsar.functions.utils.io.ConnectorUtils; +import org.apache.pulsar.functions.worker.WorkerConfig; import org.apache.pulsar.functions.worker.WorkerUtils; import org.apache.pulsar.functions.worker.rest.api.SourcesImpl; import org.glassfish.jersey.media.multipart.FormDataContentDisposition; @@ -80,6 +82,13 @@ protected void doSetup() { this.resource = spy(new SourcesImpl(() -> mockedWorkerService)); } + @Override + protected void customizeWorkerConfig(WorkerConfig workerConfig, Method method) { + if (method.getName().endsWith("UploadFailure") || method.getName().contains("BKPackage")) { + workerConfig.setFunctionsWorkerEnablePackageManagement(false); + } + } + @Override protected FunctionDetails.ComponentType getComponentType() { return FunctionDetails.ComponentType.SOURCE; From a955d9f33a0cd146cc7e1f3e855fb7ccb41f1717 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 5 Mar 2024 00:23:41 +0200 Subject: [PATCH 311/699] Fix presto-distribution/LICENSE (cherry picked from commit 7e28e8404f50d065ed37a590c188b8934cae4a75) --- pulsar-sql/presto-distribution/LICENSE | 32 +++++++++++++------------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 32fcf0437ffac..a3ceee66894c0 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -283,22 +283,22 @@ The Apache Software License, Version 2.0 - joda-time-2.10.10.jar - failsafe-2.4.4.jar * Jetty - - http2-client-9.4.53.v20231009.jar - - http2-common-9.4.53.v20231009.jar - - http2-hpack-9.4.53.v20231009.jar - - http2-http-client-transport-9.4.53.v20231009.jar - - jetty-alpn-client-9.4.53.v20231009.jar - - http2-server-9.4.53.v20231009.jar - - jetty-alpn-java-client-9.4.53.v20231009.jar - - jetty-client-9.4.53.v20231009.jar - - jetty-http-9.4.53.v20231009.jar - - jetty-io-9.4.53.v20231009.jar - - jetty-jmx-9.4.53.v20231009.jar - - jetty-security-9.4.53.v20231009.jar - - jetty-server-9.4.53.v20231009.jar - - jetty-servlet-9.4.53.v20231009.jar - - jetty-util-9.4.53.v20231009.jar - - jetty-util-ajax-9.4.53.v20231009.jar + - http2-client-9.4.54.v20240208.jar + - http2-common-9.4.54.v20240208.jar + - http2-hpack-9.4.54.v20240208.jar + - http2-http-client-transport-9.4.54.v20240208.jar + - jetty-alpn-client-9.4.54.v20240208.jar + - http2-server-9.4.54.v20240208.jar + - jetty-alpn-java-client-9.4.54.v20240208.jar + - jetty-client-9.4.54.v20240208.jar + - jetty-http-9.4.54.v20240208.jar + - jetty-io-9.4.54.v20240208.jar + - jetty-jmx-9.4.54.v20240208.jar + - jetty-security-9.4.54.v20240208.jar + - jetty-server-9.4.54.v20240208.jar + - jetty-servlet-9.4.54.v20240208.jar + - jetty-util-9.4.54.v20240208.jar + - jetty-util-ajax-9.4.54.v20240208.jar * Byte Buddy - byte-buddy-1.14.12.jar * Apache BVal From d6cfb19e61d3c2d66ed5e1f9236c0fc65e14e06a Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Tue, 5 Mar 2024 10:20:46 +0530 Subject: [PATCH 312/699] Fix for github ci latest-version-image docker image --- tests/docker-images/latest-version-image/Dockerfile | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/docker-images/latest-version-image/Dockerfile b/tests/docker-images/latest-version-image/Dockerfile index 625242d06913e..349ee615f10c0 100644 --- a/tests/docker-images/latest-version-image/Dockerfile +++ b/tests/docker-images/latest-version-image/Dockerfile @@ -83,13 +83,10 @@ COPY target/plugins/ /pulsar/examples/ 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-kinesis-*.nar /pulsar/connectors/ # download Oracle JDBC driver for Oracle Debezium Connector tests From f4348dcb6cd9456ab77db16f66fddb2c8ff2272a Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Tue, 5 Mar 2024 12:13:32 +0530 Subject: [PATCH 313/699] Fix for github system ci pulsar-io --- docker/pulsar-all/Dockerfile | 1 + tests/docker-images/latest-version-image/Dockerfile | 1 + 2 files changed, 2 insertions(+) diff --git a/docker/pulsar-all/Dockerfile b/docker/pulsar-all/Dockerfile index 6238c19579f7d..d4d7aa42848be 100644 --- a/docker/pulsar-all/Dockerfile +++ b/docker/pulsar-all/Dockerfile @@ -32,6 +32,7 @@ COPY --from=pulsar-all /connectors/pulsar-io-elastic-search-*.nar /pulsar/connec COPY --from=pulsar-all /connectors/pulsar-io-kinesis-*.nar /pulsar/connectors/ # Exclude kafka-connect connector COPY --from=pulsar-all "/connectors/pulsar-io-kafka-[^connect-adaptor*]*.nar" /pulsar/connectors/ +COPY --from=pulsar-all /connectors/pulsar-io-cassandra-*.nar /pulsar/connectors/ COPY --from=pulsar-all /connectors/pulsar-io-jdbc-*.nar /pulsar/connectors/ COPY --from=pulsar-all /connectors/pulsar-io-debezium-*.nar /pulsar/connectors/ COPY --from=pulsar-all /connectors/pulsar-io-data-generator-*.nar /pulsar/connectors/ diff --git a/tests/docker-images/latest-version-image/Dockerfile b/tests/docker-images/latest-version-image/Dockerfile index 349ee615f10c0..feadd77f91bff 100644 --- a/tests/docker-images/latest-version-image/Dockerfile +++ b/tests/docker-images/latest-version-image/Dockerfile @@ -83,6 +83,7 @@ COPY target/plugins/ /pulsar/examples/ 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-jdbc-postgres-*.nar /pulsar/connectors/ From 463e163216743fa2c90c021262f647e771c7d504 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Sun, 17 Sep 2023 20:44:13 -0500 Subject: [PATCH 314/699] [fix][broker] Fix PulsarService.getLookupServiceAddress returns wrong port if TLS is enabled (#21015) (cherry picked from commit 1363777918547b0f56bdbbf04cec28d050a43586) (cherry picked from commit 628e79ded04b8aeb0e146e717e8ed5442c566a6b) --- .../pulsar/broker/admin/AdminApi2Test.java | 14 ++----- .../loadbalance/AdvertisedListenersTest.java | 2 - .../broker/loadbalance/LoadBalancerTest.java | 1 - .../SimpleLoadManagerImplTest.java | 1 - .../impl/ModularLoadManagerImplTest.java | 6 --- .../OwnerShipForCurrentServerTestBase.java | 2 - .../broker/service/BrokerServiceTest.java | 9 +++++ .../transaction/TransactionTestBase.java | 2 - .../containers/BrokerContainer.java | 9 ++++- .../containers/ProxyContainer.java | 9 ++++- .../tests/integration/tls/ClientTlsTest.java | 9 +++++ .../integration/topologies/PulsarCluster.java | 40 +++++++++++++------ .../topologies/PulsarClusterSpec.java | 6 +++ 13 files changed, 69 insertions(+), 41 deletions(-) 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 4d6ac2201225c..7cf84673be86a 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 @@ -439,19 +439,13 @@ public void testTopicPoliciesWithMultiBroker() throws Exception { String tenantName = newUniqueName("prop-xyz2"); admin.tenants().createTenant(tenantName, tenantInfo); admin.namespaces().createNamespace(tenantName + "/ns1", Set.of("test")); - conf.setBrokerServicePort(Optional.of(1024)); - conf.setBrokerServicePortTls(Optional.of(1025)); - conf.setWebServicePort(Optional.of(1026)); - conf.setWebServicePortTls(Optional.of(1027)); + ServiceConfiguration config2 = super.getDefaultConf(); @Cleanup - PulsarTestContext pulsarTestContext2 = createAdditionalPulsarTestContext(conf); + PulsarTestContext pulsarTestContext2 = createAdditionalPulsarTestContext(config2); PulsarService pulsar2 = pulsarTestContext2.getPulsarService(); - conf.setBrokerServicePort(Optional.of(2048)); - conf.setBrokerServicePortTls(Optional.of(2049)); - conf.setWebServicePort(Optional.of(2050)); - conf.setWebServicePortTls(Optional.of(2051)); + ServiceConfiguration config3 = super.getDefaultConf(); @Cleanup - PulsarTestContext pulsarTestContext3 = createAdditionalPulsarTestContext(conf); + PulsarTestContext pulsarTestContext3 = createAdditionalPulsarTestContext(config3); PulsarService pulsar3 = pulsarTestContext.getPulsarService(); @Cleanup PulsarAdmin admin2 = PulsarAdmin.builder().serviceHttpUrl(pulsar2.getWebServiceAddress()).build(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AdvertisedListenersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AdvertisedListenersTest.java index 7a8154312e4dc..a88ccd60ae4c4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AdvertisedListenersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/AdvertisedListenersTest.java @@ -78,7 +78,6 @@ private void updateConfig(ServiceConfiguration conf, String advertisedAddress) { ",public_https:https://localhost:" + httpsPort); conf.setBrokerServicePort(Optional.of(pulsarPort)); conf.setWebServicePort(Optional.of(httpPort)); - conf.setWebServicePortTls(Optional.of(httpsPort)); } @Test @@ -101,7 +100,6 @@ public void testLookup() throws Exception { assertEquals(new URI(ld.getBrokerUrl()).getHost(), "localhost"); assertEquals(new URI(ld.getHttpUrl()).getHost(), "localhost"); - assertEquals(new URI(ld.getHttpUrlTls()).getHost(), "localhost"); // Produce data diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java index d98c96c69d139..04a2175b1d10f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/LoadBalancerTest.java @@ -124,7 +124,6 @@ void setup() throws Exception { config.setAdvertisedAddress("localhost"); config.setWebServicePort(Optional.of(0)); config.setBrokerServicePortTls(Optional.of(0)); - config.setWebServicePortTls(Optional.of(0)); config.setMetadataStoreUrl("zk:127.0.0.1:" + bkEnsemble.getZookeeperPort()); config.setBrokerShutdownTimeoutMs(0L); config.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java index fac9a149b3210..f92faa3e4bd54 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/SimpleLoadManagerImplTest.java @@ -125,7 +125,6 @@ void setup() throws Exception { config1.setBrokerServicePort(Optional.of(0)); config1.setLoadManagerClassName(SimpleLoadManagerImpl.class.getName()); config1.setBrokerServicePortTls(Optional.of(0)); - config1.setWebServicePortTls(Optional.of(0)); config1.setAdvertisedAddress("localhost"); pulsar1 = new PulsarService(config1); pulsar1.start(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java index 90c72e690314f..f80a113c86f97 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/ModularLoadManagerImplTest.java @@ -176,8 +176,6 @@ void setup() throws Exception { config1.setBrokerShutdownTimeoutMs(0L); config1.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); config1.setBrokerServicePort(Optional.of(0)); - config1.setBrokerServicePortTls(Optional.of(0)); - config1.setWebServicePortTls(Optional.of(0)); pulsar1 = new PulsarService(config1); pulsar1.start(); @@ -197,8 +195,6 @@ void setup() throws Exception { config2.setBrokerShutdownTimeoutMs(0L); config2.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); config2.setBrokerServicePort(Optional.of(0)); - config2.setBrokerServicePortTls(Optional.of(0)); - config2.setWebServicePortTls(Optional.of(0)); pulsar2 = new PulsarService(config2); pulsar2.start(); @@ -213,8 +209,6 @@ void setup() throws Exception { config.setBrokerShutdownTimeoutMs(0L); config.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); config.setBrokerServicePort(Optional.of(0)); - config.setBrokerServicePortTls(Optional.of(0)); - config.setWebServicePortTls(Optional.of(0)); pulsar3 = new PulsarService(config); secondaryBrokerId = pulsar2.getBrokerId(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipForCurrentServerTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipForCurrentServerTestBase.java index 8dd4f53db8240..46e8989ac3df4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipForCurrentServerTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/OwnerShipForCurrentServerTestBase.java @@ -80,10 +80,8 @@ protected void startBroker() throws Exception { conf.setBrokerShutdownTimeoutMs(0L); conf.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); conf.setBrokerServicePort(Optional.of(0)); - conf.setBrokerServicePortTls(Optional.of(0)); conf.setAdvertisedAddress("localhost"); conf.setWebServicePort(Optional.of(0)); - conf.setWebServicePortTls(Optional.of(0)); serviceConfigurationList.add(conf); PulsarTestContext.Builder testContextBuilder = 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 2b3d73a957925..5eebf1e998531 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 @@ -1786,4 +1786,13 @@ public void testUnsubscribeNonDurableSub() throws Exception { fail("Unsubscribe failed"); } } + + @Test + public void testGetLookupServiceAddress() throws Exception { + cleanup(); + setup(); + conf.setWebServicePortTls(Optional.of(8081)); + assertEquals(pulsar.getLookupServiceAddress(), "localhost:8081"); + resetState(); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTestBase.java index c0300c63b3587..1ff835732aab5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTestBase.java @@ -157,10 +157,8 @@ protected void startBroker() throws Exception { conf.setBrokerShutdownTimeoutMs(0L); conf.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); conf.setBrokerServicePort(Optional.of(0)); - conf.setBrokerServicePortTls(Optional.of(0)); conf.setAdvertisedAddress("localhost"); conf.setWebServicePort(Optional.of(0)); - conf.setWebServicePortTls(Optional.of(0)); conf.setTransactionCoordinatorEnabled(true); conf.setBrokerDeduplicationEnabled(true); conf.setTransactionBufferSnapshotMaxTransactionCount(2); diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/BrokerContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/BrokerContainer.java index 616d45554d75c..a51397050b97f 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/BrokerContainer.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/BrokerContainer.java @@ -28,8 +28,13 @@ public class BrokerContainer extends PulsarContainer { public static final String NAME = "pulsar-broker"; public BrokerContainer(String clusterName, String hostName) { - super(clusterName, hostName, hostName, "bin/run-broker.sh", BROKER_PORT, BROKER_PORT_TLS, - BROKER_HTTP_PORT, BROKER_HTTPS_PORT, DEFAULT_HTTP_PATH, DEFAULT_IMAGE_NAME); + this(clusterName, hostName, false); + } + + public BrokerContainer(String clusterName, String hostName, boolean enableTls) { + super(clusterName, hostName, hostName, "bin/run-broker.sh", BROKER_PORT, + enableTls ? BROKER_PORT_TLS : 0, BROKER_HTTP_PORT, + enableTls ? BROKER_HTTPS_PORT : 0, DEFAULT_HTTP_PATH, DEFAULT_IMAGE_NAME); tailContainerLog(); } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/ProxyContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/ProxyContainer.java index 53283447378f5..f3926878f37c5 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/ProxyContainer.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/ProxyContainer.java @@ -28,8 +28,13 @@ public class ProxyContainer extends PulsarContainer { public static final String NAME = "pulsar-proxy"; public ProxyContainer(String clusterName, String hostName) { - super(clusterName, hostName, hostName, "bin/run-proxy.sh", BROKER_PORT, BROKER_PORT_TLS, BROKER_HTTP_PORT, - BROKER_HTTPS_PORT, DEFAULT_HTTP_PATH, DEFAULT_IMAGE_NAME); + this(clusterName, hostName, false); + } + + public ProxyContainer(String clusterName, String hostName, boolean enableTls) { + super(clusterName, hostName, hostName, "bin/run-proxy.sh", BROKER_PORT, + enableTls ? BROKER_PORT_TLS : 0, BROKER_HTTP_PORT, + enableTls ? BROKER_HTTPS_PORT : 0, DEFAULT_HTTP_PATH, DEFAULT_IMAGE_NAME); } @Override diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/tls/ClientTlsTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/tls/ClientTlsTest.java index 59ff978cafa06..080912cd49262 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/tls/ClientTlsTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/tls/ClientTlsTest.java @@ -29,6 +29,7 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.tests.integration.suites.PulsarTestSuite; +import org.apache.pulsar.tests.integration.topologies.PulsarClusterSpec; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -41,6 +42,14 @@ private static String loadCertificateAuthorityFile(String name) { return Resources.getResource("certificate-authority/" + name).getPath(); } + @Override + protected PulsarClusterSpec.PulsarClusterSpecBuilder beforeSetupCluster( + String clusterName, + PulsarClusterSpec.PulsarClusterSpecBuilder specBuilder) { + specBuilder.enableTls(true); + return specBuilder; + } + @DataProvider(name = "adminUrls") public Object[][] adminUrls() { return new Object[][]{ 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 781f964e21bfd..e6a425956cf0b 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 @@ -38,6 +38,7 @@ import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.commons.io.IOUtils; +import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.tests.integration.containers.BKContainer; import org.apache.pulsar.tests.integration.containers.BrokerContainer; import org.apache.pulsar.tests.integration.containers.CSContainer; @@ -132,14 +133,16 @@ private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean s this.brokerContainers = Maps.newTreeMap(); this.workerContainers = Maps.newTreeMap(); - this.proxyContainer = new ProxyContainer(appendClusterName("pulsar-proxy"), ProxyContainer.NAME) + this.proxyContainer = new ProxyContainer(appendClusterName("pulsar-proxy"), ProxyContainer.NAME, spec.enableTls) .withNetwork(network) .withNetworkAliases(appendClusterName("pulsar-proxy")) .withEnv("zkServers", appendClusterName(ZKContainer.NAME)) .withEnv("zookeeperServers", appendClusterName(ZKContainer.NAME)) .withEnv("configurationStoreServers", CSContainer.NAME + ":" + CS_PORT) - .withEnv("clusterName", clusterName) + .withEnv("clusterName", clusterName); // enable mTLS + if (spec.enableTls) { + proxyContainer .withEnv("webServicePortTls", String.valueOf(BROKER_HTTPS_PORT)) .withEnv("servicePortTls", String.valueOf(BROKER_PORT_TLS)) .withEnv("forwardAuthorizationCredentials", "true") @@ -147,7 +150,15 @@ private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean s .withEnv("tlsAllowInsecureConnection", "false") .withEnv("tlsCertificateFilePath", "/pulsar/certificate-authority/server-keys/proxy.cert.pem") .withEnv("tlsKeyFilePath", "/pulsar/certificate-authority/server-keys/proxy.key-pk8.pem") - .withEnv("tlsTrustCertsFilePath", "/pulsar/certificate-authority/certs/ca.cert.pem"); + .withEnv("tlsTrustCertsFilePath", "/pulsar/certificate-authority/certs/ca.cert.pem") + .withEnv("brokerClientAuthenticationPlugin", AuthenticationTls.class.getName()) + .withEnv("brokerClientAuthenticationParameters", String.format("tlsCertFile:%s,tlsKeyFile:%s", "/pulsar/certificate-authority/client-keys/admin.cert.pem", "/pulsar/certificate-authority/client-keys/admin.key-pk8.pem")) + .withEnv("tlsEnabledWithBroker", "true") + .withEnv("brokerClientTrustCertsFilePath", "/pulsar/certificate-authority/certs/ca.cert.pem") + .withEnv("brokerClientCertificateFilePath", "/pulsar/certificate-authority/server-keys/proxy.cert.pem") + .withEnv("brokerClientKeyFilePath", "/pulsar/certificate-authority/server-keys/proxy.key-pk8.pem"); + + } if (spec.proxyEnvs != null) { spec.proxyEnvs.forEach(this.proxyContainer::withEnv); } @@ -184,7 +195,7 @@ private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean s // create brokers brokerContainers.putAll( runNumContainers("broker", spec.numBrokers(), (name) -> { - BrokerContainer brokerContainer = new BrokerContainer(clusterName, appendClusterName(name)) + BrokerContainer brokerContainer = new BrokerContainer(clusterName, appendClusterName(name), spec.enableTls) .withNetwork(network) .withNetworkAliases(appendClusterName(name)) .withEnv("zkServers", appendClusterName(ZKContainer.NAME)) @@ -195,16 +206,19 @@ private PulsarCluster(PulsarClusterSpec spec, CSContainer csContainer, boolean s .withEnv("loadBalancerOverrideBrokerNicSpeedGbps", "1") // used in s3 tests .withEnv("AWS_ACCESS_KEY_ID", "accesskey").withEnv("AWS_SECRET_KEY", "secretkey") - .withEnv("maxMessageSize", "" + spec.maxMessageSize) + .withEnv("maxMessageSize", "" + spec.maxMessageSize); + if (spec.enableTls) { // enable mTLS - .withEnv("webServicePortTls", String.valueOf(BROKER_HTTPS_PORT)) - .withEnv("brokerServicePortTls", String.valueOf(BROKER_PORT_TLS)) - .withEnv("authenticateOriginalAuthData", "true") - .withEnv("tlsRequireTrustedClientCertOnConnect", "true") - .withEnv("tlsAllowInsecureConnection", "false") - .withEnv("tlsCertificateFilePath", "/pulsar/certificate-authority/server-keys/broker.cert.pem") - .withEnv("tlsKeyFilePath", "/pulsar/certificate-authority/server-keys/broker.key-pk8.pem") - .withEnv("tlsTrustCertsFilePath", "/pulsar/certificate-authority/certs/ca.cert.pem"); + brokerContainer + .withEnv("webServicePortTls", String.valueOf(BROKER_HTTPS_PORT)) + .withEnv("brokerServicePortTls", String.valueOf(BROKER_PORT_TLS)) + .withEnv("authenticateOriginalAuthData", "true") + .withEnv("tlsAllowInsecureConnection", "false") + .withEnv("tlsRequireTrustedClientCertOnConnect", "true") + .withEnv("tlsTrustCertsFilePath", "/pulsar/certificate-authority/certs/ca.cert.pem") + .withEnv("tlsCertificateFilePath", "/pulsar/certificate-authority/server-keys/broker.cert.pem") + .withEnv("tlsKeyFilePath", "/pulsar/certificate-authority/server-keys/broker.key-pk8.pem"); + } if (spec.queryLastMessage) { brokerContainer.withEnv("bookkeeperExplicitLacIntervalInMills", "10"); brokerContainer.withEnv("bookkeeperUseV2WireProtocol", "false"); 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 fa28d20e6b356..c141e990d62e0 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 @@ -177,4 +177,10 @@ public class PulsarClusterSpec { * Additional ports to expose on bookie containers. */ List bookieAdditionalPorts; + + /** + * Enable TLS for connection. + */ + @Default + boolean enableTls = false; } From 07abd9e6ea6783a87b9d2e1c627787f164beec33 Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Wed, 6 Mar 2024 16:23:21 +0530 Subject: [PATCH 315/699] Revert "Fix for github system ci pulsar-io" This reverts commit 6d379b9da1ae00bfed0c73406edafa0580fc1016. --- docker/pulsar-all/Dockerfile | 1 - tests/docker-images/latest-version-image/Dockerfile | 1 - 2 files changed, 2 deletions(-) diff --git a/docker/pulsar-all/Dockerfile b/docker/pulsar-all/Dockerfile index d4d7aa42848be..6238c19579f7d 100644 --- a/docker/pulsar-all/Dockerfile +++ b/docker/pulsar-all/Dockerfile @@ -32,7 +32,6 @@ COPY --from=pulsar-all /connectors/pulsar-io-elastic-search-*.nar /pulsar/connec COPY --from=pulsar-all /connectors/pulsar-io-kinesis-*.nar /pulsar/connectors/ # Exclude kafka-connect connector COPY --from=pulsar-all "/connectors/pulsar-io-kafka-[^connect-adaptor*]*.nar" /pulsar/connectors/ -COPY --from=pulsar-all /connectors/pulsar-io-cassandra-*.nar /pulsar/connectors/ COPY --from=pulsar-all /connectors/pulsar-io-jdbc-*.nar /pulsar/connectors/ COPY --from=pulsar-all /connectors/pulsar-io-debezium-*.nar /pulsar/connectors/ COPY --from=pulsar-all /connectors/pulsar-io-data-generator-*.nar /pulsar/connectors/ diff --git a/tests/docker-images/latest-version-image/Dockerfile b/tests/docker-images/latest-version-image/Dockerfile index feadd77f91bff..349ee615f10c0 100644 --- a/tests/docker-images/latest-version-image/Dockerfile +++ b/tests/docker-images/latest-version-image/Dockerfile @@ -83,7 +83,6 @@ COPY target/plugins/ /pulsar/examples/ 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-jdbc-postgres-*.nar /pulsar/connectors/ From 64d83198a5856eacb1a39a2d7176763c785e70d0 Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Wed, 6 Mar 2024 16:33:49 +0530 Subject: [PATCH 316/699] Disabled PulsarSinksTest.testCassandraSink & fix for BrokerServiceTest --- .../apache/pulsar/broker/service/BrokerServiceTest.java | 9 --------- .../tests/integration/io/sinks/PulsarSinksTest.java | 2 +- 2 files changed, 1 insertion(+), 10 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceTest.java index 5eebf1e998531..2b3d73a957925 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 @@ -1786,13 +1786,4 @@ public void testUnsubscribeNonDurableSub() throws Exception { fail("Unsubscribe failed"); } } - - @Test - public void testGetLookupServiceAddress() throws Exception { - cleanup(); - setup(); - conf.setWebServicePortTls(Optional.of(8081)); - assertEquals(pulsar.getLookupServiceAddress(), "localhost:8081"); - resetState(); - } } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarSinksTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarSinksTest.java index ca21c4f89b9d2..5bc9f0d9dbcb5 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarSinksTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarSinksTest.java @@ -45,7 +45,7 @@ public void testKafkaSink() throws Exception { testSink(new KafkaSinkTester(kafkaContainerName), true, new KafkaSourceTester(kafkaContainerName)); } - @Test(groups = "sink") +// @Test(groups = "sink") public void testCassandraSink() throws Exception { testSink(CassandraSinkTester.createTester(true), true); } From e783129e3b3e0374c83d678ea3837a0f1e31639a Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Wed, 6 Mar 2024 20:28:13 +0530 Subject: [PATCH 317/699] Fix for LICENSE --- .../server/src/assemble/LICENSE.bin.txt | 44 +++++++++---------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 715cbec18bbc4..b21410d27bfc4 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -585,28 +585,28 @@ Bouncy Castle License - org.bouncycastle-bcutil-jdk18on-1.75.jar Datastax - - com.datastax.oss-managed-ledger-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-auth-oidc-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-auth-sasl-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-common-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-config-validation-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-docs-tools-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-api-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-instance-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-local-runner-original-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-proto-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-runtime-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-secrets-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-utils-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-worker-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-io-core-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-metadata-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-proxy-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-testclient-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-transaction-common-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-transaction-coordinator-3.1.0.3-SNAPSHOT.jar - - com.datastax.oss-testmocks-3.1.0.3-SNAPSHOT.jar + - com.datastax.oss-managed-ledger-3.1.3.0.jar + - com.datastax.oss-pulsar-broker-3.1.3.0.jar + - com.datastax.oss-pulsar-broker-auth-oidc-3.1.3.0.jar + - com.datastax.oss-pulsar-broker-auth-sasl-3.1.3.0.jar + - com.datastax.oss-pulsar-broker-common-3.1.3.0.jar + - com.datastax.oss-pulsar-config-validation-3.1.3.0.jar + - com.datastax.oss-pulsar-docs-tools-3.1.3.0.jar + - com.datastax.oss-pulsar-functions-api-3.1.3.0.jar + - com.datastax.oss-pulsar-functions-instance-3.1.3.0.jar + - com.datastax.oss-pulsar-functions-local-runner-original-3.1.3.0.jar + - com.datastax.oss-pulsar-functions-proto-3.1.3.0.jar + - com.datastax.oss-pulsar-functions-runtime-3.1.3.0.jar + - com.datastax.oss-pulsar-functions-secrets-3.1.3.0.jar + - com.datastax.oss-pulsar-functions-utils-3.1.3.0.jar + - com.datastax.oss-pulsar-functions-worker-3.1.3.0.jar + - com.datastax.oss-pulsar-io-core-3.1.3.0.jar + - com.datastax.oss-pulsar-metadata-3.1.3.0.jar + - com.datastax.oss-pulsar-proxy-3.1.3.0.jar + - com.datastax.oss-pulsar-testclient-3.1.3.0.jar + - com.datastax.oss-pulsar-transaction-common-3.1.3.0.jar + - com.datastax.oss-pulsar-transaction-coordinator-3.1.3.0.jar + - com.datastax.oss-testmocks-3.1.3.0.jar ------------------------ From 7f837be97919bf381396739e0bf4188780169a9f Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Wed, 6 Mar 2024 20:44:05 +0530 Subject: [PATCH 318/699] Release 3.1.3.0 --- 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-experimental/pom.xml | 2 +- docker/pulsar/pom.xml | 2 +- jclouds-shaded/pom.xml | 2 +- managed-ledger/pom.xml | 2 +- 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-client-1x-base/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-1x/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-2x-shaded/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/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/hdfs2/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-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-sql/pom.xml | 2 +- pulsar-sql/presto-distribution/pom.xml | 2 +- pulsar-sql/presto-pulsar-plugin/pom.xml | 2 +- pulsar-sql/presto-pulsar/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 +- 134 files changed, 136 insertions(+), 136 deletions(-) diff --git a/bouncy-castle/bc/pom.xml b/bouncy-castle/bc/pom.xml index 28ef2c1d49caa..a1e9e7790d37e 100644 --- a/bouncy-castle/bc/pom.xml +++ b/bouncy-castle/bc/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.0.3-SNAPSHOT + 3.1.3.0 .. diff --git a/bouncy-castle/bcfips-include-test/pom.xml b/bouncy-castle/bcfips-include-test/pom.xml index 155fe63c1a952..0df4c2ec33d9c 100644 --- a/bouncy-castle/bcfips-include-test/pom.xml +++ b/bouncy-castle/bcfips-include-test/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.0.3-SNAPSHOT + 3.1.3.0 .. diff --git a/bouncy-castle/bcfips/pom.xml b/bouncy-castle/bcfips/pom.xml index b4d95fa49da01..b0295115898e3 100644 --- a/bouncy-castle/bcfips/pom.xml +++ b/bouncy-castle/bcfips/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.0.3-SNAPSHOT + 3.1.3.0 .. diff --git a/bouncy-castle/pom.xml b/bouncy-castle/pom.xml index faf4baacc5454..9458a41085a1d 100644 --- a/bouncy-castle/pom.xml +++ b/bouncy-castle/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.0.3-SNAPSHOT + 3.1.3.0 .. diff --git a/buildtools/pom.xml b/buildtools/pom.xml index a364662d3514e..c89b47ba92267 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -31,12 +31,12 @@ com.datastax.oss buildtools - 3.1.0.3-SNAPSHOT + 3.1.3.0 jar Pulsar Build Tools - 2024-02-13T10:25:34Z + 2024-03-06T15:14:05Z 1.8 1.8 3.1.0 diff --git a/distribution/io/pom.xml b/distribution/io/pom.xml index 71fa332971b22..f58e0b97ce54d 100644 --- a/distribution/io/pom.xml +++ b/distribution/io/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.0.3-SNAPSHOT + 3.1.3.0 .. diff --git a/distribution/offloaders/pom.xml b/distribution/offloaders/pom.xml index b71be609c8bc4..e57297c59fbb5 100644 --- a/distribution/offloaders/pom.xml +++ b/distribution/offloaders/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.0.3-SNAPSHOT + 3.1.3.0 .. diff --git a/distribution/pom.xml b/distribution/pom.xml index 98e8998799cb5..954416cc40050 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.0.3-SNAPSHOT + 3.1.3.0 .. diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index f311685c6f6ae..e57a6c30deafb 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.0.3-SNAPSHOT + 3.1.3.0 .. diff --git a/distribution/shell/pom.xml b/distribution/shell/pom.xml index b20b18e37ceb8..0e79b092456be 100644 --- a/distribution/shell/pom.xml +++ b/distribution/shell/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.0.3-SNAPSHOT + 3.1.3.0 .. diff --git a/docker/pom.xml b/docker/pom.xml index ba7ee1a4b9fb2..33bfc630c6f9d 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.0.3-SNAPSHOT + 3.1.3.0 docker-images Apache Pulsar :: Docker Images diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index 471e9fed7c49a..eaa6cf7feac12 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.0.3-SNAPSHOT + 3.1.3.0 4.0.0 pulsar-all-docker-image diff --git a/docker/pulsar-experimental/pom.xml b/docker/pulsar-experimental/pom.xml index c86b4731caa23..4f13585ff405c 100644 --- a/docker/pulsar-experimental/pom.xml +++ b/docker/pulsar-experimental/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.0.3-SNAPSHOT + 3.1.3.0 4.0.0 pulsar-experimental-docker-image diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 5a907e1e0ac7a..c801d08a9486c 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.0.3-SNAPSHOT + 3.1.3.0 4.0.0 pulsar-docker-image diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml index aa2ca7fc56bdc..007897c59c066 100644 --- a/jclouds-shaded/pom.xml +++ b/jclouds-shaded/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.0.3-SNAPSHOT + 3.1.3.0 .. diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index dc27c0c12640e..d94033957e078 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.0.3-SNAPSHOT + 3.1.3.0 .. diff --git a/pom.xml b/pom.xml index 1db2ca3e23f22..66e38d147e830 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ com.datastax.oss pulsar - 3.1.0.3-SNAPSHOT + 3.1.3.0 Pulsar Pulsar is a distributed pub-sub messaging platform with a very @@ -92,7 +92,7 @@ flexible messaging model and an intuitive client API. UTF-8 UTF-8 - 2024-02-13T10:25:34Z + 2024-03-06T15:14:05Z true diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index a3ceee66894c0..df7887fed3e59 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -472,8 +472,8 @@ The Apache Software License, Version 2.0 - memory-0.8.3.jar - sketches-core-0.8.3.jar * Apache Zookeeper - - zookeeper-3.9.1.jar - - zookeeper-jute-3.9.1.jar + - zookeeper-3.9.2.jar + - zookeeper-jute-3.9.2.jar * Apache Yetus Audience Annotations - audience-annotations-0.12.0.jar * Perfmark From c78f70f91ef0c20d0c182b2faff1a318840294a8 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Fri, 15 Mar 2024 07:56:30 -0700 Subject: [PATCH 329/699] [fix] Upgrade jose4j to 0.9.4 (#22273) (cherry picked from commit 999e39b0c7e568a9ac02fca945a558ca4adad909) (cherry picked from commit be3be093d7bf24d99e445083f5a5f1ed56475a25) --- distribution/server/src/assemble/LICENSE.bin.txt | 2 +- pom.xml | 7 +++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 0fb335b048c47..b7da9969a4bf8 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -260,7 +260,7 @@ The Apache Software License, Version 2.0 * Caffeine -- com.github.ben-manes.caffeine-caffeine-2.9.1.jar * Conscrypt -- org.conscrypt-conscrypt-openjdk-uber-2.5.2.jar * Proto Google Common Protos -- com.google.api.grpc-proto-google-common-protos-2.9.0.jar - * Bitbucket -- org.bitbucket.b_c-jose4j-0.9.3.jar + * Bitbucket -- org.bitbucket.b_c-jose4j-0.9.4.jar * Gson - com.google.code.gson-gson-2.8.9.jar - io.gsonfire-gson-fire-1.8.5.jar diff --git a/pom.xml b/pom.xml index 9eec5fce2cbf1..3aee992a8605e 100644 --- a/pom.xml +++ b/pom.xml @@ -234,6 +234,7 @@ flexible messaging model and an intuitive client API. 2.0.2 5.12.1 18.0.0 + 0.9.4 4.9.3 3.4.0 @@ -625,6 +626,12 @@ flexible messaging model and an intuitive client API. ${rocksdb.version} + + org.bitbucket.b_c + jose4j + ${jose4j.version} + + org.eclipse.jetty jetty-server From 248efe9c81f0f590aeda6a6b54e42c416585b9d2 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Mon, 18 Mar 2024 21:37:03 +0800 Subject: [PATCH 330/699] [improve][misc] Upgrade checkstyle to 10.14.2 (#22291) (cherry picked from commit cd512e4da6a04c68d448a452c76b10b49014033d) (cherry picked from commit 31c5206619f7b9235f6a225ed82152b74b8bc1a3) --- buildtools/pom.xml | 2 +- .../src/main/resources/pulsar/checkstyle.xml | 2 +- pom.xml | 2 +- .../service/persistent/PersistentTopic.java | 4 ++-- .../apache/pulsar/client/impl/ProducerImpl.java | 6 +++--- .../pulsar/common/tls/InetAddressUtils.java | 16 +++++++++------- .../common/util/collections/LongPairSet.java | 4 ++-- 7 files changed, 19 insertions(+), 17 deletions(-) diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 565e6ca8f19e0..56002fe068bef 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -45,7 +45,7 @@ 7.7.1 3.11 4.1 - 8.37 + 10.14.2 3.1.2 4.1.100.Final 4.2.3 diff --git a/buildtools/src/main/resources/pulsar/checkstyle.xml b/buildtools/src/main/resources/pulsar/checkstyle.xml index c63c8993408de..14808cf86638b 100644 --- a/buildtools/src/main/resources/pulsar/checkstyle.xml +++ b/buildtools/src/main/resources/pulsar/checkstyle.xml @@ -179,7 +179,7 @@ page at http://checkstyle.sourceforge.net/config.html --> - + diff --git a/pom.xml b/pom.xml index 3aee992a8605e..a4400f4b7fa8f 100644 --- a/pom.xml +++ b/pom.xml @@ -158,7 +158,7 @@ flexible messaging model and an intuitive client API. 2.14.2 0.10.2 1.6.2 - 8.37 + 10.14.2 0.43.3 true 0.5.0 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 b434d8a2dbc57..5bceed099c1f0 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 @@ -888,8 +888,8 @@ private CompletableFuture internalSubscribe(final TransportCnx cnx, St lock.readLock().unlock(); } - CompletableFuture subscriptionFuture = isDurable ? // - getDurableSubscription(subscriptionName, initialPosition, startMessageRollbackDurationSec, + CompletableFuture subscriptionFuture = isDurable + ? getDurableSubscription(subscriptionName, initialPosition, startMessageRollbackDurationSec, replicatedSubscriptionState, subscriptionProperties) : getNonDurableSubscription(subscriptionName, startMessageId, initialPosition, startMessageRollbackDurationSec, readCompacted, subscriptionProperties); 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 8d2d917d7b880..3fc3bbf3cf783 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 @@ -1831,9 +1831,9 @@ public CompletableFuture connectionOpened(final ClientCnx cnx) { producerCreatedFuture.completeExceptionally(cause); closeProducerTasks(); client.cleanupProducer(this); - } else if (producerCreatedFuture.isDone() || // - (cause instanceof PulsarClientException && PulsarClientException.isRetriableError(cause) - && System.currentTimeMillis() < PRODUCER_DEADLINE_UPDATER.get(ProducerImpl.this))) { + } else if (producerCreatedFuture.isDone() + || (cause instanceof PulsarClientException && PulsarClientException.isRetriableError(cause) + && System.currentTimeMillis() < PRODUCER_DEADLINE_UPDATER.get(ProducerImpl.this))) { // Either we had already created the producer once (producerCreatedFuture.isDone()) or we are // still within the initial timeout budget and we are dealing with a retriable error future.completeExceptionally(cause); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/tls/InetAddressUtils.java b/pulsar-common/src/main/java/org/apache/pulsar/common/tls/InetAddressUtils.java index a8bf837ef5666..d0f3c81a074a1 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/tls/InetAddressUtils.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/tls/InetAddressUtils.java @@ -35,9 +35,12 @@ private InetAddressUtils() { } private static final String IPV4_BASIC_PATTERN_STRING = - "(([1-9]|[1-9][0-9]|1[0-9]{2}|2[0-4][0-9]|25[0-5])\\.){1}" + // initial first field, 1-255 - "(([0-9]|[1-9][0-9]|1[0-9]{2}|2[0-4][0-9]|25[0-5])\\.){2}" + // following 2 fields, 0-255 followed by . - "([0-9]|[1-9][0-9]|1[0-9]{2}|2[0-4][0-9]|25[0-5])"; // final field, 0-255 + // initial first field, 1-255 + "(([1-9]|[1-9][0-9]|1[0-9]{2}|2[0-4][0-9]|25[0-5])\\.){1}" + // following 2 fields, 0-255 followed by . + + "(([0-9]|[1-9][0-9]|1[0-9]{2}|2[0-4][0-9]|25[0-5])\\.){2}" + // final field, 0-255 + + "([0-9]|[1-9][0-9]|1[0-9]{2}|2[0-4][0-9]|25[0-5])"; private static final Pattern IPV4_PATTERN = Pattern.compile("^" + IPV4_BASIC_PATTERN_STRING + "$"); @@ -50,10 +53,9 @@ private InetAddressUtils() { "^[0-9a-fA-F]{1,4}(:[0-9a-fA-F]{1,4}){7}$"); private static final Pattern IPV6_HEX_COMPRESSED_PATTERN = - Pattern.compile( - "^(([0-9A-Fa-f]{1,4}(:[0-9A-Fa-f]{1,4}){0,5})?)" + // 0-6 hex fields - "::" + // concat - "(([0-9A-Fa-f]{1,4}(:[0-9A-Fa-f]{1,4}){0,5})?)$"); // 0-6 hex fields + Pattern.compile("^(([0-9A-Fa-f]{1,4}(:[0-9A-Fa-f]{1,4}){0,5})?)" // 0-6 hex fields + + "::" // concat + + "(([0-9A-Fa-f]{1,4}(:[0-9A-Fa-f]{1,4}){0,5})?)$"); // 0-6 hex fields /* * The above pattern is not totally rigorous as it allows for more than 7 hex fields in total diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairSet.java index 3750d8c22020f..e699d01b9c21c 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairSet.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/LongPairSet.java @@ -96,7 +96,7 @@ public interface LongPairSet { /** * Predicate to checks for a key-value pair where both of them have long types. */ - public interface LongPairPredicate { + interface LongPairPredicate { boolean test(long v1, long v2); } @@ -132,7 +132,7 @@ public interface LongPairPredicate { * */ @FunctionalInterface - public interface LongPairFunction { + interface LongPairFunction { /** * Applies this function to the given arguments. From b1c4de992795000630953f17850a2b0122cd4fd9 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 20 Mar 2024 12:06:19 +0200 Subject: [PATCH 331/699] [fix][ci][branch-3.1] Increase Maven's heap size from 1024m to 1500m as it is in master (cherry picked from commit a013f4aad071dbd14edbfe0079d5d55ac92e3f54) --- .github/workflows/pulsar-ci-flaky.yaml | 2 +- .github/workflows/pulsar-ci.yaml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/pulsar-ci-flaky.yaml b/.github/workflows/pulsar-ci-flaky.yaml index b20812d5b1f85..86fc41f0d8d10 100644 --- a/.github/workflows/pulsar-ci-flaky.yaml +++ b/.github/workflows/pulsar-ci-flaky.yaml @@ -37,7 +37,7 @@ concurrency: cancel-in-progress: true env: - MAVEN_OPTS: -Xss1500k -Xmx1024m -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 -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 # 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 7af0f0991343c..5bba36c1d9caa 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -37,7 +37,7 @@ concurrency: cancel-in-progress: true env: - MAVEN_OPTS: -Xss1500k -Xmx1024m -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 -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 # 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 0a5100ff5da809f73bd0590a882fb41ae2d126bb Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 26 Mar 2024 14:37:10 +0800 Subject: [PATCH 332/699] [fix][broker] Fix ResourceGroup report local usage (#22340) Signed-off-by: Zixuan Liu (cherry picked from commit 0b2b6d593bb6ee9094d4c0a7a311490e7362f68f) (cherry picked from commit 2481cd84ca1da6baa0c639fd3e887b7baf196bdc) --- .../broker/resourcegroup/ResourceGroup.java | 38 ++++--- .../resourcegroup/ResourceGroupService.java | 4 +- .../ResourceGroupReportLocalUsageTest.java | 105 ++++++++++++++++++ 3 files changed, 130 insertions(+), 17 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupReportLocalUsageTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroup.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroup.java index ef40a18ab08ed..c61a8e5e72ba8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroup.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroup.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.resourcegroup; +import com.google.common.annotations.VisibleForTesting; import io.prometheus.client.Counter; import java.util.HashMap; import java.util.Set; @@ -216,24 +217,28 @@ public void rgFillResourceUsage(ResourceUsage resourceUsage) { resourceUsage.setOwner(this.getID()); p = resourceUsage.setPublish(); - this.setUsageInMonitoredEntity(ResourceGroupMonitoringClass.Publish, p); + if (!this.setUsageInMonitoredEntity(ResourceGroupMonitoringClass.Publish, p)) { + resourceUsage.clearPublish(); + } p = resourceUsage.setDispatch(); - this.setUsageInMonitoredEntity(ResourceGroupMonitoringClass.Dispatch, p); + if (!this.setUsageInMonitoredEntity(ResourceGroupMonitoringClass.Dispatch, p)) { + resourceUsage.clearDispatch(); + } // Punt storage for now. } // Transport manager mandated op. public void rgResourceUsageListener(String broker, ResourceUsage resourceUsage) { - NetworkUsage p; - - p = resourceUsage.getPublish(); - this.getUsageFromMonitoredEntity(ResourceGroupMonitoringClass.Publish, p, broker); - - p = resourceUsage.getDispatch(); - this.getUsageFromMonitoredEntity(ResourceGroupMonitoringClass.Dispatch, p, broker); + if (resourceUsage.hasPublish()) { + this.getUsageFromMonitoredEntity(ResourceGroupMonitoringClass.Publish, resourceUsage.getPublish(), broker); + } + if (resourceUsage.hasDispatch()) { + this.getUsageFromMonitoredEntity(ResourceGroupMonitoringClass.Dispatch, resourceUsage.getDispatch(), + broker); + } // Punt storage for now. } @@ -449,12 +454,6 @@ protected boolean setUsageInMonitoredEntity(ResourceGroupMonitoringClass monClas bytesUsed = monEntity.usedLocallySinceLastReport.bytes; messagesUsed = monEntity.usedLocallySinceLastReport.messages; - monEntity.usedLocallySinceLastReport.bytes = monEntity.usedLocallySinceLastReport.messages = 0; - - monEntity.totalUsedLocally.bytes += bytesUsed; - monEntity.totalUsedLocally.messages += messagesUsed; - - monEntity.lastResourceUsageFillTimeMSecsSinceEpoch = System.currentTimeMillis(); if (sendReport) { p.setBytesPerPeriod(bytesUsed); @@ -462,6 +461,10 @@ protected boolean setUsageInMonitoredEntity(ResourceGroupMonitoringClass monClas monEntity.lastReportedValues.bytes = bytesUsed; monEntity.lastReportedValues.messages = messagesUsed; monEntity.numSuppressedUsageReports = 0; + monEntity.usedLocallySinceLastReport.bytes = monEntity.usedLocallySinceLastReport.messages = 0; + monEntity.totalUsedLocally.bytes += bytesUsed; + monEntity.totalUsedLocally.messages += messagesUsed; + monEntity.lastResourceUsageFillTimeMSecsSinceEpoch = System.currentTimeMillis(); } else { numSuppressions = monEntity.numSuppressedUsageReports++; } @@ -594,6 +597,11 @@ public void acceptResourceUsage(String broker, ResourceUsage resourceUsage) { }; } + @VisibleForTesting + PerMonitoringClassFields getMonitoredEntity(ResourceGroupMonitoringClass monClass) { + return this.monitoringClassFields[monClass.ordinal()]; + } + public final String resourceGroupName; public PerMonitoringClassFields[] monitoringClassFields = diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupService.java index d3f8eb7613a40..1f4b3f32cb902 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupService.java @@ -687,7 +687,7 @@ protected void calculateQuotaForAllResourceGroups() { timeUnitScale); this.resourceUsagePublishPeriodInSeconds = newPeriodInSeconds; maxIntervalForSuppressingReportsMSecs = - this.resourceUsagePublishPeriodInSeconds * MaxUsageReportSuppressRounds; + TimeUnit.SECONDS.toMillis(this.resourceUsagePublishPeriodInSeconds) * MaxUsageReportSuppressRounds; } } @@ -706,7 +706,7 @@ private void initialize() { periodInSecs, this.timeUnitScale); maxIntervalForSuppressingReportsMSecs = - this.resourceUsagePublishPeriodInSeconds * MaxUsageReportSuppressRounds; + TimeUnit.SECONDS.toMillis(this.resourceUsagePublishPeriodInSeconds) * MaxUsageReportSuppressRounds; } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupReportLocalUsageTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupReportLocalUsageTest.java new file mode 100644 index 0000000000000..658b7c94165d9 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupReportLocalUsageTest.java @@ -0,0 +1,105 @@ +/* + * 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.resourcegroup; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.broker.resourcegroup.ResourceGroup.BytesAndMessagesCount; +import org.apache.pulsar.broker.resourcegroup.ResourceGroup.PerMonitoringClassFields; +import org.apache.pulsar.broker.resourcegroup.ResourceGroup.ResourceGroupMonitoringClass; +import org.apache.pulsar.broker.service.resource.usage.ResourceUsage; +import org.apache.pulsar.common.policies.data.ResourceGroup; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +public class ResourceGroupReportLocalUsageTest extends MockedPulsarServiceBaseTest { + + @BeforeClass + @Override + protected void setup() throws Exception { + super.internalSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + + @Test + public void testRgFillResourceUsage() throws Exception { + pulsar.getResourceGroupServiceManager().close(); + AtomicBoolean needReport = new AtomicBoolean(false); + ResourceGroupService service = new ResourceGroupService(pulsar, TimeUnit.HOURS, null, + new ResourceQuotaCalculator() { + @Override + public boolean needToReportLocalUsage(long currentBytesUsed, long lastReportedBytes, + long currentMessagesUsed, long lastReportedMessages, + long lastReportTimeMSecsSinceEpoch) { + return needReport.get(); + } + + @Override + public long computeLocalQuota(long confUsage, long myUsage, long[] allUsages) { + return 0; + } + }); + String rgName = "rg-1"; + ResourceGroup rgConfig = new ResourceGroup(); + rgConfig.setPublishRateInBytes(1000L); + rgConfig.setPublishRateInMsgs(2000); + service.resourceGroupCreate(rgName, rgConfig); + + org.apache.pulsar.broker.resourcegroup.ResourceGroup resourceGroup = service.resourceGroupGet(rgName); + BytesAndMessagesCount bytesAndMessagesCount = new BytesAndMessagesCount(); + bytesAndMessagesCount.bytes = 20; + bytesAndMessagesCount.messages = 10; + resourceGroup.incrementLocalUsageStats(ResourceGroupMonitoringClass.Publish, bytesAndMessagesCount); + ResourceUsage resourceUsage = new ResourceUsage(); + resourceGroup.rgFillResourceUsage(resourceUsage); + assertFalse(resourceUsage.hasDispatch()); + assertFalse(resourceUsage.hasPublish()); + + PerMonitoringClassFields publishMonitoredEntity = + resourceGroup.getMonitoredEntity(ResourceGroupMonitoringClass.Publish); + assertEquals(publishMonitoredEntity.usedLocallySinceLastReport.messages, bytesAndMessagesCount.messages); + assertEquals(publishMonitoredEntity.usedLocallySinceLastReport.bytes, bytesAndMessagesCount.bytes); + assertEquals(publishMonitoredEntity.totalUsedLocally.messages, 0); + assertEquals(publishMonitoredEntity.totalUsedLocally.bytes, 0); + assertEquals(publishMonitoredEntity.lastReportedValues.messages, 0); + assertEquals(publishMonitoredEntity.lastReportedValues.bytes, 0); + + needReport.set(true); + resourceGroup.rgFillResourceUsage(resourceUsage); + assertTrue(resourceUsage.hasDispatch()); + assertTrue(resourceUsage.hasPublish()); + assertEquals(publishMonitoredEntity.usedLocallySinceLastReport.messages, 0); + assertEquals(publishMonitoredEntity.usedLocallySinceLastReport.bytes, 0); + assertEquals(publishMonitoredEntity.totalUsedLocally.messages, bytesAndMessagesCount.messages); + assertEquals(publishMonitoredEntity.totalUsedLocally.bytes, bytesAndMessagesCount.bytes); + assertEquals(publishMonitoredEntity.lastReportedValues.messages, bytesAndMessagesCount.messages); + assertEquals(publishMonitoredEntity.lastReportedValues.bytes, bytesAndMessagesCount.bytes); + } +} \ No newline at end of file From 6cd168bf2d3672cf2d23e3f7ea47a28a5d181706 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 26 Mar 2024 16:49:58 +0800 Subject: [PATCH 333/699] [fix][broker] Fix ResourceGroups loading (#21781) Signed-off-by: Zixuan Liu (cherry picked from commit 80b491dab0fd8a948db0a3d85a3ccb8490ecf266) (cherry picked from commit dfb1a67cf42381e6962c5d5a728947b248a3fe52) --- .../ResourceGroupConfigListener.java | 76 ++++++++++++++----- .../ResourceGroupConfigListenerTest.java | 48 ++++++++++++ 2 files changed, 104 insertions(+), 20 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListener.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListener.java index c15edd2be4e43..4a5b8a8bcc244 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListener.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListener.java @@ -18,15 +18,22 @@ */ package org.apache.pulsar.broker.resourcegroup; +import static org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Sets; +import java.util.ArrayList; import java.util.HashSet; +import java.util.List; import java.util.Optional; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import java.util.function.Consumer; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.resources.ResourceGroupResources; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.policies.data.ResourceGroup; +import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.metadata.api.Notification; import org.apache.pulsar.metadata.api.NotificationType; import org.slf4j.Logger; @@ -47,24 +54,32 @@ public class ResourceGroupConfigListener implements Consumer { private final ResourceGroupService rgService; private final PulsarService pulsarService; private final ResourceGroupResources rgResources; - private final ResourceGroupNamespaceConfigListener rgNamespaceConfigListener; + private volatile ResourceGroupNamespaceConfigListener rgNamespaceConfigListener; public ResourceGroupConfigListener(ResourceGroupService rgService, PulsarService pulsarService) { this.rgService = rgService; this.pulsarService = pulsarService; this.rgResources = pulsarService.getPulsarResources().getResourcegroupResources(); - loadAllResourceGroups(); this.rgResources.getStore().registerListener(this); - rgNamespaceConfigListener = new ResourceGroupNamespaceConfigListener( - rgService, pulsarService, this); + execute(() -> loadAllResourceGroupsWithRetryAsync(0)); } - private void loadAllResourceGroups() { - rgResources.listResourceGroupsAsync().whenCompleteAsync((rgList, ex) -> { - if (ex != null) { - LOG.error("Exception when fetching resource groups", ex); - return; + private void loadAllResourceGroupsWithRetryAsync(long retry) { + loadAllResourceGroupsAsync().thenAccept(__ -> { + if (rgNamespaceConfigListener == null) { + rgNamespaceConfigListener = new ResourceGroupNamespaceConfigListener(rgService, pulsarService, this); } + }).exceptionally(e -> { + long nextRetry = retry + 1; + long delay = 500 * nextRetry; + LOG.error("Failed to load all resource groups during initialization, retrying after {}ms: ", delay, e); + schedule(() -> loadAllResourceGroupsWithRetryAsync(nextRetry), delay); + return null; + }); + } + + private CompletableFuture loadAllResourceGroupsAsync() { + return rgResources.listResourceGroupsAsync().thenCompose(rgList -> { final Set existingSet = rgService.resourceGroupGetAll(); HashSet newSet = new HashSet<>(); @@ -72,21 +87,26 @@ private void loadAllResourceGroups() { final Sets.SetView deleteList = Sets.difference(existingSet, newSet); - for (String rgName: deleteList) { + for (String rgName : deleteList) { deleteResourceGroup(rgName); } final Sets.SetView addList = Sets.difference(newSet, existingSet); - for (String rgName: addList) { - pulsarService.getPulsarResources().getResourcegroupResources() - .getResourceGroupAsync(rgName).thenAcceptAsync(optionalRg -> { - ResourceGroup rg = optionalRg.get(); - createResourceGroup(rgName, rg); - }).exceptionally((ex1) -> { - LOG.error("Failed to fetch resourceGroup", ex1); - return null; - }); + List> futures = new ArrayList<>(); + for (String rgName : addList) { + futures.add(pulsarService.getPulsarResources() + .getResourcegroupResources() + .getResourceGroupAsync(rgName) + .thenAccept(optionalRg -> { + if (optionalRg.isPresent()) { + ResourceGroup rg = optionalRg.get(); + createResourceGroup(rgName, rg); + } + }) + ); } + + return FutureUtil.waitForAll(futures); }); } @@ -140,7 +160,10 @@ public void accept(Notification notification) { Optional rgName = ResourceGroupResources.resourceGroupNameFromPath(notifyPath); if ((notification.getType() == NotificationType.ChildrenChanged) || (notification.getType() == NotificationType.Created)) { - loadAllResourceGroups(); + loadAllResourceGroupsAsync().exceptionally((ex) -> { + LOG.error("Exception when fetching resource groups", ex); + return null; + }); } else if (rgName.isPresent()) { switch (notification.getType()) { case Modified: @@ -151,4 +174,17 @@ public void accept(Notification notification) { } } } + + protected void execute(Runnable runnable) { + pulsarService.getExecutor().execute(catchingAndLoggingThrowables(runnable)); + } + + protected void schedule(Runnable runnable, long delayMs) { + pulsarService.getExecutor().schedule(catchingAndLoggingThrowables(runnable), delayMs, TimeUnit.MILLISECONDS); + } + + @VisibleForTesting + ResourceGroupNamespaceConfigListener getRgNamespaceConfigListener() { + return rgNamespaceConfigListener; + } } \ No newline at end of file diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListenerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListenerTest.java index 90c26530850a3..4010635ed9952 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListenerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/resourcegroup/ResourceGroupConfigListenerTest.java @@ -18,20 +18,31 @@ */ package org.apache.pulsar.broker.resourcegroup; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertThrows; import com.google.common.collect.Sets; +import java.util.ArrayList; import java.util.Random; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.broker.resources.ResourceGroupResources; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ResourceGroup; import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.metadata.api.MetadataStore; import org.awaitility.Awaitility; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -288,4 +299,41 @@ private void prepareData() throws PulsarAdminException { testAddRg.setDispatchRateInBytes(200L); } + + @Test + public void testNewResourceGroupNamespaceConfigListener() { + PulsarService pulsarService = mock(PulsarService.class); + PulsarResources pulsarResources = mock(PulsarResources.class); + doReturn(pulsarResources).when(pulsarService).getPulsarResources(); + ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(); + doReturn(scheduledExecutorService).when(pulsarService).getExecutor(); + + ResourceGroupService resourceGroupService = mock(ResourceGroupService.class); + ResourceGroupResources resourceGroupResources = mock(ResourceGroupResources.class); + RuntimeException exception = new RuntimeException("listResourceGroupsAsync error"); + doReturn(CompletableFuture.failedFuture(exception)) + .when(resourceGroupResources).listResourceGroupsAsync(); + doReturn(mock(MetadataStore.class)) + .when(resourceGroupResources).getStore(); + doReturn(resourceGroupResources).when(pulsarResources).getResourcegroupResources(); + + ServiceConfiguration ServiceConfiguration = new ServiceConfiguration(); + doReturn(ServiceConfiguration).when(pulsarService).getConfiguration(); + + ResourceGroupConfigListener resourceGroupConfigListener = + new ResourceGroupConfigListener(resourceGroupService, pulsarService); + + // getResourcegroupResources() returns an error, ResourceGroupNamespaceConfigListener doesn't be created. + Awaitility.await().pollDelay(3, TimeUnit.SECONDS).untilAsserted(() -> { + assertNull(resourceGroupConfigListener.getRgNamespaceConfigListener()); + }); + + // ResourceGroupNamespaceConfigListener will be created, and uses real pulsar resource. + doReturn(CompletableFuture.completedFuture(new ArrayList())) + .when(resourceGroupResources).listResourceGroupsAsync(); + doReturn(pulsar.getPulsarResources()).when(pulsarService).getPulsarResources(); + Awaitility.await().untilAsserted(() -> { + assertNotNull(resourceGroupConfigListener.getRgNamespaceConfigListener()); + }); + } } From 5f68f5122ab622ba33fa3c8ae4d7a7dac336da36 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Thu, 28 Mar 2024 23:14:19 +0800 Subject: [PATCH 334/699] [fix][broker] Fix issue of field 'topic' is not set when handle GetSchema request (#22377) (cherry picked from commit d8903da3d5ea5bab207d119186f2be6fa1147f60) (cherry picked from commit 585fc54f339b78ce1a5edf19a7093355284da0a4) --- .../pulsar/broker/service/ServerCnx.java | 5 +-- .../org/apache/pulsar/schema/SchemaTest.java | 33 +++++++++++++++++++ 2 files changed, 36 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 5057b7b045a92..29ba7cb866ef2 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 @@ -2389,9 +2389,10 @@ remoteAddress, new String(commandGetSchema.getSchemaVersion()), schemaVersion = schemaService.versionFromBytes(commandGetSchema.getSchemaVersion()); } + final String topic = commandGetSchema.getTopic(); String schemaName; try { - schemaName = TopicName.get(commandGetSchema.getTopic()).getSchemaName(); + schemaName = TopicName.get(topic).getSchemaName(); } catch (Throwable t) { commandSender.sendGetSchemaErrorResponse(requestId, ServerError.InvalidTopicName, t.getMessage()); return; @@ -2400,7 +2401,7 @@ remoteAddress, new String(commandGetSchema.getSchemaVersion()), schemaService.getSchema(schemaName, schemaVersion).thenAccept(schemaAndMetadata -> { if (schemaAndMetadata == null) { commandSender.sendGetSchemaErrorResponse(requestId, ServerError.TopicNotFound, - String.format("Topic not found or no-schema %s", commandGetSchema.getTopic())); + String.format("Topic not found or no-schema %s", topic)); } else { commandSender.sendGetSchemaResponse(requestId, SchemaInfoUtil.newSchemaInfo(schemaName, schemaAndMetadata.schema), schemaAndMetadata.version); 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 7eae6462545c8..e10b45868bf47 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 @@ -46,6 +46,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.regex.Pattern; import java.util.stream.Collectors; import lombok.Cleanup; import lombok.EqualsAndHashCode; @@ -69,6 +70,8 @@ import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.client.api.schema.GenericRecord; import org.apache.pulsar.client.api.schema.SchemaDefinition; +import org.apache.pulsar.client.impl.ConsumerImpl; +import org.apache.pulsar.client.impl.MultiTopicsConsumerImpl; import org.apache.pulsar.client.impl.schema.KeyValueSchemaImpl; import org.apache.pulsar.client.impl.schema.ProtobufSchema; import org.apache.pulsar.client.impl.schema.SchemaInfoImpl; @@ -98,6 +101,7 @@ public class SchemaTest extends MockedPulsarServiceBaseTest { @BeforeMethod @Override public void setup() throws Exception { + isTcpLookup = true; super.internalSetup(); // Setup namespaces @@ -106,6 +110,7 @@ public void setup() throws Exception { .allowedClusters(Collections.singleton(CLUSTER_NAME)) .build(); admin.tenants().createTenant(PUBLIC_TENANT, tenantInfo); + admin.namespaces().createNamespace(PUBLIC_TENANT + "/my-ns"); } @AfterMethod(alwaysRun = true) @@ -130,6 +135,34 @@ public void testGetSchemaWhenCreateAutoProduceBytesProducer() throws Exception{ pulsarClient.newProducer(org.apache.pulsar.client.api.Schema.AUTO_PRODUCE_BYTES()).topic(topic).create(); } + @Test + public void testGetSchemaWithPatternTopic() throws Exception { + final String topicPrefix = "persistent://public/my-ns/test-getSchema"; + + int topicNums = 10; + for (int i = 0; i < topicNums; i++) { + String topic = topicPrefix + "-" + i; + admin.topics().createNonPartitionedTopic(topic); + } + + Pattern pattern = Pattern.compile(topicPrefix + "-.*"); + @Cleanup + Consumer consumer = pulsarClient.newConsumer(Schema.AUTO_CONSUME()) + .topicsPattern(pattern) + .subscriptionName("sub") + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + + List> consumers = + ((MultiTopicsConsumerImpl) consumer).getConsumers(); + Assert.assertEquals(topicNums, consumers.size()); + + for (int i = 0; i < topicNums; i++) { + String topic = topicPrefix + "-" + i; + admin.topics().delete(topic, true); + } + } + @Test public void testMultiTopicSetSchemaProvider() throws Exception { final String tenant = PUBLIC_TENANT; From 2ac546c5d19b6bb349d6ca3377c7e103e942bcd2 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 2 Apr 2024 08:17:26 -0700 Subject: [PATCH 335/699] [improve][broker] Don't log brokerClientAuthenticationParameters and bookkeeperClientAuthenticationParameters by default (#22395) (cherry picked from commit ad28a7c1ef717aafa1c457762f43101152665572) (cherry picked from commit 3392170bfb51497ac5026264ed34513c37dc969d) --- .../java/org/apache/pulsar/broker/ServiceConfiguration.java | 2 ++ 1 file changed, 2 insertions(+) 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 307943ea37523..ffa5828f4975e 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 @@ -1689,6 +1689,7 @@ The delayed message index time step(in seconds) in per bucket snapshot segment, category = CATEGORY_STORAGE_BK, doc = "Parameters for bookkeeper auth plugin" ) + @ToString.Exclude private String bookkeeperClientAuthenticationParameters; @FieldContext( @@ -3273,6 +3274,7 @@ The delayed message index time step(in seconds) in per bucket snapshot segment, doc = "Authentication parameters of the authentication plugin the broker is using to connect " + "to other brokers" ) + @ToString.Exclude private String brokerClientAuthenticationParameters = ""; @FieldContext( category = CATEGORY_REPLICATION, From b080c7f5acb0b07af8a92b548f085d2efa1b4aa1 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Wed, 3 Apr 2024 04:37:27 +0800 Subject: [PATCH 336/699] [fix][build] Fix networkaddress.cache.negative.ttl config (#22400) Signed-off-by: Zixuan Liu (cherry picked from commit be012c8accf58c616c2e66333d5bd3d684326d22) (cherry picked from commit 20d35f130b86c204edafb60f18df7f4245159f38) --- docker/pulsar/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index b465bbf251da8..10bd2b5ee2675 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -79,6 +79,7 @@ RUN mkdir -p /etc/apt/keyrings \ && apt-get -y install temurin-17-jdk \ && export ARCH=$(uname -m | sed -r 's/aarch64/arm64/g' | awk '!/arm64/{$0="amd64"}1') \ && echo networkaddress.cache.ttl=1 >> /usr/lib/jvm/temurin-17-jdk-$ARCH/conf/security/java.security \ + && echo networkaddress.cache.negative.ttl=1 >> /usr/lib/jvm/temurin-17-jdk-$ARCH/conf/security/java.security # Cleanup apt RUN apt-get -y --purge autoremove \ From 95a75a827ba763c75437fc5ce939152ae4d6b292 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Tue, 2 Apr 2024 19:07:49 -0700 Subject: [PATCH 337/699] [fix][broker] Skip topic.close during unloading if the topic future fails with ownership check, and fix isBundleOwnedByAnyBroker to use ns.checkOwnershipPresentAsync for ExtensibleLoadBalancer (#22379) (#22406) (cherry picked from commit e664432b7b7bf98e53afe6fe8fcb71e1aa664303) --- .../extensions/manager/UnloadManager.java | 20 ++++++++- .../pulsar/broker/service/BrokerService.java | 11 ++++- .../pulsar/broker/web/PulsarWebResource.java | 5 +++ .../ExtensibleLoadManagerImplBaseTest.java | 4 -- .../ExtensibleLoadManagerImplTest.java | 27 ++++++++++++ .../extensions/manager/UnloadManagerTest.java | 44 +++++++++++-------- 6 files changed, 85 insertions(+), 26 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java index ffdbbc2af4219..bf9885b2a252e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker.loadbalance.extensions.manager; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigning; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.models.UnloadDecision.Label.Failure; import static org.apache.pulsar.broker.loadbalance.extensions.models.UnloadDecision.Reason.Unknown; import java.util.Map; @@ -25,6 +27,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState; import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData; import org.apache.pulsar.broker.loadbalance.extensions.models.UnloadCounter; @@ -88,14 +91,27 @@ public CompletableFuture waitAsync(CompletableFuture eventPubFuture, @Override public void handleEvent(String serviceUnit, ServiceUnitStateData data, Throwable t) { - if (t != null && inFlightUnloadRequest.containsKey(serviceUnit)) { + ServiceUnitState state = ServiceUnitStateData.state(data); + + if (StringUtils.isBlank(data.sourceBroker()) && (state == Owned || state == Assigning)) { + if (log.isDebugEnabled()) { + log.debug("Skipping {} for service unit {} from the assignment command.", data, serviceUnit); + } + return; + } + + if (t != null) { if (log.isDebugEnabled()) { log.debug("Handling {} for service unit {} with exception.", data, serviceUnit, t); } this.complete(serviceUnit, t); return; } - ServiceUnitState state = ServiceUnitStateData.state(data); + + if (log.isDebugEnabled()) { + log.debug("Handling {} for service unit {}", data, serviceUnit); + } + switch (state) { case Free, Owned -> this.complete(serviceUnit, t); default -> { 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 6f61616548b30..8d289446b7b42 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 @@ -2270,9 +2270,18 @@ private CompletableFuture unloadServiceUnit(NamespaceBundle serviceUnit } closeFutures.add(topicFuture .thenCompose(t -> t.isPresent() ? t.get().close(closeWithoutWaitingClientDisconnect) - : CompletableFuture.completedFuture(null))); + : CompletableFuture.completedFuture(null)) + .exceptionally(e -> { + if (e.getCause() instanceof BrokerServiceException.ServiceUnitNotReadyException + && e.getMessage().contains("Please redo the lookup")) { + log.warn("[{}] Topic ownership check failed. Skipping it", topicName); + return null; + } + throw FutureUtil.wrapToCompletionException(e); + })); } }); + if (getPulsar().getConfig().isTransactionCoordinatorEnabled() && serviceUnit.getNamespaceObject().equals(NamespaceName.SYSTEM_NAMESPACE)) { TransactionMetadataStoreService metadataStoreService = 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 cde555e24aaff..0d01a2c50419d 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 @@ -609,11 +609,16 @@ protected CompletableFuture isBundleOwnedByAnyBroker(NamespaceName fqnn NamespaceBundle nsBundle = validateNamespaceBundleRange(fqnn, bundles, bundleRange); NamespaceService nsService = pulsar().getNamespaceService(); + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { + return nsService.checkOwnershipPresentAsync(nsBundle); + } + LookupOptions options = LookupOptions.builder() .authoritative(false) .requestHttps(isRequestHttps()) .readOnly(true) .loadTopicsInBundle(false).build(); + return nsService.getWebServiceUrlAsync(nsBundle, options).thenApply(Optional::isPresent); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java index d9c6f78b8d01f..0b58f98f78fd9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java @@ -37,7 +37,6 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; -import org.apache.pulsar.common.policies.data.TopicType; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; @@ -66,13 +65,10 @@ protected ExtensibleLoadManagerImplBaseTest(String defaultTestNamespace) { protected ServiceConfiguration initConfig(ServiceConfiguration conf) { conf.setForceDeleteNamespaceAllowed(true); - conf.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); - conf.setAllowAutoTopicCreation(true); conf.setLoadManagerClassName(ExtensibleLoadManagerImpl.class.getName()); conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); conf.setLoadBalancerSheddingEnabled(false); conf.setLoadBalancerDebugModeEnabled(true); - conf.setTopicLevelPoliciesEnabled(true); return conf; } 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 c32deddaae2b1..9c363325c4cc8 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 @@ -64,6 +64,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -237,6 +238,32 @@ public CompletableFuture> filterAsync(Map { + future1.completeExceptionally(new CompletionException( + new BrokerServiceException.ServiceUnitNotReadyException("Please redo the lookup"))); + future2.completeExceptionally(new CompletionException( + new BrokerServiceException.ServiceUnitNotReadyException("Please redo the lookup"))); + }); + admin.namespaces().unloadNamespaceBundle(bundle.getNamespaceObject().toString(), bundle.getBundleRange()); + } finally { + pulsar1.getBrokerService().getTopics().remove(topicName.toString()); + pulsar2.getBrokerService().getTopics().remove(topicName.toString()); + } + } + + @Test(timeOut = 30 * 1000) public void testUnloadAdminAPI() throws Exception { TopicName topicName = TopicName.get(defaultTestNamespace + "/test-unload"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManagerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManagerTest.java index 6a2ae1cc562cc..45b1cd9544f91 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManagerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManagerTest.java @@ -94,53 +94,59 @@ public void testTimeout() throws IllegalAccessException { public void testSuccess() throws IllegalAccessException, ExecutionException, InterruptedException { UnloadCounter counter = new UnloadCounter(); UnloadManager manager = new UnloadManager(counter); + String dstBroker = "broker-2"; + String srcBroker = "broker-1"; + String bundle = "bundle-1"; var unloadDecision = - new UnloadDecision(new Unload("broker-1", "bundle-1"), Success, Admin); + new UnloadDecision(new Unload(srcBroker, bundle), Success, Admin); CompletableFuture future = manager.waitAsync(CompletableFuture.completedFuture(null), - "bundle-1", unloadDecision, 5, TimeUnit.SECONDS); + bundle, unloadDecision, 5, TimeUnit.SECONDS); Map> inFlightUnloadRequestMap = getInFlightUnloadRequestMap(manager); assertEquals(inFlightUnloadRequestMap.size(), 1); - manager.handleEvent("bundle-1", - new ServiceUnitStateData(ServiceUnitState.Assigning, "broker-1", VERSION_ID_INIT), null); + manager.handleEvent(bundle, + new ServiceUnitStateData(ServiceUnitState.Assigning, null, srcBroker, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequestMap.size(), 1); - manager.handleEvent("bundle-1", - new ServiceUnitStateData(ServiceUnitState.Deleted, "broker-1", VERSION_ID_INIT), null); + manager.handleEvent(bundle, + new ServiceUnitStateData(ServiceUnitState.Deleted, null, srcBroker, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequestMap.size(), 1); - manager.handleEvent("bundle-1", - new ServiceUnitStateData(ServiceUnitState.Splitting, "broker-1", VERSION_ID_INIT), null); + manager.handleEvent(bundle, + new ServiceUnitStateData(ServiceUnitState.Splitting, null, srcBroker, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequestMap.size(), 1); - manager.handleEvent("bundle-1", - new ServiceUnitStateData(ServiceUnitState.Releasing, "broker-1", VERSION_ID_INIT), null); + manager.handleEvent(bundle, + new ServiceUnitStateData(ServiceUnitState.Releasing, null, srcBroker, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequestMap.size(), 1); - manager.handleEvent("bundle-1", - new ServiceUnitStateData(ServiceUnitState.Init, "broker-1", VERSION_ID_INIT), null); + manager.handleEvent(bundle, + new ServiceUnitStateData(ServiceUnitState.Init, null, srcBroker, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequestMap.size(), 1); - manager.handleEvent("bundle-1", - new ServiceUnitStateData(ServiceUnitState.Free, "broker-1", VERSION_ID_INIT), null); + manager.handleEvent(bundle, + new ServiceUnitStateData(ServiceUnitState.Free, null, srcBroker, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequestMap.size(), 0); future.get(); assertEquals(counter.getBreakdownCounters().get(Success).get(Admin).get(), 1); // Success with Owned state. future = manager.waitAsync(CompletableFuture.completedFuture(null), - "bundle-1", unloadDecision, 5, TimeUnit.SECONDS); + bundle, unloadDecision, 5, TimeUnit.SECONDS); inFlightUnloadRequestMap = getInFlightUnloadRequestMap(manager); + assertEquals(inFlightUnloadRequestMap.size(), 1); + manager.handleEvent(bundle, + new ServiceUnitStateData(ServiceUnitState.Owned, dstBroker, null, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequestMap.size(), 1); - manager.handleEvent("bundle-1", - new ServiceUnitStateData(ServiceUnitState.Owned, "broker-1", VERSION_ID_INIT), null); + manager.handleEvent(bundle, + new ServiceUnitStateData(ServiceUnitState.Owned, dstBroker, srcBroker, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequestMap.size(), 0); - future.get(); + future.get(); assertEquals(counter.getBreakdownCounters().get(Success).get(Admin).get(), 2); } @@ -158,7 +164,7 @@ public void testFailedStage() throws IllegalAccessException { assertEquals(inFlightUnloadRequestMap.size(), 1); manager.handleEvent("bundle-1", - new ServiceUnitStateData(ServiceUnitState.Owned, "broker-1", VERSION_ID_INIT), + new ServiceUnitStateData(ServiceUnitState.Owned, null, "broker-1", VERSION_ID_INIT), new IllegalStateException("Failed stage.")); try { From cca9630d93d3ce3cd29d7fa49f7443a023e811bc Mon Sep 17 00:00:00 2001 From: Teet Vaher <89580420+teet-vaher-sympower@users.noreply.github.com> Date: Wed, 3 Apr 2024 14:26:24 +0300 Subject: [PATCH 338/699] [fix][broker] Fix invalid condition in logging exceptions (#22412) (cherry picked from commit cd6f53baee7dbe0651e6581cb9bd3570017348c7) (cherry picked from commit 265e4ed40cd1e5f98730b831e49b777a59269efd) --- .../admin/impl/PersistentTopicsBase.java | 84 +++++++++---------- .../broker/admin/v2/NonPersistentTopics.java | 6 +- .../pulsar/broker/admin/v3/Transactions.java | 12 +-- 3 files changed, 51 insertions(+), 51 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 fe451d1650e17..9a2dd5c7c54fd 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 @@ -889,7 +889,7 @@ protected void internalUnloadTopic(AsyncResponse asyncResponse, boolean authorit } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned metadata while unloading topic {}", clientAppId(), topicName, ex); } @@ -899,7 +899,7 @@ protected void internalUnloadTopic(AsyncResponse asyncResponse, boolean authorit } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to validate the global namespace ownership while unloading topic {}", clientAppId(), topicName, ex); } @@ -1066,7 +1066,7 @@ private void internalUnloadNonPartitionedTopicAsync(AsyncResponse asyncResponse, }) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to unload topic {}, {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1086,7 +1086,7 @@ private void internalUnloadTransactionCoordinatorAsync(AsyncResponse asyncRespon }) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to unload tc {},{}", clientAppId(), topicName.getPartitionIndex(), ex); } @@ -1188,7 +1188,7 @@ protected void internalGetSubscriptions(AsyncResponse asyncResponse, boolean aut } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned topic metadata while get" + " subscriptions for topic {}", clientAppId(), topicName, ex); } @@ -1198,7 +1198,7 @@ protected void internalGetSubscriptions(AsyncResponse asyncResponse, boolean aut } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to validate the global namespace/topic ownership while get subscriptions" + " for topic {}", clientAppId(), topicName, ex); } @@ -1207,7 +1207,7 @@ protected void internalGetSubscriptions(AsyncResponse asyncResponse, boolean aut }) ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get subscriptions for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1246,7 +1246,7 @@ private void internalGetSubscriptionsForNonPartitionedTopic(AsyncResponse asyncR .thenAccept(topic -> asyncResponse.resume(new ArrayList<>(topic.getSubscriptions().keys()))) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get list of subscriptions for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1358,7 +1358,7 @@ protected void internalGetManagedLedgerInfo(AsyncResponse asyncResponse, boolean } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned metadata while get managed info for {}", clientAppId(), topicName, ex); } @@ -1368,7 +1368,7 @@ protected void internalGetManagedLedgerInfo(AsyncResponse asyncResponse, boolean } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to validate the global namespace ownership while get managed info for {}", clientAppId(), topicName, ex); } @@ -1482,7 +1482,7 @@ protected void internalGetPartitionedStats(AsyncResponse asyncResponse, boolean }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned internal stats for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1538,7 +1538,7 @@ protected void internalGetPartitionedStatsInternal(AsyncResponse asyncResponse, }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get partitioned internal stats for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -1666,7 +1666,7 @@ private void internalAnalyzeSubscriptionBacklogForNonPartitionedTopic(AsyncRespo }).exceptionally(ex -> { Throwable cause = ex.getCause(); // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to analyze subscription backlog {} {}", clientAppId(), topicName, subName, cause); } @@ -1693,7 +1693,7 @@ private void internalUpdateSubscriptionPropertiesForNonPartitionedTopic(AsyncRes }).exceptionally(ex -> { Throwable cause = ex.getCause(); // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to update subscription {} {}", clientAppId(), topicName, subName, cause); } asyncResponse.resume(new RestException(cause)); @@ -1722,7 +1722,7 @@ private void internalGetSubscriptionPropertiesForNonPartitionedTopic(AsyncRespon }).exceptionally(ex -> { Throwable cause = ex.getCause(); // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to update subscription {} {}", clientAppId(), topicName, subName, cause); } asyncResponse.resume(new RestException(cause)); @@ -1891,7 +1891,7 @@ protected void internalSkipAllMessages(AsyncResponse asyncResponse, String subNa } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to skip all messages for subscription {} on topic {}", clientAppId(), subName, topicName, ex); } @@ -1935,7 +1935,7 @@ private CompletableFuture internalSkipAllMessagesForNonPartitionedTopicAsy } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to skip all messages for subscription {} on topic {}", clientAppId(), subName, topicName, ex); } @@ -1999,7 +1999,7 @@ protected void internalSkipMessages(AsyncResponse asyncResponse, String subName, }) ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to skip {} messages {} {}", clientAppId(), numMessages, topicName, subName, ex); } @@ -2069,7 +2069,7 @@ protected void internalExpireMessagesForAllSubscriptions(AsyncResponse asyncResp ) ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to expire messages for all subscription on topic {}", clientAppId(), topicName, ex); } @@ -2136,7 +2136,7 @@ private void internalExpireMessagesForAllSubscriptionsForNonPartitionedTopic(Asy }) ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to expire messages for all subscription up to {} on {}", clientAppId(), expireTimeInSeconds, topicName, ex); } @@ -2344,7 +2344,7 @@ protected void internalCreateSubscription(AsyncResponse asyncResponse, String su })).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to create subscription {} on topic {}", clientAppId(), subscriptionName, topicName, ex); } @@ -2354,7 +2354,7 @@ protected void internalCreateSubscription(AsyncResponse asyncResponse, String su } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to create subscription {} on topic {}", clientAppId(), subscriptionName, topicName, ex); } @@ -2484,7 +2484,7 @@ protected void internalUpdateSubscriptionProperties(AsyncResponse asyncResponse, } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to update subscription {} from topic {}", clientAppId(), subName, topicName, ex); } @@ -2523,7 +2523,7 @@ protected void internalAnalyzeSubscriptionBacklog(AsyncResponse asyncResponse, S }) .exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to analyze back log of subscription {} from topic {}", clientAppId(), subName, topicName, ex); } @@ -2607,7 +2607,7 @@ protected void internalGetSubscriptionProperties(AsyncResponse asyncResponse, St } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to update subscription {} from topic {}", clientAppId(), subName, topicName, ex); } @@ -2693,7 +2693,7 @@ protected void internalResetCursorOnPosition(AsyncResponse asyncResponse, String }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.warn("[{}][{}] Failed to reset cursor on subscription {} to position {}", clientAppId(), topicName, subName, messageId, ex.getCause()); } @@ -2702,7 +2702,7 @@ protected void internalResetCursorOnPosition(AsyncResponse asyncResponse, String }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.warn("[{}][{}] Failed to reset cursor on subscription {} to position {}", clientAppId(), topicName, subName, messageId, ex.getCause()); } @@ -3326,7 +3326,7 @@ protected void internalGetBacklogSizeByMessageId(AsyncResponse asyncResponse, } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get backlog size for topic {}", clientAppId(), topicName, ex); } @@ -3334,7 +3334,7 @@ protected void internalGetBacklogSizeByMessageId(AsyncResponse asyncResponse, return null; })).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to validate global namespace ownership " + "to get backlog size for topic {}", clientAppId(), topicName, ex); } @@ -3872,7 +3872,7 @@ protected void internalTerminatePartitionedTopic(AsyncResponse asyncResponse, bo } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to terminate topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -3880,7 +3880,7 @@ protected void internalTerminatePartitionedTopic(AsyncResponse asyncResponse, bo }) ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to terminate topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -3970,7 +3970,7 @@ protected void internalExpireMessagesByTimestamp(AsyncResponse asyncResponse, St ).exceptionally(ex -> { Throwable cause = FutureUtil.unwrapCompletionException(ex); // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(cause)) { + if (isNot307And404Exception(cause)) { if (cause instanceof RestException) { log.warn("[{}] Failed to expire messages up to {} on {}: {}", clientAppId(), expireTimeInSeconds, topicName, cause.toString()); @@ -4085,7 +4085,7 @@ protected void internalExpireMessagesByPosition(AsyncResponse asyncResponse, Str messageId, isExcluded, batchIndex); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to expire messages up to {} on subscription {} to position {}", clientAppId(), topicName, subName, messageId, ex); } @@ -4236,7 +4236,7 @@ protected void internalTriggerCompaction(AsyncResponse asyncResponse, boolean au } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to trigger compaction on topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -4245,7 +4245,7 @@ protected void internalTriggerCompaction(AsyncResponse asyncResponse, boolean au } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to validate global namespace ownership to trigger compaction on topic {}", clientAppId(), topicName, ex); } @@ -4274,7 +4274,7 @@ protected void internalTriggerCompactionNonPartitionedTopic(AsyncResponse asyncR } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to trigger compaction for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -4310,7 +4310,7 @@ protected void internalTriggerOffload(AsyncResponse asyncResponse, } }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to trigger offload for {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -4327,7 +4327,7 @@ protected void internalOffloadStatus(AsyncResponse asyncResponse, boolean author asyncResponse.resume(offloadProcessStatus); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to offload status on topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -4652,7 +4652,7 @@ protected void internalGetLastMessageId(AsyncResponse asyncResponse, boolean aut }); }).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get last messageId {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -5157,7 +5157,7 @@ protected void internalSetReplicatedSubscriptionStatus(AsyncResponse asyncRespon resultFuture.exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.warn("[{}] Failed to change replicated subscription status to {} - {} {}", clientAppId(), enabled, topicName, subName, ex); } @@ -5204,7 +5204,7 @@ private void internalSetReplicatedSubscriptionStatusForNonPartitionedTopic( } ).exceptionally(ex -> { // If the exception is not redirect exception we need to log it. - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to set replicated subscription status on {} {}", clientAppId(), topicName, subName, ex); } @@ -5305,7 +5305,7 @@ protected void internalGetReplicatedSubscriptionStatus(AsyncResponse asyncRespon } resultFuture.exceptionally(ex -> { - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get replicated subscription status on {} {}", clientAppId(), topicName, subName, ex); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java index e0a1b8f89ee3f..f8fe30ccecb2c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java @@ -132,7 +132,7 @@ public void getInternalStats( }) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get internal stats for topic {}", clientAppId(), topicName, ex); } resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -468,7 +468,7 @@ public void getListFromBundle( } asyncResponse.resume(topicList); }).exceptionally(ex -> { - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to list topics on namespace bundle {}/{}", clientAppId(), namespaceName, bundleRange, ex); } @@ -477,7 +477,7 @@ public void getListFromBundle( }); } }).exceptionally(ex -> { - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to list topics on namespace bundle {}/{}", clientAppId(), namespaceName, bundleRange, ex); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java index 667d8ce581ece..b2a3ae8cdb935 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java @@ -105,7 +105,7 @@ public void getTransactionInBufferStats(@Suspended final AsyncResponse asyncResp Long.parseLong(leastSigBits)) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get transaction state in transaction buffer {}", clientAppId(), topicName, ex); } @@ -143,7 +143,7 @@ public void getTransactionInPendingAckStats(@Suspended final AsyncResponse async Long.parseLong(leastSigBits), subName) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get transaction state in pending ack {}", clientAppId(), topicName, ex); } @@ -181,7 +181,7 @@ public void getTransactionBufferStats(@Suspended final AsyncResponse asyncRespon internalGetTransactionBufferStats(authoritative, lowWaterMarks, segmentStats) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get transaction buffer stats in topic {}", clientAppId(), topicName, ex); } @@ -217,7 +217,7 @@ public void getPendingAckStats(@Suspended final AsyncResponse asyncResponse, internalGetPendingAckStats(authoritative, subName, lowWaterMarks) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get transaction pending ack stats in topic {}", clientAppId(), topicName, ex); } @@ -314,7 +314,7 @@ public void getPendingAckInternalStats(@Suspended final AsyncResponse asyncRespo internalGetPendingAckInternalStats(authoritative, subName, metadata) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get pending ack internal stats {}", clientAppId(), topicName, ex); } @@ -365,7 +365,7 @@ public void getTransactionBufferInternalStats(@Suspended final AsyncResponse asy internalGetTransactionBufferInternalStats(authoritative, metadata) .thenAccept(asyncResponse::resume) .exceptionally(ex -> { - if (!isNot307And404Exception(ex)) { + if (isNot307And404Exception(ex)) { log.error("[{}] Failed to get transaction buffer internal stats {}", clientAppId(), topicName, ex); } From 08fcc87ccb99856d5ab3af9ddf1c4c2b0ea61a1e Mon Sep 17 00:00:00 2001 From: Paul Gier Date: Fri, 15 Mar 2024 09:46:33 -0500 Subject: [PATCH 339/699] [fix][broker] upgrade jclouds 2.5.0 -> 2.6.0 (#22220) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: 道君 Co-authored-by: Lari Hotari (cherry picked from commit 73dc213d4cec3513a1addbcb3518f441093c57ec) --- jclouds-shaded/pom.xml | 78 ++++++++++++++++++++++++++++++++---------- pom.xml | 2 +- 2 files changed, 60 insertions(+), 20 deletions(-) diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml index be69e0bf1c326..a37f99504de9f 100644 --- a/jclouds-shaded/pom.xml +++ b/jclouds-shaded/pom.xml @@ -33,8 +33,17 @@ jclouds-shaded Apache Pulsar :: Jclouds shaded + + + 2.10.1 + 32.0.0-jre + 7.0.0 + 2.0.1 + 3.0.0 + 2.0.0 + - org.apache.jclouds jclouds-allblobstore @@ -61,12 +70,48 @@ jclouds-slf4j ${jclouds.version} - - javax.annotation - javax.annotation-api - + + + + com.google.code.gson + gson + ${gson.version} + + + com.google.guava + guava + ${guava.version} + + + com.google.inject + guice + ${guice.version} + + + com.google.inject.extensions + guice-assistedinject + ${guice.version} + + + jakarta.inject + jakarta.inject-api + ${jakarta.inject.api.version} + + + jakarta.ws.rs + jakarta.ws.rs-api + ${jakarta.ws.rs-api.version} + + + jakarta.annotation + jakarta.annotation-api + ${jakarta.annotation-api.version} + + + + @@ -97,13 +142,13 @@ com.google.inject.extensions:guice-multibindings com.google.code.gson:gson org.apache.httpcomponents:* - javax.ws.rs:* com.jamesmurty.utils:* net.iharder:* aopalliance:* - javax.inject:* - javax.annotation:* com.google.errorprone:* + jakarta.inject:jakarta.inject-api + jakarta.annotation:jakarta.annotation-api + jakarta.ws.rs:jakarta.ws.rs-api @@ -112,10 +157,6 @@ com.google org.apache.pulsar.jcloud.shade.com.google - - javax.ws - org.apache.pulsar.jcloud.shade.javax.ws - com.jamesmurty.utils org.apache.pulsar.jcloud.shade.com.jamesmurty.utils @@ -129,18 +170,17 @@ org.apache.pulsar.jcloud.shade.net.iharder - javax.inject - org.apache.pulsar.jcloud.shade.javax.inject + com.google.errorprone + org.apache.pulsar.jcloud.shade.com.google.errorprone - javax.annotation - org.apache.pulsar.jcloud.shade.javax.annotation + jakarta + org.apache.pulsar.jcloud.shade.jakarta - com.google.errorprone - org.apache.pulsar.jcloud.shade.com.google.errorprone + org.aopalliance + org.apache.pulsar.jcloud.shade.org.aopalliance - diff --git a/pom.xml b/pom.xml index a4400f4b7fa8f..ffa5f1178ad5e 100644 --- a/pom.xml +++ b/pom.xml @@ -181,7 +181,7 @@ flexible messaging model and an intuitive client API. 1.12.262 1.11.3 2.10.10 - 2.5.0 + 2.6.0 5.1.0 3.42.0.0 8.0.11 From d0373e275628de1225b48c05f3e13b5cff2601cf Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Wed, 3 Apr 2024 21:01:49 -0700 Subject: [PATCH 340/699] [fix][broker] Update TransferShedder underloaded broker check to consider max loaded broker's msgThroughputEMA and update IsExtensibleLoadBalancerImpl check (#22321) (#22417) (cherry picked from commit 651908a903301da9c07dc93300635cc28d8ee69f) --- .../apache/pulsar/broker/PulsarService.java | 8 ++--- .../broker/admin/impl/NamespacesBase.java | 4 +-- .../extensions/ExtensibleLoadManagerImpl.java | 4 --- .../extensions/scheduler/TransferShedder.java | 22 +++++++++----- .../broker/namespace/NamespaceService.java | 30 +++++++++---------- .../pulsar/broker/web/PulsarWebResource.java | 4 +-- .../ExtensibleLoadManagerImplTest.java | 1 - .../scheduler/TransferShedderTest.java | 13 ++++---- 8 files changed, 45 insertions(+), 41 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 64f4ee0288122..5410bacbe78eb 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 @@ -379,7 +379,7 @@ public void closeMetadataServiceSession() throws Exception { } private void closeLeaderElectionService() throws Exception { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(this)) { ExtensibleLoadManagerImpl.get(loadManager.get()).getLeaderElectionService().close(); } else { if (this.leaderElectionService != null) { @@ -1135,7 +1135,7 @@ protected void closeLocalMetadataStore() throws Exception { } protected void startLeaderElectionService() { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(this)) { LOG.info("The load manager extension is enabled. Skipping PulsarService LeaderElectionService."); return; } @@ -1250,7 +1250,7 @@ protected void startLoadManagementService() throws PulsarServerException { LOG.info("Starting load management service ..."); this.loadManager.get().start(); - if (config.isLoadBalancerEnabled() && !ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (config.isLoadBalancerEnabled() && !ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(this)) { LOG.info("Starting load balancer"); if (this.loadReportTask == null) { long loadReportMinInterval = config.getLoadBalancerReportUpdateMinIntervalMillis(); @@ -1343,7 +1343,7 @@ public boolean isRunning() { * @return a reference of the current LeaderElectionService instance. */ public LeaderElectionService getLeaderElectionService() { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(this)) { return ExtensibleLoadManagerImpl.get(loadManager.get()).getLeaderElectionService(); } else { return this.leaderElectionService; 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 9478857032f22..f4732cad38040 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 @@ -968,13 +968,13 @@ public CompletableFuture setNamespaceBundleAffinityAsync(String bundleRang return CompletableFuture.completedFuture(null); }) .thenCompose(__ -> { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config())) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar())) { return CompletableFuture.completedFuture(null); } return validateLeaderBrokerAsync(); }) .thenAccept(__ -> { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config())) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar())) { return; } // For ExtensibleLoadManager, this operation will be ignored. 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 6a0e677c66268..26ee45b7444f3 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 @@ -263,10 +263,6 @@ public ExtensibleLoadManagerImpl() { this.brokerSelectionStrategy = new LeastResourceUsageWithWeight(); } - public static boolean isLoadManagerExtensionEnabled(ServiceConfiguration conf) { - return ExtensibleLoadManagerImpl.class.getName().equals(conf.getLoadManagerClassName()); - } - public static boolean isLoadManagerExtensionEnabled(PulsarService pulsar) { return pulsar.getLoadManager().get() instanceof ExtensibleLoadManagerWrapper; } 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 3564b4e9e3b94..7126ccb034196 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 @@ -362,7 +362,7 @@ public Set findBundlesForUnloading(LoadManagerContext context, final double targetStd = conf.getLoadBalancerBrokerLoadTargetStd(); boolean transfer = conf.isLoadBalancerTransferEnabled(); if (stats.std() > targetStd - || isUnderLoaded(context, stats.peekMinBroker(), stats.avg) + || isUnderLoaded(context, stats.peekMinBroker(), stats) || isOverLoaded(context, stats.peekMaxBroker(), stats.avg)) { unloadConditionHitCount++; } else { @@ -390,7 +390,7 @@ public Set findBundlesForUnloading(LoadManagerContext context, UnloadDecision.Reason reason; if (stats.std() > targetStd) { reason = Overloaded; - } else if (isUnderLoaded(context, stats.peekMinBroker(), stats.avg)) { + } else if (isUnderLoaded(context, stats.peekMinBroker(), stats)) { reason = Underloaded; if (debugMode) { log.info(String.format("broker:%s is underloaded:%s although " @@ -669,19 +669,27 @@ public Set findBundlesForUnloading(LoadManagerContext context, } - private boolean isUnderLoaded(LoadManagerContext context, String broker, double avgLoad) { + private boolean isUnderLoaded(LoadManagerContext context, String broker, LoadStats stats) { var brokerLoadDataOptional = context.brokerLoadDataStore().get(broker); if (brokerLoadDataOptional.isEmpty()) { return false; } var brokerLoadData = brokerLoadDataOptional.get(); - if (brokerLoadData.getMsgThroughputEMA() < 1) { + + var underLoadedMultiplier = + Math.min(0.5, Math.max(0.0, context.brokerConfiguration().getLoadBalancerBrokerLoadTargetStd() / 2.0)); + + if (brokerLoadData.getWeightedMaxEMA() < stats.avg * underLoadedMultiplier) { return true; } - return brokerLoadData.getWeightedMaxEMA() - < avgLoad * Math.min(0.5, Math.max(0.0, - context.brokerConfiguration().getLoadBalancerBrokerLoadTargetStd() / 2)); + var maxBrokerLoadDataOptional = context.brokerLoadDataStore().get(stats.peekMaxBroker()); + if (maxBrokerLoadDataOptional.isEmpty()) { + return false; + } + + return brokerLoadData.getMsgThroughputEMA() + < maxBrokerLoadDataOptional.get().getMsgThroughputEMA() * underLoadedMultiplier; } private boolean isOverLoaded(LoadManagerContext context, String broker, double avgLoad) { 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 61e045ed304fd..e04be25fe499c 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 @@ -195,7 +195,7 @@ public CompletableFuture> getBrokerServiceUrlAsync(TopicN pulsar.getBrokerId(), optResult.get(), topic); return CompletableFuture.completedFuture(optResult); } - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return loadManager.get().findBrokerServiceUrl(Optional.of(topic), bundle); } else { // TODO: Add unit tests cover it. @@ -311,7 +311,7 @@ private CompletableFuture> internalGetWebServiceUrl(@Nullable Serv return CompletableFuture.completedFuture(Optional.empty()); } CompletableFuture> future = - ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config) + ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar) ? loadManager.get().findBrokerServiceUrl(Optional.ofNullable(topic), bundle) : findBrokerServiceUrl(bundle, options); @@ -375,7 +375,7 @@ public boolean registerNamespace(NamespaceName nsname, boolean ensureOwned) thro NamespaceBundle nsFullBundle = bundleFactory.getFullBundle(nsname); // v2 namespace will always use full bundle object final NamespaceEphemeralData otherData; - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { ExtensibleLoadManagerImpl loadManager = ExtensibleLoadManagerImpl.get(this.loadManager.get()); otherData = loadManager.tryAcquiringOwnership(nsFullBundle).get(); } else { @@ -781,7 +781,7 @@ public CompletableFuture unloadNamespaceBundle(NamespaceBundle bundle, long timeout, TimeUnit timeoutUnit, boolean closeWithoutWaitingClientDisconnect) { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return ExtensibleLoadManagerImpl.get(loadManager.get()) .unloadNamespaceBundleAsync(bundle, destinationBroker); } @@ -803,7 +803,7 @@ public CompletableFuture> getOwnedNameSpac .getIsolationDataPoliciesAsync(pulsar.getConfiguration().getClusterName()) .thenApply(nsIsolationPoliciesOpt -> nsIsolationPoliciesOpt.orElseGet(NamespaceIsolationPolicies::new)) .thenCompose(namespaceIsolationPolicies -> { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); var statusMap = extensibleLoadManager.getOwnedServiceUnits().stream() @@ -883,7 +883,7 @@ public boolean isNamespaceBundleDisabled(NamespaceBundle bundle) throws Exceptio public CompletableFuture splitAndOwnBundle(NamespaceBundle bundle, boolean unload, NamespaceBundleSplitAlgorithm splitAlgorithm, List boundaries) { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return ExtensibleLoadManagerImpl.get(loadManager.get()) .splitNamespaceBundleAsync(bundle, splitAlgorithm, boundaries); } @@ -1126,7 +1126,7 @@ public OwnershipCache getOwnershipCache() { } public Set getOwnedServiceUnits() { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); return extensibleLoadManager.getOwnedServiceUnits(); } @@ -1148,7 +1148,7 @@ public CompletableFuture isServiceUnitOwnedAsync(ServiceUnitId suName) } if (suName instanceof NamespaceBundle) { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return loadManager.get().checkOwnershipAsync(Optional.empty(), suName); } // TODO: Add unit tests cover it. @@ -1176,7 +1176,7 @@ public boolean isServiceUnitActive(TopicName topicName) { public CompletableFuture isServiceUnitActiveAsync(TopicName topicName) { // TODO: Add unit tests cover it. - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return getBundleAsync(topicName) .thenCompose(bundle -> loadManager.get().checkOwnershipAsync(Optional.of(topicName), bundle)); } @@ -1191,7 +1191,7 @@ public CompletableFuture isServiceUnitActiveAsync(TopicName topicName) private CompletableFuture isNamespaceOwnedAsync(NamespaceName fqnn) { // TODO: Add unit tests cover it. - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return getFullBundleAsync(fqnn) .thenCompose(bundle -> loadManager.get().checkOwnershipAsync(Optional.empty(), bundle)); } @@ -1201,7 +1201,7 @@ private CompletableFuture isNamespaceOwnedAsync(NamespaceName fqnn) { private CompletableFuture isTopicOwnedAsync(TopicName topic) { // TODO: Add unit tests cover it. - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return getBundleAsync(topic) .thenCompose(bundle -> loadManager.get().checkOwnershipAsync(Optional.of(topic), bundle)); } @@ -1210,7 +1210,7 @@ private CompletableFuture isTopicOwnedAsync(TopicName topic) { public CompletableFuture checkTopicOwnership(TopicName topicName) { // TODO: Add unit tests cover it. - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return getBundleAsync(topicName) .thenCompose(bundle -> loadManager.get().checkOwnershipAsync(Optional.of(topicName), bundle)); } @@ -1220,7 +1220,7 @@ public CompletableFuture checkTopicOwnership(TopicName topicName) { public CompletableFuture removeOwnedServiceUnitAsync(NamespaceBundle nsBundle) { CompletableFuture future; - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); future = extensibleLoadManager.unloadNamespaceBundleAsync(nsBundle, Optional.empty()); } else { @@ -1528,7 +1528,7 @@ public PulsarClientImpl getNamespaceClient(ClusterDataImpl cluster) { } public CompletableFuture> getOwnerAsync(NamespaceBundle bundle) { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); return extensibleLoadManager.getOwnershipWithLookupDataAsync(bundle) .thenCompose(lookupData -> lookupData @@ -1545,7 +1545,7 @@ public boolean checkOwnershipPresent(NamespaceBundle bundle) throws Exception { } public CompletableFuture checkOwnershipPresentAsync(NamespaceBundle bundle) { - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); return extensibleLoadManager.getOwnershipAsync(Optional.empty(), bundle) .thenApply(Optional::isPresent); 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 0d01a2c50419d..2f437962002a3 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 @@ -732,7 +732,7 @@ public CompletableFuture validateBundleOwnershipAsync(NamespaceBundle bund .host(webUrl.get().getHost()) .port(webUrl.get().getPort()) .replaceQueryParam("authoritative", newAuthoritative); - if (!ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(config())) { + if (!ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { uriBuilder.replaceQueryParam("destinationBroker", null); } URI redirect = uriBuilder.build(); @@ -1007,7 +1007,7 @@ protected boolean isLeaderBroker() { protected static boolean isLeaderBroker(PulsarService pulsar) { // For extensible load manager, it doesn't have leader election service on pulsar broker. - if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar.getConfig())) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return true; } return pulsar.getLeaderElectionService().isLeader(); 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 9c363325c4cc8..03ea937c44edc 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 @@ -317,7 +317,6 @@ public boolean test(NamespaceBundle namespaceBundle) { assertEquals(unloadCount.get(), 1); }); - String dstBrokerUrl = pulsar1.getBrokerId(); String dstBrokerServiceUrl; if (broker.equals(pulsar1.getBrokerServiceUrl())) { 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 0ff64616973d9..efca2880949f2 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 @@ -1104,16 +1104,17 @@ public void testUnloadBundlesGreaterThanTargetThroughputAfterSplit() throws Ille assertEquals(stats.std(), 2.5809568279517847E-8); } - @Test - public void testMinBrokerWithZeroTraffic() throws IllegalAccessException { + public void testMinBrokerWithLowTraffic() throws IllegalAccessException { UnloadCounter counter = new UnloadCounter(); TransferShedder transferShedder = new TransferShedder(counter); var ctx = setupContext(); var brokerLoadDataStore = ctx.brokerLoadDataStore(); - var load = getCpuLoad(ctx, 4, "broker2:8080"); - FieldUtils.writeDeclaredField(load,"msgThroughputEMA", 0, true); + var load = getCpuLoad(ctx, 4, "broker2:8080"); + FieldUtils.writeDeclaredField(load, "msgThroughputEMA", 10, true); + + brokerLoadDataStore.pushAsync("broker2:8080", load); brokerLoadDataStore.pushAsync("broker4:8080", getCpuLoad(ctx, 55, "broker4:8080")); brokerLoadDataStore.pushAsync("broker5:8080", getCpuLoad(ctx, 65, "broker5:8080")); @@ -1268,10 +1269,10 @@ public void testOverloadOutlier() { Assertions.assertThat(res).isIn( Set.of(new UnloadDecision( new Unload("broker99:8080", "my-tenant/my-namespace99/0x00000000_0x0FFFFFFF", - Optional.of("broker52:8080")), Success, Overloaded)), + Optional.of("broker52:8080")), Success, Underloaded)), Set.of(new UnloadDecision( new Unload("broker99:8080", "my-tenant/my-namespace99/0x00000000_0x0FFFFFFF", - Optional.of("broker83:8080")), Success, Overloaded)) + Optional.of("broker83:8080")), Success, Underloaded)) ); assertEquals(counter.getLoadAvg(), 0.019900000000000008, 0.00001); assertEquals(counter.getLoadStd(), 0.09850375627355534, 0.00001); From 7ab68ca3dc8dc216b60372f58b594a96c6dd5597 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 27 Mar 2024 12:34:14 -0700 Subject: [PATCH 341/699] [improve][misc] Upgrade to Netty 4.1.108 and tcnative 2.0.65 (#22369) (cherry picked from commit be0a9d9d9bb23dabc065f091b853f27c0ebcaa16) (cherry picked from commit 1987398a19df03743f636ae7493ae92cc8cd2e82) --- buildtools/pom.xml | 2 +- .../server/src/assemble/LICENSE.bin.txt | 54 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 52 +++++++++--------- pom.xml | 2 +- pulsar-sql/presto-distribution/LICENSE | 54 +++++++++---------- 5 files changed, 82 insertions(+), 82 deletions(-) diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 56002fe068bef..61c56f65a2ab3 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -47,7 +47,7 @@ 4.1 10.14.2 3.1.2 - 4.1.100.Final + 4.1.108.Final 4.2.3 32.1.1-jre 1.10.12 diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index b7da9969a4bf8..d42c7cc9fe1bd 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -289,33 +289,33 @@ The Apache Software License, Version 2.0 - org.apache.commons-commons-lang3-3.11.jar - org.apache.commons-commons-text-1.10.0.jar * Netty - - io.netty-netty-buffer-4.1.100.Final.jar - - io.netty-netty-codec-4.1.100.Final.jar - - io.netty-netty-codec-dns-4.1.100.Final.jar - - io.netty-netty-codec-http-4.1.100.Final.jar - - io.netty-netty-codec-http2-4.1.100.Final.jar - - io.netty-netty-codec-socks-4.1.100.Final.jar - - io.netty-netty-codec-haproxy-4.1.100.Final.jar - - io.netty-netty-common-4.1.100.Final.jar - - io.netty-netty-handler-4.1.100.Final.jar - - io.netty-netty-handler-proxy-4.1.100.Final.jar - - io.netty-netty-resolver-4.1.100.Final.jar - - io.netty-netty-resolver-dns-4.1.100.Final.jar - - io.netty-netty-resolver-dns-classes-macos-4.1.100.Final.jar - - io.netty-netty-resolver-dns-native-macos-4.1.100.Final-osx-aarch_64.jar - - io.netty-netty-resolver-dns-native-macos-4.1.100.Final-osx-x86_64.jar - - io.netty-netty-transport-4.1.100.Final.jar - - io.netty-netty-transport-classes-epoll-4.1.100.Final.jar - - io.netty-netty-transport-native-epoll-4.1.100.Final-linux-x86_64.jar - - io.netty-netty-transport-native-unix-common-4.1.100.Final.jar - - io.netty-netty-transport-native-unix-common-4.1.100.Final-linux-x86_64.jar - - io.netty-netty-tcnative-boringssl-static-2.0.61.Final.jar - - io.netty-netty-tcnative-boringssl-static-2.0.61.Final-linux-aarch_64.jar - - io.netty-netty-tcnative-boringssl-static-2.0.61.Final-linux-x86_64.jar - - io.netty-netty-tcnative-boringssl-static-2.0.61.Final-osx-aarch_64.jar - - io.netty-netty-tcnative-boringssl-static-2.0.61.Final-osx-x86_64.jar - - io.netty-netty-tcnative-boringssl-static-2.0.61.Final-windows-x86_64.jar - - io.netty-netty-tcnative-classes-2.0.61.Final.jar + - io.netty-netty-buffer-4.1.108.Final.jar + - io.netty-netty-codec-4.1.108.Final.jar + - io.netty-netty-codec-dns-4.1.108.Final.jar + - io.netty-netty-codec-http-4.1.108.Final.jar + - io.netty-netty-codec-http2-4.1.108.Final.jar + - io.netty-netty-codec-socks-4.1.108.Final.jar + - io.netty-netty-codec-haproxy-4.1.108.Final.jar + - io.netty-netty-common-4.1.108.Final.jar + - io.netty-netty-handler-4.1.108.Final.jar + - io.netty-netty-handler-proxy-4.1.108.Final.jar + - io.netty-netty-resolver-4.1.108.Final.jar + - io.netty-netty-resolver-dns-4.1.108.Final.jar + - io.netty-netty-resolver-dns-classes-macos-4.1.108.Final.jar + - io.netty-netty-resolver-dns-native-macos-4.1.108.Final-osx-aarch_64.jar + - io.netty-netty-resolver-dns-native-macos-4.1.108.Final-osx-x86_64.jar + - io.netty-netty-transport-4.1.108.Final.jar + - io.netty-netty-transport-classes-epoll-4.1.108.Final.jar + - io.netty-netty-transport-native-epoll-4.1.108.Final-linux-x86_64.jar + - io.netty-netty-transport-native-unix-common-4.1.108.Final.jar + - io.netty-netty-transport-native-unix-common-4.1.108.Final-linux-x86_64.jar + - io.netty-netty-tcnative-boringssl-static-2.0.65.Final.jar + - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-linux-aarch_64.jar + - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-linux-x86_64.jar + - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-osx-aarch_64.jar + - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-osx-x86_64.jar + - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-windows-x86_64.jar + - io.netty-netty-tcnative-classes-2.0.65.Final.jar - io.netty.incubator-netty-incubator-transport-classes-io_uring-0.0.21.Final.jar - io.netty.incubator-netty-incubator-transport-native-io_uring-0.0.21.Final-linux-x86_64.jar - io.netty.incubator-netty-incubator-transport-native-io_uring-0.0.21.Final-linux-aarch_64.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 5d880a018c53f..df04985a3a880 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -344,35 +344,35 @@ The Apache Software License, Version 2.0 - commons-text-1.10.0.jar - commons-compress-1.26.0.jar * Netty - - netty-buffer-4.1.100.Final.jar - - netty-codec-4.1.100.Final.jar - - netty-codec-dns-4.1.100.Final.jar - - netty-codec-http-4.1.100.Final.jar - - netty-codec-socks-4.1.100.Final.jar - - netty-codec-haproxy-4.1.100.Final.jar - - netty-common-4.1.100.Final.jar - - netty-handler-4.1.100.Final.jar - - netty-handler-proxy-4.1.100.Final.jar - - netty-resolver-4.1.100.Final.jar - - netty-resolver-dns-4.1.100.Final.jar - - netty-transport-4.1.100.Final.jar - - netty-transport-classes-epoll-4.1.100.Final.jar - - netty-transport-native-epoll-4.1.100.Final-linux-x86_64.jar - - netty-transport-native-unix-common-4.1.100.Final.jar - - netty-transport-native-unix-common-4.1.100.Final-linux-x86_64.jar - - netty-tcnative-boringssl-static-2.0.61.Final.jar - - netty-tcnative-boringssl-static-2.0.61.Final-linux-aarch_64.jar - - netty-tcnative-boringssl-static-2.0.61.Final-linux-x86_64.jar - - netty-tcnative-boringssl-static-2.0.61.Final-osx-aarch_64.jar - - netty-tcnative-boringssl-static-2.0.61.Final-osx-x86_64.jar - - netty-tcnative-boringssl-static-2.0.61.Final-windows-x86_64.jar - - netty-tcnative-classes-2.0.61.Final.jar + - netty-buffer-4.1.108.Final.jar + - netty-codec-4.1.108.Final.jar + - netty-codec-dns-4.1.108.Final.jar + - netty-codec-http-4.1.108.Final.jar + - netty-codec-socks-4.1.108.Final.jar + - netty-codec-haproxy-4.1.108.Final.jar + - netty-common-4.1.108.Final.jar + - netty-handler-4.1.108.Final.jar + - netty-handler-proxy-4.1.108.Final.jar + - netty-resolver-4.1.108.Final.jar + - netty-resolver-dns-4.1.108.Final.jar + - netty-transport-4.1.108.Final.jar + - netty-transport-classes-epoll-4.1.108.Final.jar + - netty-transport-native-epoll-4.1.108.Final-linux-x86_64.jar + - netty-transport-native-unix-common-4.1.108.Final.jar + - netty-transport-native-unix-common-4.1.108.Final-linux-x86_64.jar + - netty-tcnative-boringssl-static-2.0.65.Final.jar + - netty-tcnative-boringssl-static-2.0.65.Final-linux-aarch_64.jar + - netty-tcnative-boringssl-static-2.0.65.Final-linux-x86_64.jar + - netty-tcnative-boringssl-static-2.0.65.Final-osx-aarch_64.jar + - netty-tcnative-boringssl-static-2.0.65.Final-osx-x86_64.jar + - netty-tcnative-boringssl-static-2.0.65.Final-windows-x86_64.jar + - netty-tcnative-classes-2.0.65.Final.jar - netty-incubator-transport-classes-io_uring-0.0.21.Final.jar - netty-incubator-transport-native-io_uring-0.0.21.Final-linux-aarch_64.jar - netty-incubator-transport-native-io_uring-0.0.21.Final-linux-x86_64.jar - - netty-resolver-dns-classes-macos-4.1.100.Final.jar - - netty-resolver-dns-native-macos-4.1.100.Final-osx-aarch_64.jar - - netty-resolver-dns-native-macos-4.1.100.Final-osx-x86_64.jar + - netty-resolver-dns-classes-macos-4.1.108.Final.jar + - netty-resolver-dns-native-macos-4.1.108.Final-osx-aarch_64.jar + - netty-resolver-dns-native-macos-4.1.108.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 ffa5f1178ad5e..33e979083ffe0 100644 --- a/pom.xml +++ b/pom.xml @@ -140,7 +140,7 @@ flexible messaging model and an intuitive client API. 1.1.10.5 4.1.12.1 5.1.0 - 4.1.100.Final + 4.1.108.Final 0.0.21.Final 9.4.54.v20240208 2.5.2 diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index df7887fed3e59..079b033c0942b 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -231,34 +231,34 @@ The Apache Software License, Version 2.0 - commons-compress-1.26.0.jar - commons-lang3-3.11.jar * Netty - - netty-buffer-4.1.100.Final.jar - - netty-codec-4.1.100.Final.jar - - netty-codec-dns-4.1.100.Final.jar - - netty-codec-http-4.1.100.Final.jar - - netty-codec-haproxy-4.1.100.Final.jar - - netty-codec-socks-4.1.100.Final.jar - - netty-handler-proxy-4.1.100.Final.jar - - netty-common-4.1.100.Final.jar - - netty-handler-4.1.100.Final.jar + - netty-buffer-4.1.108.Final.jar + - netty-codec-4.1.108.Final.jar + - netty-codec-dns-4.1.108.Final.jar + - netty-codec-http-4.1.108.Final.jar + - netty-codec-haproxy-4.1.108.Final.jar + - netty-codec-socks-4.1.108.Final.jar + - netty-handler-proxy-4.1.108.Final.jar + - netty-common-4.1.108.Final.jar + - netty-handler-4.1.108.Final.jar - netty-reactive-streams-2.0.6.jar - - netty-resolver-4.1.100.Final.jar - - netty-resolver-dns-4.1.100.Final.jar - - netty-resolver-dns-classes-macos-4.1.100.Final.jar - - netty-resolver-dns-native-macos-4.1.100.Final-osx-aarch_64.jar - - netty-resolver-dns-native-macos-4.1.100.Final-osx-x86_64.jar - - netty-tcnative-boringssl-static-2.0.61.Final.jar - - netty-tcnative-boringssl-static-2.0.61.Final-linux-aarch_64.jar - - netty-tcnative-boringssl-static-2.0.61.Final-linux-x86_64.jar - - netty-tcnative-boringssl-static-2.0.61.Final-osx-aarch_64.jar - - netty-tcnative-boringssl-static-2.0.61.Final-osx-x86_64.jar - - netty-tcnative-boringssl-static-2.0.61.Final-windows-x86_64.jar - - netty-tcnative-classes-2.0.61.Final.jar - - netty-transport-4.1.100.Final.jar - - netty-transport-classes-epoll-4.1.100.Final.jar - - netty-transport-native-epoll-4.1.100.Final-linux-x86_64.jar - - netty-transport-native-unix-common-4.1.100.Final.jar - - netty-transport-native-unix-common-4.1.100.Final-linux-x86_64.jar - - netty-codec-http2-4.1.100.Final.jar + - netty-resolver-4.1.108.Final.jar + - netty-resolver-dns-4.1.108.Final.jar + - netty-resolver-dns-classes-macos-4.1.108.Final.jar + - netty-resolver-dns-native-macos-4.1.108.Final-osx-aarch_64.jar + - netty-resolver-dns-native-macos-4.1.108.Final-osx-x86_64.jar + - netty-tcnative-boringssl-static-2.0.65.Final.jar + - netty-tcnative-boringssl-static-2.0.65.Final-linux-aarch_64.jar + - netty-tcnative-boringssl-static-2.0.65.Final-linux-x86_64.jar + - netty-tcnative-boringssl-static-2.0.65.Final-osx-aarch_64.jar + - netty-tcnative-boringssl-static-2.0.65.Final-osx-x86_64.jar + - netty-tcnative-boringssl-static-2.0.65.Final-windows-x86_64.jar + - netty-tcnative-classes-2.0.65.Final.jar + - netty-transport-4.1.108.Final.jar + - netty-transport-classes-epoll-4.1.108.Final.jar + - netty-transport-native-epoll-4.1.108.Final-linux-x86_64.jar + - netty-transport-native-unix-common-4.1.108.Final.jar + - netty-transport-native-unix-common-4.1.108.Final-linux-x86_64.jar + - netty-codec-http2-4.1.108.Final.jar - netty-incubator-transport-classes-io_uring-0.0.21.Final.jar - netty-incubator-transport-native-io_uring-0.0.21.Final-linux-x86_64.jar - netty-incubator-transport-native-io_uring-0.0.21.Final-linux-aarch_64.jar From d18f429933c0d3eedb8f75f6abc94e45fb008b05 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 4 Apr 2024 11:25:16 -0700 Subject: [PATCH 342/699] [improve][test] Move most flaky tests to flaky group (#22433) - also add solution for running test methods added to flaky group since that was missing (cherry picked from commit 5f31ec383bb7526eca24b95002f6cd498057fee7) # Conflicts: # pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java # pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java (cherry picked from commit 097805d68eb45504da31e1cd0444cea4147c8d4b) --- build/run_unit_group.sh | 14 +++++++++--- .../pulsar/tests/AnnotationListener.java | 22 +++++++++++++++++++ .../mledger/impl/ManagedLedgerTest.java | 4 ++-- pom.xml | 2 +- .../admin/AdminApiMultiBrokersTest.java | 2 +- .../pulsar/broker/admin/TopicAuthZTest.java | 4 ++-- .../service/PersistentMessageFinderTest.java | 6 ++--- .../impl/ProducerConsumerInternalTest.java | 2 +- .../client/impl/TransactionEndToEndTest.java | 4 ++-- ...ctionEndToEndWithoutBatchIndexAckTest.java | 2 +- 10 files changed, 46 insertions(+), 16 deletions(-) diff --git a/build/run_unit_group.sh b/build/run_unit_group.sh index 8597848b69bf8..b6275903be6f0 100755 --- a/build/run_unit_group.sh +++ b/build/run_unit_group.sh @@ -135,13 +135,21 @@ function print_testng_failures() { function test_group_broker_flaky() { echo "::endgroup::" echo "::group::Running quarantined tests" - mvn_test --no-fail-fast -pl pulsar-broker -Dgroups='quarantine' -DexcludedGroups='' -DfailIfNoTests=false \ + mvn_test --no-fail-fast -pl pulsar-broker -Dgroups='quarantine' -DexcludedGroups='flaky' -DfailIfNoTests=false \ -DtestForkCount=2 || print_testng_failures pulsar-broker/target/surefire-reports/testng-failed.xml "Quarantined test failure in" "Quarantined test failures" echo "::endgroup::" echo "::group::Running flaky tests" - mvn_test --no-fail-fast -pl pulsar-broker -Dgroups='flaky' -DtestForkCount=2 + mvn_test --no-fail-fast -pl pulsar-broker -Dgroups='flaky' -DexcludedGroups='quarantine' -DtestForkCount=2 echo "::endgroup::" + local modules_with_flaky_tests=$(git grep -l '@Test.*"flaky"' | grep '/src/test/java/' | \ + awk -F '/src/test/java/' '{ print $1 }' | grep -v -E 'pulsar-broker' | sort | uniq | \ + perl -0777 -p -e 's/\n(\S)/,$1/g') + if [ -n "${modules_with_flaky_tests}" ]; then + echo "::group::Running flaky tests in modules '${modules_with_flaky_tests}'" + mvn_test --no-fail-fast -pl "${modules_with_flaky_tests}" -Dgroups='flaky' -DexcludedGroups='quarantine' -DfailIfNoTests=false + echo "::endgroup::" + fi } function test_group_proxy() { @@ -175,7 +183,7 @@ function test_group_other() { perl -0777 -p -e 's/\n(\S)/,$1/g') if [ -n "${modules_with_quarantined_tests}" ]; then echo "::group::Running quarantined tests outside of pulsar-broker & pulsar-proxy (if any)" - mvn_test --no-fail-fast -pl "${modules_with_quarantined_tests}" test -Dgroups='quarantine' -DexcludedGroups='' \ + mvn_test --no-fail-fast -pl "${modules_with_quarantined_tests}" test -Dgroups='quarantine' -DexcludedGroups='flaky' \ -DfailIfNoTests=false || \ echo "::warning::There were test failures in the 'quarantine' test group." echo "::endgroup::" diff --git a/buildtools/src/main/java/org/apache/pulsar/tests/AnnotationListener.java b/buildtools/src/main/java/org/apache/pulsar/tests/AnnotationListener.java index 38cd2a1747a63..0c464fd97a970 100644 --- a/buildtools/src/main/java/org/apache/pulsar/tests/AnnotationListener.java +++ b/buildtools/src/main/java/org/apache/pulsar/tests/AnnotationListener.java @@ -32,6 +32,10 @@ public class AnnotationListener implements IAnnotationTransformer { private static final long DEFAULT_TEST_TIMEOUT_MILLIS = TimeUnit.MINUTES.toMillis(5); private static final String OTHER_GROUP = "other"; + private static final String FLAKY_GROUP = "flaky"; + + private static final String QUARANTINE_GROUP = "quarantine"; + public AnnotationListener() { System.out.println("Created annotation listener"); } @@ -51,9 +55,27 @@ public void transform(ITestAnnotation annotation, annotation.setTimeOut(DEFAULT_TEST_TIMEOUT_MILLIS); } + replaceGroupsIfFlakyOrQuarantineGroupIsIncluded(annotation); addToOtherGroupIfNoGroupsSpecified(annotation); } + // A test method will inherit the test groups from the class level and this solution ensures that a test method + // added to the flaky or quarantine group will not be executed as part of other groups. + private void replaceGroupsIfFlakyOrQuarantineGroupIsIncluded(ITestAnnotation annotation) { + if (annotation.getGroups() != null && annotation.getGroups().length > 1) { + for (String group : annotation.getGroups()) { + if (group.equals(QUARANTINE_GROUP)) { + annotation.setGroups(new String[]{QUARANTINE_GROUP}); + return; + } + if (group.equals(FLAKY_GROUP)) { + annotation.setGroups(new String[]{FLAKY_GROUP}); + return; + } + } + } + } + private void addToOtherGroupIfNoGroupsSpecified(ITestOrConfiguration annotation) { // Add test to "other" group if there's no specified group if (annotation.getGroups() == null || annotation.getGroups().length == 0) { 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 34f65dfd00ee8..94692aeea976a 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 @@ -2440,7 +2440,7 @@ public void testRetentionSize() throws Exception { }); } - @Test + @Test(groups = "flaky") public void testTimestampOnWorkingLedger() throws Exception { ManagedLedgerConfig conf = new ManagedLedgerConfig(); conf.setMaxEntriesPerLedger(1); @@ -3505,7 +3505,7 @@ public void testLedgerReachMaximumRolloverTime() throws Exception { .until(() -> firstLedgerId != ml.addEntry("test".getBytes()).getLedgerId()); } - @Test + @Test(groups = "flaky") public void testLedgerNotRolloverWithoutOpenState() throws Exception { ManagedLedgerConfig config = new ManagedLedgerConfig(); config.setMaxEntriesPerLedger(2); diff --git a/pom.xml b/pom.xml index 33e979083ffe0..27243281bd98a 100644 --- a/pom.xml +++ b/pom.xml @@ -88,7 +88,7 @@ flexible messaging model and an intuitive client API. **/Test*.java,**/*Test.java,**/*Tests.java,**/*TestCase.java - quarantine + quarantine,flaky UTF-8 UTF-8 diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiMultiBrokersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiMultiBrokersTest.java index 7c9154a27ff69..46b24abd6d4e2 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiMultiBrokersTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminApiMultiBrokersTest.java @@ -132,7 +132,7 @@ public void testTopicLookup(TopicDomain topicDomain, boolean isPartition) throws Assert.assertEquals(lookupResultSet.size(), 1); } - @Test + @Test(groups = "flaky") public void testForceDeletePartitionedTopicWithSub() throws Exception { final int numPartitions = 10; TenantInfoImpl tenantInfo = new TenantInfoImpl(Set.of("role1", "role2"), Set.of("test")); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java index e23f9bbaf9b30..9e975ba0854d4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAuthZTest.java @@ -53,7 +53,7 @@ public class TopicAuthZTest extends MockedPulsarStandalone { .claim("sub", TENANT_ADMIN_SUBJECT).signWith(SECRET_KEY).compact(); @SneakyThrows - @BeforeClass + @BeforeClass(alwaysRun = true) public void before() { configureTokenAuthentication(); configureDefaultAuthorization(); @@ -73,7 +73,7 @@ public void before() { @SneakyThrows - @AfterClass + @AfterClass(alwaysRun = true) public void after() { if (superUserAdmin != null) { superUserAdmin.close(); 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 ace552a55a72a..ac5ab94c213c5 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 @@ -266,7 +266,7 @@ void testPersistentMessageFinderWhenLastMessageDelete() throws Exception { ledger.addEntry(createMessageWrittenToLedger("msg2")); ledger.addEntry(createMessageWrittenToLedger("msg3")); Position lastPosition = ledger.addEntry(createMessageWrittenToLedger("last-message")); - + long endTimestamp = System.currentTimeMillis() + 1000; Result result = new Result(); @@ -383,7 +383,7 @@ public static Set getBrokerEntryMetadataIntercep * * @throws Exception */ - @Test + @Test(groups = "flaky") void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { final String ledgerAndCursorName = "testPersistentMessageExpiryWithNonRecoverableLedgers"; @@ -440,7 +440,7 @@ void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { } - @Test + @Test(groups = "flaky") public void testIncorrectClientClock() throws Exception { final String ledgerAndCursorName = "testIncorrectClientClock"; int maxTTLSeconds = 1; 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 240d8d2304768..4ec8107030600 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 @@ -116,7 +116,7 @@ private void removeServiceProducerMaintainedByServerCnx(ServiceProducer serviceP }); } - @Test + @Test(groups = "flaky") public void testExclusiveConsumerWillAlwaysRetryEvenIfReceivedConsumerBusyError() throws Exception { final String topicName = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/tp_"); final String subscriptionName = "subscription1"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java index 348fb04b7dd23..4abcb09c0fe6d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndTest.java @@ -96,7 +96,7 @@ public class TransactionEndToEndTest extends TransactionTestBase { protected static final String TOPIC_OUTPUT = NAMESPACE1 + "/output"; protected static final String TOPIC_MESSAGE_ACK_TEST = NAMESPACE1 + "/message-ack-test"; protected static final int NUM_PARTITIONS = 16; - @BeforeClass + @BeforeClass(alwaysRun = true) protected void setup() throws Exception { conf.setAcknowledgmentAtBatchIndexLevelEnabled(true); setUpBase(1, NUM_PARTITIONS, TOPIC_OUTPUT, TOPIC_PARTITION); @@ -1624,7 +1624,7 @@ public void testSendTxnAckBatchMessageToDLQ() throws Exception { admin.topics().delete(topic, true); } - @Test + @Test(groups = "flaky") public void testDelayedTransactionMessages() throws Exception { String topic = NAMESPACE1 + "/testDelayedTransactionMessages"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndWithoutBatchIndexAckTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndWithoutBatchIndexAckTest.java index 52faae2f8ea1f..df4ad32b6a8ae 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndWithoutBatchIndexAckTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TransactionEndToEndWithoutBatchIndexAckTest.java @@ -30,7 +30,7 @@ @Test(groups = "broker-impl") public class TransactionEndToEndWithoutBatchIndexAckTest extends TransactionEndToEndTest { - @BeforeClass + @BeforeClass(alwaysRun = true) protected void setup() throws Exception { conf.setAcknowledgmentAtBatchIndexLevelEnabled(false); setUpBase(1, NUM_PARTITIONS, TOPIC_OUTPUT, TOPIC_PARTITION); From 4e10b226a2a2ce163e0d67fc5b101a96c44b4d8a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Fri, 5 Apr 2024 15:42:40 +0800 Subject: [PATCH 343/699] [fix][broker][admin] Fix cannot update properties on NonDurable subscription. (#22411) (cherry picked from commit 902728ef6590233b87c14d2528590ad7e6fdcc12) # Conflicts: # pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PersistentTopicsTest.java (cherry picked from commit 7cd0924bc9996f95b36601958ae156408614c523) --- .../mledger/impl/ManagedCursorImpl.java | 10 +++-- .../broker/admin/PersistentTopicsTest.java | 38 +++++++++++++++++++ 2 files changed, 45 insertions(+), 3 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 8db8a5714398c..f67f534f86d45 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 @@ -350,15 +350,19 @@ private CompletableFuture computeCursorProperties( final Function, Map> updateFunction) { CompletableFuture updateCursorPropertiesResult = new CompletableFuture<>(); - final Stat lastCursorLedgerStat = ManagedCursorImpl.this.cursorLedgerStat; - Map newProperties = updateFunction.apply(ManagedCursorImpl.this.cursorProperties); + if (!isDurable()) { + this.cursorProperties = Collections.unmodifiableMap(newProperties); + updateCursorPropertiesResult.complete(null); + return updateCursorPropertiesResult; + } + ManagedCursorInfo copy = ManagedCursorInfo .newBuilder(ManagedCursorImpl.this.managedCursorInfo) .clearCursorProperties() .addAllCursorProperties(buildStringPropertiesMap(newProperties)) .build(); - + final Stat lastCursorLedgerStat = ManagedCursorImpl.this.cursorLedgerStat; ledger.getStore().asyncUpdateCursorInfo(ledger.getName(), name, copy, lastCursorLedgerStat, new MetaStoreCallback<>() { @Override 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 1f755234009ba..c7b066faea9da 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 @@ -32,6 +32,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertSame; import static org.testng.Assert.assertTrue; import java.lang.reflect.Field; @@ -53,6 +54,8 @@ import javax.ws.rs.core.UriInfo; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.commons.collections4.MapUtils; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.admin.v2.ExtPersistentTopics; import org.apache.pulsar.broker.admin.v2.NonPersistentTopics; @@ -65,6 +68,8 @@ import org.apache.pulsar.broker.resources.TopicResources; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.persistent.PersistentSubscription; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.web.PulsarWebResource; import org.apache.pulsar.broker.web.RestException; import org.apache.pulsar.client.admin.LongRunningProcessStatus; @@ -76,6 +81,7 @@ 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.Reader; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.interceptor.ProducerInterceptor; import org.apache.pulsar.client.impl.BatchMessageIdImpl; @@ -1773,4 +1779,36 @@ public void testNamespaceResources() throws Exception { assertTrue(namespaces.contains(ns1V2)); assertTrue(namespaces.contains(ns1V1)); } + + @Test + public void testUpdatePropertiesOnNonDurableSub() throws Exception { + String topic = "persistent://" + testTenant + "/" + testNamespaceLocal + "/testUpdatePropertiesOnNonDurableSub"; + String subscription = "sub"; + admin.topics().createNonPartitionedTopic(topic); + + @Cleanup + Reader __ = pulsarClient.newReader(Schema.STRING) + .startMessageId(MessageId.earliest) + .subscriptionName(subscription) + .topic(topic) + .create(); + + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).get().get(); + PersistentSubscription subscription1 = persistentTopic.getSubscriptions().get(subscription); + assertNotNull(subscription1); + ManagedCursor cursor = subscription1.getCursor(); + + Map properties = admin.topics().getSubscriptionProperties(topic, subscription); + assertEquals(properties.size(), 0); + assertTrue(MapUtils.isEmpty(cursor.getCursorProperties())); + + admin.topics().updateSubscriptionProperties(topic, subscription, Map.of("foo", "bar")); + properties = admin.topics().getSubscriptionProperties(topic, subscription); + assertEquals(properties.size(), 1); + assertEquals(properties.get("foo"), "bar"); + + assertEquals(cursor.getCursorProperties().size(), 1); + assertEquals(cursor.getCursorProperties().get("foo"), "bar"); + } } From a20af42da6529d358558522ee0d411bd671d0a52 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 22 Feb 2024 14:42:26 +0200 Subject: [PATCH 344/699] [fix][build] Fix building docker images without setting UBUNTU_MIRROR - similar change was made in master as part of #22095 (cherry picked from commit c27bcf000322720601f2ab8308684fed703ce546) Conflicts: pom.xml --- docker/pulsar/pom.xml | 5 ---- pom.xml | 29 +++++++++++++++++++++ tests/docker-images/java-test-image/pom.xml | 4 +-- 3 files changed, 31 insertions(+), 7 deletions(-) diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 9873f2c6bec06..eab6428939700 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -47,11 +47,6 @@ - - mirror://mirrors.ubuntu.com/mirrors.txt - http://security.ubuntu.com/ubuntu/ - - git-commit-id-no-git diff --git a/pom.xml b/pom.xml index 27243281bd98a..ccb127f2d1066 100644 --- a/pom.xml +++ b/pom.xml @@ -84,6 +84,9 @@ flexible messaging model and an intuitive client API. 3.4.0 + http://archive.ubuntu.com/ubuntu/ + http://security.ubuntu.com/ubuntu/ + **/Test*.java,**/*Test.java,**/*Tests.java,**/*TestCase.java @@ -2691,6 +2694,32 @@ 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} + + + diff --git a/tests/docker-images/java-test-image/pom.xml b/tests/docker-images/java-test-image/pom.xml index ec3cc1a6b4f49..26f13dd3783b3 100644 --- a/tests/docker-images/java-test-image/pom.xml +++ b/tests/docker-images/java-test-image/pom.xml @@ -35,8 +35,8 @@ docker target/pulsar-server-distribution-bin.tar.gz - ${env.UBUNTU_MIRROR} - ${env.UBUNTU_SECURITY_MIRROR} + ${UBUNTU_MIRROR} + ${UBUNTU_SECURITY_MIRROR} From 5b82329a24731b5c54a89366cf4f83ed840879f2 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 5 Apr 2024 09:14:22 -0700 Subject: [PATCH 345/699] [improve][misc] Specify valid home dir for the default user in the Ubuntu based docker image (#22446) (cherry picked from commit e008580f0fd00451fb9b05caf53757ba6539ba61) (cherry picked from commit c9b656ca8c4485c2e4a9a0d9af7f47a1f909c0cc) --- docker/pulsar/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/pulsar/Dockerfile b/docker/pulsar/Dockerfile index 10bd2b5ee2675..b899fb6182829 100644 --- a/docker/pulsar/Dockerfile +++ b/docker/pulsar/Dockerfile @@ -110,5 +110,5 @@ RUN chmod +x /pulsar/bin/install-pulsar-client.sh RUN /pulsar/bin/install-pulsar-client.sh # The UID must be non-zero. Otherwise, it is arbitrary. No logic should rely on its specific value. -RUN useradd ${DEFAULT_USERNAME} -u 10000 -g 0 +RUN useradd ${DEFAULT_USERNAME} -u 10000 -g 0 --no-create-home --home-dir /pulsar/data USER 10000 From 53b3dbd4af4f2b77a4440d0b3093dca9af7e4b57 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Mon, 8 Apr 2024 15:39:26 +0530 Subject: [PATCH 346/699] Disabled AlluxioSinkTest --- pom.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pom.xml b/pom.xml index ccb127f2d1066..5c5ddf881896f 100644 --- a/pom.xml +++ b/pom.xml @@ -1616,6 +1616,9 @@ flexible messaging model and an intuitive client API. org.apache.pulsar.tests.PulsarTestListener,org.apache.pulsar.tests.JacocoDumpListener,org.apache.pulsar.tests.AnnotationListener,org.apache.pulsar.tests.FailFastNotifier,org.apache.pulsar.tests.MockitoCleanupListener,org.apache.pulsar.tests.FastThreadLocalCleanupListener,org.apache.pulsar.tests.ThreadLeakDetectorListener,org.apache.pulsar.tests.SingletonCleanerListener + + **/AlluxioSink*.java + From c171310c685d8910867481851d2b3d0b18bbf3c2 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Tue, 9 Apr 2024 10:18:07 +0530 Subject: [PATCH 347/699] Release 3.1.3.1 --- 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-experimental/pom.xml | 2 +- docker/pulsar/pom.xml | 2 +- jclouds-shaded/pom.xml | 2 +- managed-ledger/pom.xml | 2 +- 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-client-1x-base/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-1x/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-2x-shaded/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/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/hdfs2/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-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-sql/pom.xml | 2 +- pulsar-sql/presto-distribution/pom.xml | 2 +- pulsar-sql/presto-pulsar-plugin/pom.xml | 2 +- pulsar-sql/presto-pulsar/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 +- 134 files changed, 136 insertions(+), 136 deletions(-) diff --git a/bouncy-castle/bc/pom.xml b/bouncy-castle/bc/pom.xml index fbe74aadf872c..edf470f59a747 100644 --- a/bouncy-castle/bc/pom.xml +++ b/bouncy-castle/bc/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.3.1-SNAPSHOT + 3.1.3.1 .. diff --git a/bouncy-castle/bcfips-include-test/pom.xml b/bouncy-castle/bcfips-include-test/pom.xml index 2f1b06d22bb80..e7d3e99a9e05c 100644 --- a/bouncy-castle/bcfips-include-test/pom.xml +++ b/bouncy-castle/bcfips-include-test/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.3.1-SNAPSHOT + 3.1.3.1 .. diff --git a/bouncy-castle/bcfips/pom.xml b/bouncy-castle/bcfips/pom.xml index 2495b92016533..92abef11b946f 100644 --- a/bouncy-castle/bcfips/pom.xml +++ b/bouncy-castle/bcfips/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.3.1-SNAPSHOT + 3.1.3.1 .. diff --git a/bouncy-castle/pom.xml b/bouncy-castle/pom.xml index fe50577c9d6f8..ed8307e0b6cda 100644 --- a/bouncy-castle/pom.xml +++ b/bouncy-castle/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.3.1-SNAPSHOT + 3.1.3.1 .. diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 61c56f65a2ab3..2d6d11edcc147 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -31,12 +31,12 @@ com.datastax.oss buildtools - 3.1.3.1-SNAPSHOT + 3.1.3.1 jar Pulsar Build Tools - 2024-03-06T15:14:31Z + 2024-04-09T04:48:06Z 1.8 1.8 3.1.0 diff --git a/distribution/io/pom.xml b/distribution/io/pom.xml index 21fb4967edca8..bcf5006cc4a7c 100644 --- a/distribution/io/pom.xml +++ b/distribution/io/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.3.1-SNAPSHOT + 3.1.3.1 .. diff --git a/distribution/offloaders/pom.xml b/distribution/offloaders/pom.xml index 7a0c70c396c5c..e9eebd37e52e1 100644 --- a/distribution/offloaders/pom.xml +++ b/distribution/offloaders/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.3.1-SNAPSHOT + 3.1.3.1 .. diff --git a/distribution/pom.xml b/distribution/pom.xml index caf071dbe2641..b49d0bc717f41 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.3.1-SNAPSHOT + 3.1.3.1 .. diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index d5e2399cf27bf..aa9f93561168f 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.3.1-SNAPSHOT + 3.1.3.1 .. diff --git a/distribution/shell/pom.xml b/distribution/shell/pom.xml index 3ccbebe113b1a..c833d57f808e4 100644 --- a/distribution/shell/pom.xml +++ b/distribution/shell/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.3.1-SNAPSHOT + 3.1.3.1 .. diff --git a/docker/pom.xml b/docker/pom.xml index bd517616c52a0..24165701d38ee 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.3.1-SNAPSHOT + 3.1.3.1 docker-images Apache Pulsar :: Docker Images diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index e6b2138586e93..fe09289d1325b 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.3.1-SNAPSHOT + 3.1.3.1 4.0.0 pulsar-all-docker-image diff --git a/docker/pulsar-experimental/pom.xml b/docker/pulsar-experimental/pom.xml index c2c60b533a2a5..451ff0f6cc915 100644 --- a/docker/pulsar-experimental/pom.xml +++ b/docker/pulsar-experimental/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.3.1-SNAPSHOT + 3.1.3.1 4.0.0 pulsar-experimental-docker-image diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index eab6428939700..fde09c73f8c96 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.3.1-SNAPSHOT + 3.1.3.1 4.0.0 pulsar-docker-image diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml index a37f99504de9f..18e11a330319b 100644 --- a/jclouds-shaded/pom.xml +++ b/jclouds-shaded/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.3.1-SNAPSHOT + 3.1.3.1 .. diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index 51c763da2bb03..2dab4591b71bd 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.3.1-SNAPSHOT + 3.1.3.1 .. diff --git a/pom.xml b/pom.xml index 5c5ddf881896f..22eed1c69db91 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ com.datastax.oss pulsar - 3.1.3.1-SNAPSHOT + 3.1.3.1 Pulsar Pulsar is a distributed pub-sub messaging platform with a very @@ -95,7 +95,7 @@ flexible messaging model and an intuitive client API. UTF-8 UTF-8 - 2024-03-06T15:14:31Z + 2024-04-09T04:48:06Z true - - io.grpc - grpc-netty - ${grpc.version} - - - - io.dropwizard.metrics - metrics-jvm - ${metrics.version} - - + + + + io.netty + netty-bom + ${netty.version} + pom + import + + + io.grpc + grpc-bom + ${grpc.version} + pom + import + + + io.dropwizard.metrics + metrics-jvm + ${metrics.version} + + + + From ddf77be02ebce985a72ff27ab97d825acd43106c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 28 Mar 2024 12:36:44 +0200 Subject: [PATCH 408/699] [improve][test][branch-3.0] Improve ManagedLedgerTest.testGetNumberOfEntriesInStorage - improve the test case based on suggestion from @poorbarcode (cherry picked from commit 8f17446355095d9d24d781446bd9d60ca4a275ef) --- .../apache/bookkeeper/mledger/impl/ManagedLedgerTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 eb3240d870931..a08b51cf29fc5 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 @@ -2640,10 +2640,10 @@ public void testGetNumberOfEntriesInStorage() throws Exception { Awaitility.await().untilAsserted(() -> { assertEquals(managedLedger.getState(), ManagedLedgerImpl.State.ClosedLedger); }); - managedLedger.rollCurrentLedgerIfFull(); + managedLedger.createLedgerAfterClosed(); Awaitility.await().untilAsserted(() -> { - assertEquals(managedLedger.getLedgersInfo().size(), 2); - assertEquals(managedLedger.getState(), ManagedLedgerImpl.State.ClosedLedger); + assertEquals(managedLedger.getLedgersInfo().size(), 3); + assertEquals(managedLedger.getState(), ManagedLedgerImpl.State.LedgerOpened); }); assertEquals(5, managedLedger.getLedgersInfoAsList().get(0).getEntries()); assertEquals(5, managedLedger.getLedgersInfoAsList().get(1).getEntries()); From 3eb5c2d8f49273613651a3ccdc1c122375153e4e Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 29 Mar 2024 12:06:26 +0800 Subject: [PATCH 409/699] [improve] [broker] Avoid repeated Read-and-discard when using Key_Shared mode (#22245) (cherry picked from commit e34ea626a65da4c8e1578010f857aa961a7b5c55) (cherry picked from commit 5b37e8434d93c135b7c36c76a8177e2b51db5556) --- .../MessageRedeliveryController.java | 8 + ...PersistentDispatcherMultipleConsumers.java | 47 +++- ...tStickyKeyDispatcherMultipleConsumers.java | 104 ++++++- .../client/api/KeySharedSubscriptionTest.java | 266 ++++++++++++++++++ .../client/api/ProducerConsumerBase.java | 66 +++++ 5 files changed, 470 insertions(+), 21 deletions(-) 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 5bf3f5506fa81..6380317724207 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 @@ -95,6 +95,14 @@ private void removeFromHashBlocker(long ledgerId, long entryId) { } } + public Long getHash(long ledgerId, long entryId) { + LongPair value = hashesToBeBlocked.get(ledgerId, entryId); + if (value == null) { + return null; + } + return value.first; + } + public void removeAllUpTo(long markDeleteLedgerId, long markDeleteEntryId) { if (!allowOutOfOrderDelivery) { List keysToRemove = new ArrayList<>(); 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 c164abf905dd2..9d0dba798ad88 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 @@ -309,24 +309,25 @@ public synchronized void readMoreEntries() { } NavigableSet messagesToReplayNow = getMessagesToReplayNow(messagesToRead); - - if (!messagesToReplayNow.isEmpty()) { + NavigableSet messagesToReplayFiltered = filterOutEntriesWillBeDiscarded(messagesToReplayNow); + if (!messagesToReplayFiltered.isEmpty()) { if (log.isDebugEnabled()) { - log.debug("[{}] Schedule replay of {} messages for {} consumers", name, messagesToReplayNow.size(), - consumerList.size()); + log.debug("[{}] Schedule replay of {} messages for {} consumers", name, + messagesToReplayFiltered.size(), consumerList.size()); } havePendingReplayRead = true; minReplayedPosition = messagesToReplayNow.first(); Set deletedMessages = topic.isDelayedDeliveryEnabled() - ? asyncReplayEntriesInOrder(messagesToReplayNow) : asyncReplayEntries(messagesToReplayNow); + ? asyncReplayEntriesInOrder(messagesToReplayFiltered) + : asyncReplayEntries(messagesToReplayFiltered); // clear already acked positions from replay bucket deletedMessages.forEach(position -> redeliveryMessages.remove(((PositionImpl) position).getLedgerId(), ((PositionImpl) 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 ((messagesToReplayNow.size() - deletedMessages.size()) == 0) { + if ((messagesToReplayFiltered.size() - deletedMessages.size()) == 0) { havePendingReplayRead = false; readMoreEntriesAsync(); } @@ -335,7 +336,7 @@ public synchronized void readMoreEntries() { log.debug("[{}] Dispatcher read is blocked due to unackMessages {} reached to max {}", name, totalUnackedMessages, topic.getMaxUnackedMessagesOnSubscription()); } - } else if (!havePendingRead) { + } else if (!havePendingRead && hasConsumersNeededNormalRead()) { if (log.isDebugEnabled()) { log.debug("[{}] Schedule read of {} messages for {} consumers", name, messagesToRead, consumerList.size()); @@ -364,7 +365,16 @@ public synchronized void readMoreEntries() { topic.getMaxReadPosition()); } } else { - log.debug("[{}] Cannot schedule next read until previous one is done", name); + 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()) { @@ -1106,6 +1116,27 @@ protected synchronized NavigableSet getMessagesToReplayNow(int max } } + /** + * 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 PersistentStickyKeyDispatcherMultipleConsumers#hasConsumersNeededNormalRead}. + */ + protected boolean hasConsumersNeededNormalRead() { + return true; + } + protected synchronized boolean shouldPauseDeliveryForDelayTracker() { return delayedDeliveryTracker.isPresent() && delayedDeliveryTracker.get().shouldPauseAllDeliveries(); } 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 8f05530f58bfa..ee2ebd7ca867e 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 @@ -30,13 +30,16 @@ 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.bookkeeper.mledger.impl.PositionImpl; +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; @@ -165,6 +168,14 @@ protected Map> initialValue() throws Exception { } }; + 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; @@ -248,15 +259,9 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis assert consumer != null; // checked when added to groupedEntries List entriesWithSameKey = current.getValue(); int entriesWithSameKeyCount = entriesWithSameKey.size(); - int availablePermits = Math.max(consumer.getAvailablePermits(), 0); - if (consumer.getMaxUnackedMessages() > 0) { - int remainUnAckedMessages = - // Avoid negative number - Math.max(consumer.getMaxUnackedMessages() - consumer.getUnackedMessages(), 0); - availablePermits = Math.min(availablePermits, remainUnAckedMessages); - } - int maxMessagesForC = Math.min(entriesWithSameKeyCount, availablePermits); - int messagesForC = getRestrictedMaxEntriesForConsumer(consumer, entriesWithSameKey, maxMessagesForC, + 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 {}", @@ -289,7 +294,6 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis 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(), @@ -332,8 +336,9 @@ protected synchronized boolean trySendMessagesToConsumers(ReadType readType, Lis return false; } - private int getRestrictedMaxEntriesForConsumer(Consumer consumer, List entries, int maxMessages, - ReadType readType, Set stickyKeyHashes) { + 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; } @@ -378,7 +383,7 @@ private int getRestrictedMaxEntriesForConsumer(Consumer consumer, List en // 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 (((PositionImpl) entries.get(i).getPosition()).compareTo(maxReadPosition) >= 0) { + if (((PositionImpl) 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; @@ -405,6 +410,9 @@ && removeConsumersFromRecentJoinedConsumers()) { } private boolean removeConsumersFromRecentJoinedConsumers() { + if (MapUtils.isEmpty(recentlyJoinedConsumers)) { + return false; + } Iterator> itr = recentlyJoinedConsumers.entrySet().iterator(); boolean hasConsumerRemovedFromTheRecentJoinedConsumers = false; PositionImpl mdp = (PositionImpl) cursor.getMarkDeletedPosition(); @@ -437,6 +445,76 @@ protected synchronized NavigableSet getMessagesToReplayNow(int max } } + 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) { + if (src.isEmpty()) { + return src; + } + NavigableSet res = new TreeSet<>(); + // Group positions. + final Map> groupedPositions = localGroupedPositions.get(); + groupedPositions.clear(); + for (PositionImpl 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() { + 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; 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 18fb141be3178..7219555050839 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 @@ -38,6 +38,7 @@ import java.util.Optional; import java.util.Random; import java.util.Set; +import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentSkipListSet; @@ -48,12 +49,17 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; import lombok.Cleanup; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.nonpersistent.NonPersistentStickyKeyDispatcherMultipleConsumers; 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.common.api.proto.KeySharedMode; import org.apache.pulsar.common.naming.TopicDomain; @@ -61,6 +67,7 @@ import org.apache.pulsar.common.schema.KeyValue; import org.apache.pulsar.common.util.Murmur3_32Hash; import org.awaitility.Awaitility; +import org.mockito.Mockito; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; @@ -1630,4 +1637,263 @@ public void testContinueDispatchMessagesWhenMessageDelayed() throws Exception { log.info("Got {} other messages...", sum); Assert.assertEquals(sum, delayedMessages + messages); } + + private AtomicInteger injectReplayReadCounter(String topicName, String cursorName) throws Exception { + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).join().get(); + ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + ManagedCursorImpl cursor = (ManagedCursorImpl) managedLedger.openCursor(cursorName); + managedLedger.getCursors().removeCursor(cursor.getName()); + managedLedger.getActiveCursors().removeCursor(cursor.getName()); + ManagedCursorImpl spyCursor = Mockito.spy(cursor); + managedLedger.getCursors().add(spyCursor, PositionImpl.EARLIEST); + managedLedger.getActiveCursors().add(spyCursor, PositionImpl.EARLIEST); + AtomicInteger replyReadCounter = new AtomicInteger(); + Mockito.doAnswer(invocation -> { + if (!String.valueOf(invocation.getArguments()[2]).equals("Normal")) { + replyReadCounter.incrementAndGet(); + } + return invocation.callRealMethod(); + }).when(spyCursor).asyncReplayEntries(Mockito.anySet(), Mockito.any(), Mockito.any()); + Mockito.doAnswer(invocation -> { + if (!String.valueOf(invocation.getArguments()[2]).equals("Normal")) { + replyReadCounter.incrementAndGet(); + } + return invocation.callRealMethod(); + }).when(spyCursor).asyncReplayEntries(Mockito.anySet(), Mockito.any(), Mockito.any(), Mockito.anyBoolean()); + admin.topics().createSubscription(topicName, cursorName, MessageId.earliest); + return replyReadCounter; + } + + @Test + public void testNoRepeatedReadAndDiscard() throws Exception { + int delayedMessages = 100; + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String subName = "my-sub"; + admin.topics().createNonPartitionedTopic(topic); + AtomicInteger replyReadCounter = injectReplayReadCounter(topic, subName); + + // Send messages. + @Cleanup + 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))) + .value(100 + i) + .send(); + log.info("Published message :{}", messageId); + } + producer.close(); + + // Make ack holes. + Consumer consumer1 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(10) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + Consumer consumer2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(10) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + List msgList1 = new ArrayList<>(); + List msgList2 = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + Message msg1 = consumer1.receive(1, TimeUnit.SECONDS); + if (msg1 != null) { + msgList1.add(msg1); + } + Message msg2 = consumer2.receive(1, TimeUnit.SECONDS); + if (msg2 != null) { + msgList2.add(msg2); + } + } + Consumer redeliverConsumer = null; + if (!msgList1.isEmpty()) { + msgList1.forEach(msg -> consumer1.acknowledgeAsync(msg)); + redeliverConsumer = consumer2; + } else { + msgList2.forEach(msg -> consumer2.acknowledgeAsync(msg)); + redeliverConsumer = consumer1; + } + + // consumer3 will be added to the "recentJoinedConsumers". + Consumer consumer3 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(1000) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + redeliverConsumer.close(); + + // 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); + } + + /** + * This test is in order to guarantee the feature added by https://github.com/apache/pulsar/pull/7105. + * 1. Start 3 consumers: + * - consumer1 will be closed and trigger a messages redeliver. + * - consumer2 will not ack any messages to make the new consumer joined late will be stuck due + * to the mechanism "recentlyJoinedConsumers". + * - consumer3 will always receive and ack messages. + * 2. Add consumer4 after consumer1 was close, and consumer4 will be stuck due to the mechanism + * "recentlyJoinedConsumers". + * 3. Verify: + * - (Main purpose) consumer3 can still receive messages util the cursor.readerPosition is larger than LAC. + * - no repeated Read-and-discard. + * - at last, all messages will be received. + */ + @Test(timeOut = 180 * 1000) // the test will be finished in 60s. + public void testRecentJoinedPosWillNotStuckOtherConsumer() throws Exception { + final int messagesSentPerTime = 100; + final Set totalReceivedMessages = new TreeSet<>(); + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String subName = "my-sub"; + admin.topics().createNonPartitionedTopic(topic); + AtomicInteger replyReadCounter = injectReplayReadCounter(topic, subName); + + // Send messages. + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.INT32).topic(topic).enableBatching(false).create(); + for (int i = 0; i < messagesSentPerTime; i++) { + MessageId messageId = producer.newMessage() + .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .value(100 + i) + .send(); + log.info("Published message :{}", messageId); + } + + // 1. Start 3 consumers and make ack holes. + // - one consumer will be closed and trigger a messages redeliver. + // - one consumer will not ack any messages to make the new consumer joined late will be stuck due to the + // mechanism "recentlyJoinedConsumers". + // - one consumer will always receive and ack messages. + Consumer consumer1 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(10) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + Consumer consumer2 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(10) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + Consumer consumer3 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(10) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + List msgList1 = new ArrayList<>(); + List msgList2 = new ArrayList<>(); + List msgList3 = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + Message msg1 = consumer1.receive(1, TimeUnit.SECONDS); + if (msg1 != null) { + totalReceivedMessages.add(msg1.getValue()); + msgList1.add(msg1); + } + Message msg2 = consumer2.receive(1, TimeUnit.SECONDS); + if (msg2 != null) { + totalReceivedMessages.add(msg2.getValue()); + msgList2.add(msg2); + } + Message msg3 = consumer3.receive(1, TimeUnit.SECONDS); + if (msg2 != null) { + totalReceivedMessages.add(msg3.getValue()); + msgList3.add(msg3); + } + } + Consumer consumerWillBeClose = null; + Consumer consumerAlwaysAck = null; + Consumer consumerStuck = null; + if (!msgList1.isEmpty()) { + msgList1.forEach(msg -> consumer1.acknowledgeAsync(msg)); + consumerAlwaysAck = consumer1; + consumerWillBeClose = consumer2; + consumerStuck = consumer3; + } else if (!msgList2.isEmpty()){ + msgList2.forEach(msg -> consumer2.acknowledgeAsync(msg)); + consumerAlwaysAck = consumer2; + consumerWillBeClose = consumer3; + consumerStuck = consumer1; + } else { + msgList3.forEach(msg -> consumer3.acknowledgeAsync(msg)); + consumerAlwaysAck = consumer3; + consumerWillBeClose = consumer1; + consumerStuck = consumer2; + } + + // 2. Add consumer4 after "consumerWillBeClose" was close, and consumer4 will be stuck due to the mechanism + // "recentlyJoinedConsumers". + Consumer consumer4 = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName(subName) + .receiverQueueSize(1000) + .subscriptionType(SubscriptionType.Key_Shared) + .subscribe(); + consumerWillBeClose.close(); + + Thread.sleep(2000); + + for (int i = messagesSentPerTime; i < messagesSentPerTime * 2; i++) { + MessageId messageId = producer.newMessage() + .key(String.valueOf(random.nextInt(NUMBER_OF_KEYS))) + .value(100 + i) + .send(); + log.info("Published message :{}", messageId); + } + + // Send messages again. + // Verify: "consumerAlwaysAck" can receive messages util the cursor.readerPosition is larger than LAC. + while (true) { + Message msg = consumerAlwaysAck.receive(2, TimeUnit.SECONDS); + if (msg == null) { + break; + } + totalReceivedMessages.add(msg.getValue()); + consumerAlwaysAck.acknowledge(msg); + } + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + ManagedCursorImpl cursor = (ManagedCursorImpl) managedLedger.openCursor(subName); + log.info("cursor_readPosition {}, LAC {}", cursor.getReadPosition(), managedLedger.getLastConfirmedEntry()); + assertTrue(((PositionImpl) cursor.getReadPosition()) + .compareTo((PositionImpl) managedLedger.getLastConfirmedEntry()) > 0); + + // Make all consumers to start to read and acknowledge messages. + // Verify: no repeated Read-and-discard. + Thread.sleep(5 * 1000); + int maxReplayCount = messagesSentPerTime * 2; + log.info("Reply read count: {}", replyReadCounter.get()); + assertTrue(replyReadCounter.get() < maxReplayCount); + // Verify: at last, all messages will be received. + ReceivedMessages receivedMessages = ackAllMessages(consumerAlwaysAck, consumerStuck, consumer4); + totalReceivedMessages.addAll(receivedMessages.messagesReceived.stream().map(p -> p.getRight()).collect( + Collectors.toList())); + assertEquals(totalReceivedMessages.size(), messagesSentPerTime * 2); + + // cleanup. + consumer1.close(); + consumer2.close(); + consumer3.close(); + consumer4.close(); + producer.close(); + admin.topics().delete(topic, false); + } } 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 f58c1fa26afc7..ef070250ca1aa 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 @@ -21,9 +21,14 @@ import com.google.common.collect.Sets; import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.List; import java.util.Random; import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.function.BiFunction; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -69,4 +74,65 @@ protected String newTopicName() { return "my-property/my-ns/topic-" + Long.toHexString(random.nextLong()); } + protected ReceivedMessages receiveAndAckMessages( + BiFunction ackPredicate, + Consumer...consumers) throws Exception { + ReceivedMessages receivedMessages = new ReceivedMessages(); + while (true) { + int receivedMsgCount = 0; + for (int i = 0; i < consumers.length; i++) { + Consumer consumer = consumers[i]; + while (true) { + Message msg = consumer.receive(2, TimeUnit.SECONDS); + if (msg != null) { + receivedMsgCount++; + T v = msg.getValue(); + MessageId messageId = msg.getMessageId(); + receivedMessages.messagesReceived.add(Pair.of(msg.getMessageId(), v)); + if (ackPredicate.apply(messageId, v)) { + consumer.acknowledge(msg); + receivedMessages.messagesAcked.add(Pair.of(msg.getMessageId(), v)); + } + } else { + break; + } + } + } + // Because of the possibility of consumers getting stuck with each other, only jump out of the loop if all + // consumers could not receive messages. + if (receivedMsgCount == 0) { + break; + } + } + return receivedMessages; + } + + protected ReceivedMessages ackAllMessages(Consumer...consumers) throws Exception { + return receiveAndAckMessages((msgId, msgV) -> true, consumers); + } + + protected static class ReceivedMessages { + + List> messagesReceived = new ArrayList<>(); + + List> messagesAcked = new ArrayList<>(); + + public boolean hasReceivedMessage(T v) { + for (Pair pair : messagesReceived) { + if (pair.getRight().equals(v)) { + return true; + } + } + return false; + } + + public boolean hasAckedMessage(T v) { + for (Pair pair : messagesAcked) { + if (pair.getRight().equals(v)) { + return true; + } + } + return false; + } + } } From 2e1c0c78fe25510011be1d96da585d55482e5093 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 4 Apr 2024 00:27:25 -0700 Subject: [PATCH 410/699] [fix][broker] Support OIDC providers with JWK without alg field set in keys (#22421) (cherry picked from commit a1970ae0996b2ccaad1251a8ef692faee24b83b8) (cherry picked from commit 4a79a2637bb50f0c8f7f32aff6529cdc799481bc) --- .../oidc/AuthenticationProviderOpenID.java | 3 ++- ...enticationProviderOpenIDIntegrationTest.java | 17 ++++++++++++++++- 2 files changed, 18 insertions(+), 2 deletions(-) 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 1462b8e293f79..426e575004c98 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 @@ -303,7 +303,8 @@ private CompletableFuture authenticateToken(String token) { return verifyIssuerAndGetJwk(jwt) .thenCompose(jwk -> { try { - if (!jwt.getAlgorithm().equals(jwk.getAlgorithm())) { + // verify the algorithm, if it is set ("alg" is optional in the JWK spec) + if (jwk.getAlgorithm() != null && !jwt.getAlgorithm().equals(jwk.getAlgorithm())) { incrementFailureMetric(AuthenticationExceptionCode.ALGORITHM_MISMATCH); return CompletableFuture.failedFuture( new AuthenticationException("JWK's alg [" + jwk.getAlgorithm() diff --git a/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDIntegrationTest.java b/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDIntegrationTest.java index d22b70be7f6f1..ab6417667ecb0 100644 --- a/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDIntegrationTest.java +++ b/pulsar-broker-auth-oidc/src/test/java/org/apache/pulsar/broker/authentication/oidc/AuthenticationProviderOpenIDIntegrationTest.java @@ -75,6 +75,7 @@ public class AuthenticationProviderOpenIDIntegrationTest { // These are the kid values for JWKs in the /keys endpoint String validJwk = "valid"; String invalidJwk = "invalid"; + String validJwkWithoutAlg = "valid_without_alg"; // The valid issuer String issuer; @@ -188,10 +189,16 @@ void beforeClass() throws IOException { "kty":"RSA", "n":"invalid-key", "e":"AQAB" + }, + { + "kid":"%s", + "kty":"RSA", + "n":"%s", + "e":"%s" } ] } - """.formatted(validJwk, n, e, invalidJwk)))); + """.formatted(validJwk, n, e, invalidJwk, validJwkWithoutAlg, n, e)))); server.stubFor( get(urlEqualTo("/missing-kid/.well-known/openid-configuration")) @@ -274,6 +281,14 @@ public void testTokenWithValidJWK() throws Exception { assertEquals(role, provider.authenticateAsync(new AuthenticationDataCommand(token)).get()); } + @Test + public void testTokenWithValidJWKWithoutAlg() throws Exception { + String role = "superuser"; + // test with a key in JWK that does not have an "alg" field. "alg" is optional in the JWK spec + String token = generateToken(validJwkWithoutAlg, issuer, role, "allowed-audience", 0L, 0L, 10000L); + assertEquals(role, provider.authenticateAsync(new AuthenticationDataCommand(token)).get()); + } + @Test public void testTokenWithTrailingSlashAndValidJWK() throws Exception { String role = "superuser"; From 8e2bd551915dbdb90f1fbf40a7a02f2f90d3b187 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 4 Apr 2024 01:15:44 -0700 Subject: [PATCH 411/699] [fix][misc] Rename all shaded Netty native libraries (#22415) (cherry picked from commit f4235580e6477f0c2f846419866b70c1b057e372) (cherry picked from commit 5b141440cce6d9a03481844873e8b09c0558b497) --- src/rename-netty-native-libs.cmd | 22 +++++++++++++++++++--- src/rename-netty-native-libs.sh | 6 ++++++ 2 files changed, 25 insertions(+), 3 deletions(-) diff --git a/src/rename-netty-native-libs.cmd b/src/rename-netty-native-libs.cmd index 9003f6d0ef499..bfaa16de0812c 100644 --- a/src/rename-netty-native-libs.cmd +++ b/src/rename-netty-native-libs.cmd @@ -42,11 +42,27 @@ call %UNZIP_CMD% cd /d %TMP_DIR%/%FILE_PREFIX% :: Loop through the number of groups -SET Obj_Length=2 +SET Obj_Length=10 SET Obj[0].FROM=libnetty_transport_native_epoll_x86_64.so SET Obj[0].TO=liborg_apache_pulsar_shade_netty_transport_native_epoll_x86_64.so -SET Obj[1].FROM=libnetty_tcnative_linux_x86_64.so -SET Obj[1].TO=liborg_apache_pulsar_shade_netty_tcnative_linux_x86_64.so +SET Obj[1].FROM=libnetty_transport_native_epoll_aarch_64.so +SET Obj[1].TO=liborg_apache_pulsar_shade_netty_transport_native_epoll_aarch_64.so +SET Obj[2].FROM=libnetty_tcnative_linux_x86_64.so +SET Obj[2].TO=liborg_apache_pulsar_shade_netty_tcnative_linux_x86_64.so +SET Obj[3].FROM=libnetty_tcnative_linux_aarch_64.so +SET Obj[3].TO=liborg_apache_pulsar_shade_netty_tcnative_linux_aarch_64.so +SET Obj[4].FROM=libnetty_tcnative_osx_x86_64.jnilib +SET Obj[4].TO=liborg_apache_pulsar_shade_netty_tcnative_osx_x86_64.jnilib +SET Obj[5].FROM=libnetty_tcnative_osx_aarch_64.jnilib +SET Obj[5].TO=liborg_apache_pulsar_shade_netty_tcnative_osx_aarch_64.jnilib +SET Obj[6].FROM=libnetty_transport_native_io_uring_x86_64.so +SET Obj[6].TO=liborg_apache_pulsar_shade_netty_transport_native_io_uring_x86_64.so +SET Obj[7].FROM=libnetty_transport_native_io_uring_aarch_64.so +SET Obj[7].TO=liborg_apache_pulsar_shade_netty_transport_native_io_uring_aarch_64.so +SET Obj[8].FROM=libnetty_resolver_dns_native_macos_aarch_64.jnilib +SET Obj[8].TO=liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_aarch_64.jnilib +SET Obj[9].FROM=libnetty_resolver_dns_native_macos_x86_64.jnilib +SET Obj[9].TO=liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_x86_64.jnilib SET Obj_Index=0 :LoopStart diff --git a/src/rename-netty-native-libs.sh b/src/rename-netty-native-libs.sh index 44b971a02c912..ea2a4c0e2421e 100755 --- a/src/rename-netty-native-libs.sh +++ b/src/rename-netty-native-libs.sh @@ -27,7 +27,13 @@ FILE_PREFIX='META-INF/native' FILES_TO_RENAME=( 'libnetty_transport_native_epoll_x86_64.so liborg_apache_pulsar_shade_netty_transport_native_epoll_x86_64.so' + 'libnetty_transport_native_epoll_aarch_64.so liborg_apache_pulsar_shade_netty_transport_native_epoll_aarch_64.so' 'libnetty_tcnative_linux_x86_64.so liborg_apache_pulsar_shade_netty_tcnative_linux_x86_64.so' + 'libnetty_tcnative_linux_aarch_64.so liborg_apache_pulsar_shade_netty_tcnative_linux_aarch_64.so' + 'libnetty_tcnative_osx_x86_64.jnilib liborg_apache_pulsar_shade_netty_tcnative_osx_x86_64.jnilib' + 'libnetty_tcnative_osx_aarch_64.jnilib liborg_apache_pulsar_shade_netty_tcnative_osx_aarch_64.jnilib' + 'libnetty_transport_native_io_uring_x86_64.so liborg_apache_pulsar_shade_netty_transport_native_io_uring_x86_64.so' + 'libnetty_transport_native_io_uring_aarch_64.so liborg_apache_pulsar_shade_netty_transport_native_io_uring_aarch_64.so' 'libnetty_resolver_dns_native_macos_aarch_64.jnilib liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_aarch_64.jnilib' 'libnetty_resolver_dns_native_macos_x86_64.jnilib liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_x86_64.jnilib' ) From 8f3e1fd6759a616c6965b785fdc36f26c75a9186 Mon Sep 17 00:00:00 2001 From: Jiwei Guo Date: Mon, 8 Apr 2024 18:22:05 +0800 Subject: [PATCH 412/699] [fix][broker] Fix consumer stops receiving messages when with large backlogs processing (#22454) (cherry picked from commit 40329ee3615ebeff6eaa0dca2d454d6389fa6f43) --- .../mledger/impl/ManagedCursorImpl.java | 5 ++ .../mledger/impl/ManagedLedgerImpl.java | 9 +-- .../persistent/PersistentSubscription.java | 4 +- .../persistent/PersistentTopicTest.java | 56 ++++++++++++++++++- 4 files changed, 65 insertions(+), 9 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 f2dd045fca7cf..972abcfea983d 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 @@ -985,6 +985,11 @@ private void checkForNewEntries(OpReadEntry op, ReadEntriesCallback callback, Ob log.debug("[{}] [{}] Re-trying the read at position {}", ledger.getName(), name, op.readPosition); } + if (isClosed()) { + callback.readEntriesFailed(new CursorAlreadyClosedException("Cursor was already closed"), ctx); + return; + } + if (!hasMoreEntries()) { if (log.isDebugEnabled()) { log.debug("[{}] [{}] Still no entries available. Register for notification", ledger.getName(), 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 faa97b2043596..c589081108186 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 @@ -1034,6 +1034,7 @@ public synchronized void asyncDeleteCursor(final String consumerName, final Dele + consumerName), ctx); return; } else if (!cursor.isDurable()) { + cursor.setState(ManagedCursorImpl.State.Closed); cursors.removeCursor(consumerName); deactivateCursorByName(consumerName); callback.deleteCursorComplete(ctx); @@ -3851,13 +3852,7 @@ public void removeWaitingCursor(ManagedCursor cursor) { } public void addWaitingCursor(ManagedCursorImpl cursor) { - if (cursor instanceof NonDurableCursorImpl) { - if (cursor.isActive()) { - this.waitingCursors.add(cursor); - } - } else { - this.waitingCursors.add(cursor); - } + this.waitingCursors.add(cursor); } public boolean isCursorActive(ManagedCursor cursor) { 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 35f601aa0766d..c5eeab63ccd95 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 @@ -306,7 +306,6 @@ public synchronized void removeConsumer(Consumer consumer, boolean isResetCursor if (dispatcher != null && dispatcher.getConsumers().isEmpty()) { deactivateCursor(); - topic.getManagedLedger().removeWaitingCursor(cursor); if (!cursor.isDurable()) { // If cursor is not durable, we need to clean up the subscription as well @@ -335,11 +334,14 @@ public synchronized void removeConsumer(Consumer consumer, boolean isResetCursor if (!isResetCursor) { try { topic.getManagedLedger().deleteCursor(cursor.getName()); + topic.getManagedLedger().removeWaitingCursor(cursor); } catch (InterruptedException | ManagedLedgerException e) { log.warn("[{}] [{}] Failed to remove non durable cursor", topic.getName(), subName, e); } } }); + } else { + topic.getManagedLedger().removeWaitingCursor(cursor); } } 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 618fe8006a6b3..4c0d8eb6a49ac 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 @@ -79,6 +79,7 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; +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.common.naming.NamespaceBundle; @@ -112,6 +113,11 @@ protected void cleanup() throws Exception { super.internalCleanup(); } + @Override protected void doInitConf() throws Exception { + super.doInitConf(); + this.conf.setManagedLedgerCursorBackloggedThreshold(10); + } + /** * Test validates that broker cleans up topic which failed to unload while bundle unloading. * @@ -680,7 +686,7 @@ public void testAddWaitingCursorsForNonDurable() throws Exception { ManagedLedgerImpl ledger = (ManagedLedgerImpl)persistentTopic.getManagedLedger(); final ManagedCursor spyCursor= spy(ledger.newNonDurableCursor(PositionImpl.LATEST, "sub-2")); doAnswer((invocation) -> { - Thread.sleep(10_000); + Thread.sleep(5_000); invocation.callRealMethod(); return null; }).when(spyCursor).asyncReadEntriesOrWait(any(int.class), any(long.class), @@ -707,4 +713,52 @@ public void testAddWaitingCursorsForNonDurable() throws Exception { assertEquals(ledger.getWaitingCursorsCount(), 0); }); } + + @Test + public void testAddWaitingCursorsForNonDurable2() throws Exception { + final String ns = "prop/ns-test"; + admin.namespaces().createNamespace(ns, 2); + final String topicName = "persistent://prop/ns-test/testAddWaitingCursors2"; + admin.topics().createNonPartitionedTopic(topicName); + pulsarClient.newConsumer(Schema.STRING).topic(topicName) + .subscriptionMode(SubscriptionMode.Durable) + .subscriptionType(SubscriptionType.Shared) + .subscriptionName("sub-1").subscribe().close(); + @Cleanup + final Producer producer = pulsarClient.newProducer(Schema.STRING).enableBatching(false).topic(topicName).create(); + for (int i = 0; i < 100; i ++) { + producer.sendAsync("test-" + i); + } + @Cleanup + final Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName) + .subscriptionMode(SubscriptionMode.NonDurable) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscriptionType(SubscriptionType.Exclusive) + .subscriptionName("sub-2").subscribe(); + int count = 0; + while(true) { + final Message msg = consumer.receive(3, TimeUnit.SECONDS); + if (msg != null) { + consumer.acknowledge(msg); + count++; + } else { + break; + } + } + Assert.assertEquals(count, 100); + Thread.sleep(3_000); + for (int i = 0; i < 100; i ++) { + producer.sendAsync("test-" + i); + } + while(true) { + final Message msg = consumer.receive(5, TimeUnit.SECONDS); + if (msg != null) { + consumer.acknowledge(msg); + count++; + } else { + break; + } + } + Assert.assertEquals(count, 200); + } } From a5b9be343c89f3fd2c1fd20a15b0a3ad1dde754e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Thu, 4 Apr 2024 23:08:45 +0800 Subject: [PATCH 413/699] [admin][broker] Fix force delete subscription not working (#22423) (cherry picked from commit 110add00c1cdc59542e7ef906cd5f4409e63dc04) --- .../admin/impl/PersistentTopicsBase.java | 5 ++-- .../broker/admin/PersistentTopicsTest.java | 30 +++++++++++++++++++ 2 files changed, 32 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 52d273d5798e3..a4a12e6a9138a 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 @@ -1577,7 +1577,7 @@ protected CompletableFuture internalDeleteSubscriptionAsync(String subName for (int i = 0; i < partitionMetadata.partitions; i++) { TopicName topicNamePartition = topicName.getPartition(i); futures.add(adminClient.topics() - .deleteSubscriptionAsync(topicNamePartition.toString(), subName, false)); + .deleteSubscriptionAsync(topicNamePartition.toString(), subName, force)); } return FutureUtil.waitForAll(futures).handle((result, exception) -> { @@ -1596,8 +1596,7 @@ protected CompletableFuture internalDeleteSubscriptionAsync(String subName return null; }); } - return internalDeleteSubscriptionForNonPartitionedTopicAsync(subName, authoritative, - force); + return internalDeleteSubscriptionForNonPartitionedTopicAsync(subName, authoritative, force); }); } }); 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 af7eeda7e4965..98a33274c94e0 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 @@ -79,11 +79,13 @@ import org.apache.pulsar.client.admin.Topics; import org.apache.pulsar.client.admin.internal.TopicsImpl; 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.Reader; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.interceptor.ProducerInterceptor; import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; @@ -1781,6 +1783,34 @@ public void testNamespaceResources() throws Exception { assertTrue(namespaces.contains(ns1V1)); } + @Test + public void testForceDeleteSubscription() throws Exception { + try { + pulsar.getConfiguration().setAllowAutoSubscriptionCreation(false); + String topicName = "persistent://" + testTenant + "/" + testNamespaceLocal + "/testForceDeleteSubscription"; + String subName = "sub1"; + admin.topics().createNonPartitionedTopic(topicName); + admin.topics().createSubscription(topicName, subName, MessageId.latest); + + @Cleanup + Consumer c0 = pulsarClient.newConsumer(Schema.STRING) + .topic(topicName) + .subscriptionName(subName) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + @Cleanup + Consumer c1 = pulsarClient.newConsumer(Schema.STRING) + .topic(topicName) + .subscriptionName(subName) + .subscriptionType(SubscriptionType.Shared) + .subscribe(); + + admin.topics().deleteSubscription(topicName, subName, true); + } finally { + pulsar.getConfiguration().setAllowAutoSubscriptionCreation(true); + } + } + @Test public void testUpdatePropertiesOnNonDurableSub() throws Exception { String topic = "persistent://" + testTenant + "/" + testNamespaceLocal + "/testUpdatePropertiesOnNonDurableSub"; From fd4c9c6c8ec1e7a027f8145a5f443835a6f0d125 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Fri, 29 Mar 2024 08:33:27 +0800 Subject: [PATCH 414/699] [improve][io]: Add validation for JDBC sink not supporting primitive schema (#22376) (cherry picked from commit a503efc826e60b8e26f9792aeb45223374b8f4ca) --- pulsar-io/jdbc/core/pom.xml | 7 +++++ .../io/jdbc/BaseJdbcAutoSchemaSink.java | 5 +++ .../io/jdbc/BaseJdbcAutoSchemaSinkTest.java | 25 +++++++++++++++ .../pulsar/io/jdbc/SqliteJdbcSinkTest.java | 31 +++++++++++++------ 4 files changed, 59 insertions(+), 9 deletions(-) diff --git a/pulsar-io/jdbc/core/pom.xml b/pulsar-io/jdbc/core/pom.xml index 5c0adf6be27de..b0e5a3d0b3e8d 100644 --- a/pulsar-io/jdbc/core/pom.xml +++ b/pulsar-io/jdbc/core/pom.xml @@ -71,6 +71,13 @@ provided + + ${project.groupId} + pulsar-client-original + ${project.version} + test + + \ No newline at end of file diff --git a/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSink.java b/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSink.java index 36c3674091932..3655688c0f3ad 100644 --- a/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSink.java +++ b/pulsar-io/jdbc/core/src/main/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSink.java @@ -34,6 +34,7 @@ import org.apache.pulsar.client.api.schema.GenericRecord; import org.apache.pulsar.client.api.schema.KeyValueSchema; import org.apache.pulsar.common.schema.KeyValue; +import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.io.jdbc.JdbcUtils.ColumnId; @@ -137,6 +138,10 @@ public Mutation createMutation(Record message) { } recordValueGetter = (k) -> data.get(k); } else { + SchemaType schemaType = message.getSchema().getSchemaInfo().getType(); + if (schemaType.isPrimitive()) { + throw new UnsupportedOperationException("Primitive schema is not supported: " + schemaType); + } recordValueGetter = (key) -> ((GenericRecord) record).getField(key); } String action = message.getProperties().get(ACTION_PROPERTY); diff --git a/pulsar-io/jdbc/core/src/test/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSinkTest.java b/pulsar-io/jdbc/core/src/test/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSinkTest.java index b15eb832242c7..c088dd3c42c32 100644 --- a/pulsar-io/jdbc/core/src/test/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSinkTest.java +++ b/pulsar-io/jdbc/core/src/test/java/org/apache/pulsar/io/jdbc/BaseJdbcAutoSchemaSinkTest.java @@ -22,6 +22,10 @@ import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; import org.apache.avro.util.Utf8; +import org.apache.pulsar.client.api.schema.GenericObject; +import org.apache.pulsar.client.api.schema.GenericRecord; +import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; +import org.apache.pulsar.functions.api.Record; import org.testng.Assert; import org.testng.annotations.Test; @@ -143,5 +147,26 @@ private Schema createFieldAndGetSchema(Function record = new Record() { + @Override + public org.apache.pulsar.client.api.Schema getSchema() { + return autoConsumeSchema; + } + + @Override + public GenericRecord getValue() { + return null; + } + }; + baseJdbcAutoSchemaSink.createMutation((Record) record); + } + } \ No newline at end of file diff --git a/pulsar-io/jdbc/sqlite/src/test/java/org/apache/pulsar/io/jdbc/SqliteJdbcSinkTest.java b/pulsar-io/jdbc/sqlite/src/test/java/org/apache/pulsar/io/jdbc/SqliteJdbcSinkTest.java index d9ed4cbd442bf..ca01615bef193 100644 --- a/pulsar-io/jdbc/sqlite/src/test/java/org/apache/pulsar/io/jdbc/SqliteJdbcSinkTest.java +++ b/pulsar-io/jdbc/sqlite/src/test/java/org/apache/pulsar/io/jdbc/SqliteJdbcSinkTest.java @@ -48,6 +48,7 @@ import org.apache.pulsar.client.api.schema.RecordSchemaBuilder; import org.apache.pulsar.client.api.schema.SchemaDefinition; import org.apache.pulsar.client.impl.MessageImpl; +import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; import org.apache.pulsar.client.impl.schema.AvroSchema; import org.apache.pulsar.client.impl.schema.generic.GenericAvroSchema; import org.apache.pulsar.common.schema.KeyValue; @@ -282,9 +283,12 @@ public void TestUnknownAction() throws Exception { } @Test + @SuppressWarnings("unchecked") public void TestUpdateAction() throws Exception { AvroSchema schema = AvroSchema.of(SchemaDefinition.builder().withPojo(Foo.class).build()); + AutoConsumeSchema autoConsumeSchema = new AutoConsumeSchema(); + autoConsumeSchema.setSchema(schema); Foo updateObj = new Foo(); updateObj.setField1("ValueOfField3"); @@ -292,10 +296,11 @@ public void TestUpdateAction() throws Exception { updateObj.setField3(4); byte[] updateBytes = schema.encode(updateObj); - Message updateMessage = mock(MessageImpl.class); + Message updateMessage = mock(MessageImpl.class); CompletableFuture future = new CompletableFuture<>(); - Record updateRecord = PulsarRecord.builder() + Record updateRecord = PulsarRecord.builder() .message(updateMessage) + .schema(autoConsumeSchema) .topicName("fake_topic_name") .ackFunction(() -> future.complete(null)) .build(); @@ -312,7 +317,7 @@ public void TestUpdateAction() throws Exception { updateMessage.getValue().toString(), updateRecord.getValue().toString()); - jdbcSink.write(updateRecord); + jdbcSink.write((Record) updateRecord); future.get(1, TimeUnit.SECONDS); // value has been written to db, read it out and verify. @@ -325,18 +330,22 @@ public void TestUpdateAction() throws Exception { } @Test + @SuppressWarnings("unchecked") public void TestDeleteAction() throws Exception { AvroSchema schema = AvroSchema.of(SchemaDefinition.builder().withPojo(Foo.class).build()); + AutoConsumeSchema autoConsumeSchema = new AutoConsumeSchema(); + autoConsumeSchema.setSchema(schema); Foo deleteObj = new Foo(); deleteObj.setField3(5); byte[] deleteBytes = schema.encode(deleteObj); - Message deleteMessage = mock(MessageImpl.class); + Message deleteMessage = mock(MessageImpl.class); CompletableFuture future = new CompletableFuture<>(); - Record deleteRecord = PulsarRecord.builder() + Record deleteRecord = PulsarRecord.builder() .message(deleteMessage) + .schema(autoConsumeSchema) .topicName("fake_topic_name") .ackFunction(() -> future.complete(null)) .build(); @@ -352,7 +361,7 @@ public void TestDeleteAction() throws Exception { deleteMessage.getValue().toString(), deleteRecord.getValue().toString()); - jdbcSink.write(deleteRecord); + jdbcSink.write((Record) deleteRecord); future.get(1, TimeUnit.SECONDS); // value has been written to db, read it out and verify. @@ -848,17 +857,21 @@ public void testNullValueAction(NullValueActionTestConfig config) throws Excepti } } + @SuppressWarnings("unchecked") private Record createMockFooRecord(Foo record, Map actionProperties, CompletableFuture future) { - Message insertMessage = mock(MessageImpl.class); + Message insertMessage = mock(MessageImpl.class); GenericSchema genericAvroSchema; AvroSchema schema = AvroSchema.of(SchemaDefinition.builder().withPojo(Foo.class).withAlwaysAllowNull(true).build()); + AutoConsumeSchema autoConsumeSchema = new AutoConsumeSchema(); + autoConsumeSchema.setSchema(schema); byte[] insertBytes = schema.encode(record); - Record insertRecord = PulsarRecord.builder() + Record insertRecord = PulsarRecord.builder() .message(insertMessage) .topicName("fake_topic_name") + .schema(autoConsumeSchema) .ackFunction(() -> future.complete(true)) .failFunction(() -> future.complete(false)) .build(); @@ -866,7 +879,7 @@ private Record createMockFooRecord(Foo record, Map) insertRecord; } } From cedc250faa7f1ee0f2623b99f580ead7e6e09f3c Mon Sep 17 00:00:00 2001 From: houxiaoyu Date: Sat, 30 Mar 2024 21:38:55 +0800 Subject: [PATCH 415/699] [fix][ml] No rollover inactive ledgers when metadata service invalid (#22284) ### Motivation We should not rollover inactive ledgers when metadata service is invailable. ### Modifications Checking metadata service is vailable when schedule `checkInactiveLedgerAndRollOver` (cherry picked from commit b9bf0a83b5ccf89e036cf77a1c4cb764acf59bba) --- .../mledger/impl/ManagedLedgerImpl.java | 7 +++--- .../mledger/impl/ManagedLedgerTest.java | 24 +++++++++++++++++++ 2 files changed, 28 insertions(+), 3 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 c589081108186..dce4860e3e6f0 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 @@ -4510,9 +4510,10 @@ private void cancelScheduledTasks() { @Override public boolean checkInactiveLedgerAndRollOver() { - long currentTimeMs = System.currentTimeMillis(); - if (currentLedgerEntries > 0 && inactiveLedgerRollOverTimeMs > 0 && currentTimeMs > (lastAddEntryTimeMs - + inactiveLedgerRollOverTimeMs)) { + if (factory.isMetadataServiceAvailable() + && currentLedgerEntries > 0 + && inactiveLedgerRollOverTimeMs > 0 + && System.currentTimeMillis() > (lastAddEntryTimeMs + inactiveLedgerRollOverTimeMs)) { log.info("[{}] Closing inactive ledger, last-add entry {}", name, lastAddEntryTimeMs); if (STATE_UPDATER.compareAndSet(this, State.LedgerOpened, State.ClosingLedger)) { LedgerHandle currentLedger = this.currentLedger; 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 a08b51cf29fc5..f918ffdc755d6 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 @@ -3922,6 +3922,30 @@ public void testDontRollOverEmptyInactiveLedgers() throws Exception { ledger.close(); } + @Test + public void testDontRollOverInactiveLedgersWhenMetadataServiceInvalid() throws Exception { + int inactiveLedgerRollOverTimeMs = 5; + @Cleanup("shutdown") + ManagedLedgerFactoryImpl factory = spy(new ManagedLedgerFactoryImpl(metadataStore, bkc)); + // mock metadata service invalid + when(factory.isMetadataServiceAvailable()).thenReturn(false); + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setInactiveLedgerRollOverTime(inactiveLedgerRollOverTimeMs, TimeUnit.MILLISECONDS); + ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("rollover_inactive", config); + + long ledgerId = ledger.currentLedger.getId(); + + Thread.sleep(inactiveLedgerRollOverTimeMs * 5); + ledger.checkInactiveLedgerAndRollOver(); + + Thread.sleep(inactiveLedgerRollOverTimeMs * 5); + ledger.checkInactiveLedgerAndRollOver(); + + assertEquals(ledger.currentLedger.getId(), ledgerId); + + ledger.close(); + } + @Test public void testOffloadTaskCancelled() throws Exception { ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); From 457257c9846b5178a09bf65d866a3fae1230bdbb Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Sun, 7 Apr 2024 09:28:04 +0800 Subject: [PATCH 416/699] [fix][client] Fix client side memory leak when call MessageImpl.create and fix imprecise client-side metrics: pendingMessagesUpDownCounter, pendingBytesUpDownCounter, latencyHistogram (#22393) (cherry picked from commit 2469b97b7e4de10fec64cc7ff1f4f46a410ad125) (cherry picked from commit 14b627967906e08072c968c1c58f98624cf92cbf) --- .../api/SimpleProducerConsumerTest.java | 144 ++++++++++++++++++ .../pulsar/client/impl/ProducerImpl.java | 127 +++++++-------- 2 files changed, 211 insertions(+), 60 deletions(-) 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 4c106d39e7ad7..7552b84a1c553 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 @@ -51,6 +51,7 @@ import java.time.Instant; import java.time.ZoneId; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -69,6 +70,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.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -92,6 +94,7 @@ import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.schema.GenericRecord; +import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.ClientBuilderImpl; import org.apache.pulsar.client.impl.ConsumerBase; import org.apache.pulsar.client.impl.ConsumerImpl; @@ -99,11 +102,13 @@ import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.client.impl.MultiTopicsConsumerImpl; import org.apache.pulsar.client.impl.PartitionedProducerImpl; +import org.apache.pulsar.client.impl.ProducerBase; import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.client.impl.TopicMessageImpl; import org.apache.pulsar.client.impl.TypedMessageBuilderImpl; import org.apache.pulsar.client.impl.crypto.MessageCryptoBc; import org.apache.pulsar.client.impl.schema.writer.AvroWriter; +import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.api.EncryptionContext; import org.apache.pulsar.common.api.EncryptionContext.EncryptionKey; import org.apache.pulsar.common.api.proto.MessageMetadata; @@ -4692,4 +4697,143 @@ public void flush(ChannelHandlerContext ctx) throws Exception { consumer.close(); admin.topics().delete(topic, false); } + + @DataProvider(name = "enableBatchSend") + public Object[][] enableBatchSend() { + return new Object[][]{ + {true}, + {false} + }; + } + + @Test(dataProvider = "enableBatchSend") + public void testPublishWithCreateMessageManually(boolean enableBatchSend) throws Exception { + final int messageCount = 10; + final List messageArrayBeforeSend = Collections.synchronizedList(new ArrayList<>()); + final List messageArrayOnSendAcknowledgement = Collections.synchronizedList(new ArrayList<>()); + // Create an interceptor to verify the ref count of Message.payload is as expected. + AtomicBoolean payloadWasReleasedWhenIntercept = new AtomicBoolean(false); + ProducerInterceptor interceptor = new ProducerInterceptor(){ + + @Override + public void close() { + + } + @Override + public Message beforeSend(Producer producer, Message message) { + MessageImpl msgImpl = (MessageImpl) message; + log.info("payload.refCnf before send: {}", msgImpl.getDataBuffer().refCnt()); + if (msgImpl.getDataBuffer().refCnt() < 1) { + payloadWasReleasedWhenIntercept.set(true); + } + messageArrayBeforeSend.add(msgImpl); + return message; + } + + @Override + public void onSendAcknowledgement(Producer producer, Message message, MessageId msgId, + Throwable exception) { + MessageImpl msgImpl = (MessageImpl) message; + log.info("payload.refCnf on send acknowledgement: {}", msgImpl.getDataBuffer().refCnt()); + if (msgImpl.getDataBuffer().refCnt() < 1) { + payloadWasReleasedWhenIntercept.set(true); + } + messageArrayOnSendAcknowledgement.add(msgImpl); + } + }; + + final String topic = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/tp"); + admin.topics().createNonPartitionedTopic(topic); + ProducerBase producerBase = (ProducerBase) pulsarClient.newProducer().topic(topic).intercept(interceptor) + .enableBatching(enableBatchSend).create(); + + // Publish message. + // Note: "ProducerBase.sendAsync" is not equals to "Producer.sendAsync". + final MessageImpl[] messageArraySent = new MessageImpl[messageCount]; + final ByteBuf[] payloads = new ByteBuf[messageCount]; + List> sendFutureList = new ArrayList<>(); + List releaseFutureList = new ArrayList<>(); + for (int i = 0; i < messageCount; i++) { + // Create message payload, refCnf = 1 now. + ByteBuf payload = PulsarByteBufAllocator.DEFAULT.heapBuffer(1); + payloads[i] = payload; + log.info("payload_{}.refCnf 1st: {}", i, payload.refCnt()); + payload.writeByte(i); + // refCnf = 2 now. + payload.retain(); + log.info("payload_{}.refCnf 2nd: {}", i, payload.refCnt()); + MessageMetadata messageMetadata = new MessageMetadata(); + messageMetadata.setUncompressedSize(1); + MessageImpl message1 = MessageImpl.create(topic, null, messageMetadata, payload, Optional.empty(), + null, Schema.BYTES, 0, true, 0); + messageArraySent[i] = message1; + // Release ByteBuf the first time, refCnf = 1 now. + CompletableFuture future = producerBase.sendAsync(message1); + sendFutureList.add(future); + final int indexForLog = i; + future.whenComplete((v, ex) -> { + message1.release(); + log.info("payload_{}.refCnf 3rd after_complete_refCnf: {}, ex: {}", indexForLog, payload.refCnt(), + ex == null ? "null" : ex.getMessage()); + }); + } + sendFutureList.get(messageCount - 1).join(); + + // Left 2 seconds to wait the code in the finally-block, which is using to avoid this test to be flaky. + Thread.sleep(1000 * 2); + + // Verify: payload's refCnf. + for (int i = 0; i < messageCount; i++) { + log.info("payload_{}.refCnf 4th: {}", i, payloads[i].refCnt()); + assertEquals(payloads[i].refCnt(), 1); + } + + // Verify: the messages has not been released when calling interceptor. + assertFalse(payloadWasReleasedWhenIntercept.get()); + + // Verify: the order of send complete event. + MessageIdImpl messageIdPreviousOne = null; + for (int i = 0; i < messageCount; i++) { + MessageIdImpl messageId = (MessageIdImpl) sendFutureList.get(i).get(); + if (messageIdPreviousOne != null) { + assertTrue(compareMessageIds(messageIdPreviousOne, messageId) > 0); + } + messageIdPreviousOne = messageId; + } + + // Verify: the order of interceptor events. + for (int i = 0; i < messageCount; i++) { + assertTrue(messageArraySent[i] == messageArrayBeforeSend.get(i)); + assertTrue(messageArraySent[i] == messageArrayOnSendAcknowledgement.get(i)); + } + + // cleanup. + for (int i = 0; i < messageCount; i++) { + payloads[i].release(); + } + producerBase.close(); + admin.topics().delete(topic, false); + } + + private int compareMessageIds(MessageIdImpl messageId1, MessageIdImpl messageId2) { + if (messageId2.getLedgerId() < messageId1.getLedgerId()) { + return -1; + } + if (messageId2.getLedgerId() > messageId1.getLedgerId()) { + return 1; + } + if (messageId2.getEntryId() < messageId1.getEntryId()) { + return -1; + } + if (messageId2.getEntryId() > messageId1.getEntryId()) { + return 1; + } + if (messageId2 instanceof BatchMessageIdImpl && messageId1 instanceof BatchMessageIdImpl) { + BatchMessageIdImpl batchMessageId1 = (BatchMessageIdImpl) messageId1; + BatchMessageIdImpl batchMessageId2 = (BatchMessageIdImpl) messageId2; + return batchMessageId2.getBatchIndex() - batchMessageId1.getBatchIndex(); + } else { + return 0; + } + } } 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 8d345f0e8b1f6..2bd74db0d460d 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 @@ -336,73 +336,80 @@ CompletableFuture internalSendAsync(Message message) { if (interceptors != null) { interceptorMessage.getProperties(); } - sendAsync(interceptorMessage, new SendCallback() { - SendCallback nextCallback = null; - MessageImpl nextMsg = null; - long createdAt = System.nanoTime(); - @Override - public CompletableFuture getFuture() { - return future; - } + int msgSize = interceptorMessage.getDataBuffer().readableBytes(); + sendAsync(interceptorMessage, new DefaultSendMessageCallback(future, interceptorMessage, msgSize)); + return future; + } - @Override - public SendCallback getNextSendCallback() { - return nextCallback; - } + private class DefaultSendMessageCallback implements SendCallback { - @Override - public MessageImpl getNextMessage() { - return nextMsg; - } + CompletableFuture sendFuture; + MessageImpl currentMsg; + int msgSize; + long createdAt = System.nanoTime(); + SendCallback nextCallback = null; + MessageImpl nextMsg = null; - @Override - public void sendComplete(Exception e) { - try { - if (e != null) { - stats.incrementSendFailed(); - onSendAcknowledgement(interceptorMessage, null, e); - future.completeExceptionally(e); - } else { - onSendAcknowledgement(interceptorMessage, interceptorMessage.getMessageId(), null); - future.complete(interceptorMessage.getMessageId()); - stats.incrementNumAcksReceived(System.nanoTime() - createdAt); - } - } finally { - interceptorMessage.getDataBuffer().release(); - } + DefaultSendMessageCallback(CompletableFuture sendFuture, MessageImpl currentMsg, int msgSize) { + this.sendFuture = sendFuture; + this.currentMsg = currentMsg; + this.msgSize = msgSize; + } - while (nextCallback != null) { - SendCallback sendCallback = nextCallback; - MessageImpl msg = nextMsg; - // Retain the buffer used by interceptors callback to get message. Buffer will release after - // complete interceptors. - try { - msg.getDataBuffer().retain(); - if (e != null) { - stats.incrementSendFailed(); - onSendAcknowledgement(msg, null, e); - sendCallback.getFuture().completeExceptionally(e); - } else { - onSendAcknowledgement(msg, msg.getMessageId(), null); - sendCallback.getFuture().complete(msg.getMessageId()); - stats.incrementNumAcksReceived(System.nanoTime() - createdAt); - } - nextMsg = nextCallback.getNextMessage(); - nextCallback = nextCallback.getNextSendCallback(); - } finally { - msg.getDataBuffer().release(); - } - } - } + @Override + public CompletableFuture getFuture() { + return sendFuture; + } - @Override - public void addCallback(MessageImpl msg, SendCallback scb) { - nextMsg = msg; - nextCallback = scb; + @Override + public SendCallback getNextSendCallback() { + return nextCallback; + } + + @Override + public MessageImpl getNextMessage() { + return nextMsg; + } + + @Override + public void sendComplete(Exception e) { + SendCallback loopingCallback = this; + MessageImpl loopingMsg = currentMsg; + while (loopingCallback != null) { + onSendComplete(e, loopingCallback, loopingMsg); + loopingMsg = loopingCallback.getNextMessage(); + loopingCallback = loopingCallback.getNextSendCallback(); + } + } + + private void onSendComplete(Exception e, SendCallback sendCallback, MessageImpl msg) { + long createdAt = (sendCallback instanceof ProducerImpl.DefaultSendMessageCallback) + ? ((DefaultSendMessageCallback) sendCallback).createdAt : this.createdAt; + long latencyNanos = System.nanoTime() - createdAt; + ByteBuf payload = msg.getDataBuffer(); + if (payload == null) { + log.error("[{}] [{}] Payload is null when calling onSendComplete, which is not expected.", + topic, producerName); + } else { + ReferenceCountUtil.safeRelease(payload); } - }); - return future; + if (e != null) { + stats.incrementSendFailed(); + onSendAcknowledgement(msg, null, e); + sendCallback.getFuture().completeExceptionally(e); + } else { + stats.incrementNumAcksReceived(latencyNanos); + onSendAcknowledgement(msg, msg.getMessageId(), null); + sendCallback.getFuture().complete(msg.getMessageId()); + } + } + + @Override + public void addCallback(MessageImpl msg, SendCallback scb) { + nextMsg = msg; + nextCallback = scb; + } } @Override From 6d1bd3a82a0bfa9b875d9793f914ff60f129e24a Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Thu, 7 Mar 2024 11:32:18 +0800 Subject: [PATCH 417/699] Remove unused fields `msgSize` (cherry picked from commit 1c468773557918f5d4edd2aa41c519df241fcb80) --- .../main/java/org/apache/pulsar/client/impl/ProducerImpl.java | 2 -- 1 file changed, 2 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 2bd74db0d460d..a41de6d10512d 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 @@ -346,7 +346,6 @@ private class DefaultSendMessageCallback implements SendCallback { CompletableFuture sendFuture; MessageImpl currentMsg; - int msgSize; long createdAt = System.nanoTime(); SendCallback nextCallback = null; MessageImpl nextMsg = null; @@ -354,7 +353,6 @@ private class DefaultSendMessageCallback implements SendCallback { DefaultSendMessageCallback(CompletableFuture sendFuture, MessageImpl currentMsg, int msgSize) { this.sendFuture = sendFuture; this.currentMsg = currentMsg; - this.msgSize = msgSize; } @Override From aae0f48cb7e0b3a0c38be398d56f8020ddafade3 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 12 Apr 2024 00:15:33 -0700 Subject: [PATCH 418/699] [improve][misc] Upgrade to Bookkeeper 4.16.5 (#22484) (cherry picked from commit 4a5400f0c66dab2c3fbb0050c8f537952fef1951) (cherry picked from commit 5ba3e5713e47214d5ce2bb2591f483ee395d0263) --- .../server/src/assemble/LICENSE.bin.txt | 56 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 6 +- pom.xml | 2 +- pulsar-sql/presto-distribution/LICENSE | 30 +++++----- 4 files changed, 47 insertions(+), 47 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index c71dc60541ac7..ea88f50937441 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -345,34 +345,34 @@ The Apache Software License, Version 2.0 - net.java.dev.jna-jna-jpms-5.12.1.jar - net.java.dev.jna-jna-platform-jpms-5.12.1.jar * BookKeeper - - org.apache.bookkeeper-bookkeeper-common-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-common-allocator-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-proto-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-server-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-tools-framework-4.16.4.jar - - org.apache.bookkeeper-circe-checksum-4.16.4.jar - - org.apache.bookkeeper-cpu-affinity-4.16.4.jar - - org.apache.bookkeeper-statelib-4.16.4.jar - - org.apache.bookkeeper-stream-storage-api-4.16.4.jar - - org.apache.bookkeeper-stream-storage-common-4.16.4.jar - - org.apache.bookkeeper-stream-storage-java-client-4.16.4.jar - - org.apache.bookkeeper-stream-storage-java-client-base-4.16.4.jar - - org.apache.bookkeeper-stream-storage-proto-4.16.4.jar - - org.apache.bookkeeper-stream-storage-server-4.16.4.jar - - org.apache.bookkeeper-stream-storage-service-api-4.16.4.jar - - org.apache.bookkeeper-stream-storage-service-impl-4.16.4.jar - - org.apache.bookkeeper.http-http-server-4.16.4.jar - - org.apache.bookkeeper.http-vertx-http-server-4.16.4.jar - - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.16.4.jar - - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.16.4.jar - - org.apache.distributedlog-distributedlog-common-4.16.4.jar - - org.apache.distributedlog-distributedlog-core-4.16.4-tests.jar - - org.apache.distributedlog-distributedlog-core-4.16.4.jar - - org.apache.distributedlog-distributedlog-protocol-4.16.4.jar - - org.apache.bookkeeper.stats-codahale-metrics-provider-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-slogger-api-4.16.4.jar - - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.16.4.jar - - org.apache.bookkeeper-native-io-4.16.4.jar + - org.apache.bookkeeper-bookkeeper-common-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-common-allocator-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-proto-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-server-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-tools-framework-4.16.5.jar + - org.apache.bookkeeper-circe-checksum-4.16.5.jar + - org.apache.bookkeeper-cpu-affinity-4.16.5.jar + - org.apache.bookkeeper-statelib-4.16.5.jar + - org.apache.bookkeeper-stream-storage-api-4.16.5.jar + - org.apache.bookkeeper-stream-storage-common-4.16.5.jar + - org.apache.bookkeeper-stream-storage-java-client-4.16.5.jar + - org.apache.bookkeeper-stream-storage-java-client-base-4.16.5.jar + - org.apache.bookkeeper-stream-storage-proto-4.16.5.jar + - org.apache.bookkeeper-stream-storage-server-4.16.5.jar + - org.apache.bookkeeper-stream-storage-service-api-4.16.5.jar + - org.apache.bookkeeper-stream-storage-service-impl-4.16.5.jar + - org.apache.bookkeeper.http-http-server-4.16.5.jar + - org.apache.bookkeeper.http-vertx-http-server-4.16.5.jar + - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.16.5.jar + - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.16.5.jar + - org.apache.distributedlog-distributedlog-common-4.16.5.jar + - org.apache.distributedlog-distributedlog-core-4.16.5-tests.jar + - org.apache.distributedlog-distributedlog-core-4.16.5.jar + - org.apache.distributedlog-distributedlog-protocol-4.16.5.jar + - org.apache.bookkeeper.stats-codahale-metrics-provider-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-slogger-api-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.16.5.jar + - org.apache.bookkeeper-native-io-4.16.5.jar * Apache HTTP Client - org.apache.httpcomponents-httpclient-4.5.13.jar - org.apache.httpcomponents-httpcore-4.4.15.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index df04985a3a880..86e94fca32e7b 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -386,9 +386,9 @@ The Apache Software License, Version 2.0 - log4j-web-2.18.0.jar * BookKeeper - - bookkeeper-common-allocator-4.16.4.jar - - cpu-affinity-4.16.4.jar - - circe-checksum-4.16.4.jar + - bookkeeper-common-allocator-4.16.5.jar + - cpu-affinity-4.16.5.jar + - circe-checksum-4.16.5.jar * AirCompressor - aircompressor-0.20.jar * AsyncHttpClient diff --git a/pom.xml b/pom.xml index 80a330f88a6db..12d1c20cb1410 100644 --- a/pom.xml +++ b/pom.xml @@ -136,7 +136,7 @@ flexible messaging model and an intuitive client API. 1.26.0 - 4.16.4 + 4.16.5 3.9.2 1.5.0 1.10.0 diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 079b033c0942b..fee68f159b93c 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -430,21 +430,21 @@ The Apache Software License, Version 2.0 - async-http-client-2.12.1.jar - async-http-client-netty-utils-2.12.1.jar * Apache Bookkeeper - - bookkeeper-common-4.16.4.jar - - bookkeeper-common-allocator-4.16.4.jar - - bookkeeper-proto-4.16.4.jar - - bookkeeper-server-4.16.4.jar - - bookkeeper-stats-api-4.16.4.jar - - bookkeeper-tools-framework-4.16.4.jar - - circe-checksum-4.16.4.jar - - codahale-metrics-provider-4.16.4.jar - - cpu-affinity-4.16.4.jar - - http-server-4.16.4.jar - - prometheus-metrics-provider-4.16.4.jar - - codahale-metrics-provider-4.16.4.jar - - bookkeeper-slogger-api-4.16.4.jar - - bookkeeper-slogger-slf4j-4.16.4.jar - - native-io-4.16.4.jar + - bookkeeper-common-4.16.5.jar + - bookkeeper-common-allocator-4.16.5.jar + - bookkeeper-proto-4.16.5.jar + - bookkeeper-server-4.16.5.jar + - bookkeeper-stats-api-4.16.5.jar + - bookkeeper-tools-framework-4.16.5.jar + - circe-checksum-4.16.5.jar + - codahale-metrics-provider-4.16.5.jar + - cpu-affinity-4.16.5.jar + - http-server-4.16.5.jar + - prometheus-metrics-provider-4.16.5.jar + - codahale-metrics-provider-4.16.5.jar + - bookkeeper-slogger-api-4.16.5.jar + - bookkeeper-slogger-slf4j-4.16.5.jar + - native-io-4.16.5.jar * Apache Commons - commons-cli-1.5.0.jar - commons-codec-1.15.jar From d1b4588de82dc9c70b24d3755670c10b63aa511d Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Mon, 15 Apr 2024 08:43:12 +0800 Subject: [PATCH 419/699] [fix][txn]Handle exceptions in the transaction pending ack init (#21274) Co-authored-by: Baodi Shi (cherry picked from commit 5d18ff7b70f9de3b95d83f6a8fd4756b1c34567b) (cherry picked from commit 000ee6679e6c1b0f1ecbc867bbe0ab3d0c542a55) --- .../pulsar/broker/service/ServerCnx.java | 2 +- .../pendingack/impl/PendingAckHandleImpl.java | 54 ++++++++++-- .../broker/transaction/TransactionTest.java | 2 +- .../pendingack/PendingAckPersistentTest.java | 82 +++++++++++++++++++ 4 files changed, 132 insertions(+), 8 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 489a5aaee93b7..3d0f0ddf60cb7 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 @@ -1321,7 +1321,7 @@ protected void handleSubscribe(final CommandSubscribe subscribe) { // Send error back to client, only if not completed already. if (consumerFuture.completeExceptionally(exception)) { commandSender.sendErrorResponse(requestId, - BrokerServiceException.getClientErrorCode(exception), + BrokerServiceException.getClientErrorCode(exception.getCause()), exception.getCause().getMessage()); } consumers.remove(consumerId, consumerFuture); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java index 7dbe0385fd7e9..5ed271c6fd414 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java @@ -22,6 +22,7 @@ import static org.apache.bookkeeper.mledger.util.PositionAckSetUtil.compareToWithAckSet; import static org.apache.bookkeeper.mledger.util.PositionAckSetUtil.isAckSetOverlap; import com.google.common.annotations.VisibleForTesting; +import io.netty.util.Timer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -35,9 +36,11 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; @@ -45,6 +48,7 @@ import org.apache.commons.lang3.tuple.MutablePair; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.BrokerServiceException.NotAllowedException; import org.apache.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException; import org.apache.pulsar.broker.service.Consumer; @@ -53,7 +57,9 @@ import org.apache.pulsar.broker.transaction.pendingack.PendingAckHandleStats; import org.apache.pulsar.broker.transaction.pendingack.PendingAckStore; import org.apache.pulsar.broker.transaction.pendingack.TransactionPendingAckStoreProvider; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.transaction.TxnID; +import org.apache.pulsar.client.impl.Backoff; import org.apache.pulsar.common.api.proto.CommandAck.AckType; import org.apache.pulsar.common.policies.data.TransactionInPendingAckStats; import org.apache.pulsar.common.policies.data.TransactionPendingAckStats; @@ -134,6 +140,12 @@ public class PendingAckHandleImpl extends PendingAckHandleState implements Pendi public final RecoverTimeRecord recoverTime = new RecoverTimeRecord(); + private final long pendingAckInitFailureBackoffInitialTimeInMs = 100; + + public final Backoff backoff = new Backoff(pendingAckInitFailureBackoffInitialTimeInMs, TimeUnit.MILLISECONDS, + 1, TimeUnit.MINUTES, 0, TimeUnit.MILLISECONDS); + + private final Timer transactionOpTimer; public PendingAckHandleImpl(PersistentSubscription persistentSubscription) { super(State.None); @@ -153,7 +165,11 @@ public PendingAckHandleImpl(PersistentSubscription persistentSubscription) { this.pendingAckStoreProvider = this.persistentSubscription.getTopic() .getBrokerService().getPulsar().getTransactionPendingAckStoreProvider(); + transactionOpTimer = persistentSubscription.getTopic().getBrokerService().getPulsar().getTransactionTimer(); + init(); + } + private void init() { pendingAckStoreProvider.checkInitializedBefore(persistentSubscription) .thenAcceptAsync(init -> { if (init) { @@ -164,9 +180,9 @@ public PendingAckHandleImpl(PersistentSubscription persistentSubscription) { }, internalPinnedExecutor) .exceptionallyAsync(e -> { Throwable t = FutureUtil.unwrapCompletionException(e); - changeToErrorState(); + // Handling the exceptions in `exceptionHandleFuture`, + // it will be helpful to make the exception handling clearer. exceptionHandleFuture(t); - this.pendingAckStoreFuture.completeExceptionally(t); return null; }, internalPinnedExecutor); } @@ -180,9 +196,8 @@ private void initPendingAckStore() { recoverTime.setRecoverStartTime(System.currentTimeMillis()); pendingAckStore.replayAsync(this, internalPinnedExecutor); }).exceptionallyAsync(e -> { - handleCacheRequest(); - changeToErrorState(); - log.error("PendingAckHandleImpl init fail! TopicName : {}, SubName: {}", topicName, subName, e); + // Handling the exceptions in `exceptionHandleFuture`, + // it will be helpful to make the exception handling clearer. exceptionHandleFuture(e.getCause()); return null; }, internalPinnedExecutor); @@ -945,12 +960,39 @@ public void completeHandleFuture() { } public void exceptionHandleFuture(Throwable t) { - final boolean completedNow = this.pendingAckHandleCompletableFuture.completeExceptionally(t); + if (isRetryableException(t)) { + this.state = State.None; + long retryTime = backoff.next(); + log.warn("[{}][{}] Failed to init transaction pending ack. It will be retried in {} Ms", + persistentSubscription.getTopic().getName(), subName, retryTime, t); + transactionOpTimer.newTimeout((timeout) -> init(), retryTime, TimeUnit.MILLISECONDS); + return; + } + log.error("[{}] [{}] PendingAckHandleImpl init fail!", topicName, subName, t); + handleCacheRequest(); + changeToErrorState(); + // ToDo: Add a new serverError `TransactionComponentLoadFailedException` + // and before that a `Unknown` will be returned first. + this.pendingAckStoreFuture = FutureUtil.failedFuture(new BrokerServiceException( + String.format("[%s][%s] Failed to init transaction pending ack.", topicName, subName))); + final boolean completedNow = this.pendingAckHandleCompletableFuture.completeExceptionally( + new BrokerServiceException( + String.format("[%s][%s] Failed to init transaction pending ack.", topicName, subName))); if (completedNow) { recoverTime.setRecoverEndTime(System.currentTimeMillis()); } } + private static boolean isRetryableException(Throwable ex) { + Throwable realCause = FutureUtil.unwrapCompletionException(ex); + return (realCause instanceof ManagedLedgerException + && !(realCause instanceof ManagedLedgerException.ManagedLedgerFencedException) + && !(realCause instanceof ManagedLedgerException.NonRecoverableLedgerException)) + || realCause instanceof PulsarClientException.BrokerPersistenceException + || realCause instanceof PulsarClientException.LookupException + || realCause instanceof PulsarClientException.ConnectException; + } + @Override public TransactionInPendingAckStats getTransactionInPendingAckStats(TxnID txnID) { TransactionInPendingAckStats transactionInPendingAckStats = new TransactionInPendingAckStats(); 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 79b60273f8125..2ed12a753a0f4 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 @@ -1505,7 +1505,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { fail("Expect failure by PendingAckHandle closed, but success"); } catch (ExecutionException executionException){ Throwable t = executionException.getCause(); - Assert.assertTrue(t instanceof BrokerServiceException.ServiceUnitNotReadyException); + Assert.assertTrue(t instanceof BrokerServiceException); } } 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 bc537fb784f0e..2e154715ac9d8 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 @@ -21,7 +21,9 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; import static org.testng.AssertJUnit.assertFalse; import static org.testng.AssertJUnit.assertNotNull; import static org.testng.AssertJUnit.assertTrue; @@ -42,8 +44,10 @@ 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.PositionImpl; import org.apache.commons.collections4.map.LinkedMap; +import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.AbstractTopic; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerServiceException; @@ -58,6 +62,7 @@ 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.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; @@ -99,6 +104,83 @@ protected void cleanup() { super.internalCleanup(); } + /** + * Test consumer can be built successfully with retryable exception + * and get correct error with no-retryable exception. + * @throws Exception + */ + @Test(timeOut = 60000) + public void testBuildConsumerEncounterPendingAckInitFailure() throws Exception { + // 1. Prepare and make sure the consumer can be built successfully. + String topic = NAMESPACE1 + "/testUnloadSubscriptionWhenFailedInitPendingAck"; + @Cleanup + Consumer consumer1 = pulsarClient.newConsumer() + .subscriptionName("subName1") + .topic(topic) + .subscribe(); + // 2. Mock a transactionPendingAckStoreProvider to test building consumer + // failing at transactionPendingAckStoreProvider::checkInitializedBefore. + Field transactionPendingAckStoreProviderField = PulsarService.class + .getDeclaredField("transactionPendingAckStoreProvider"); + transactionPendingAckStoreProviderField.setAccessible(true); + TransactionPendingAckStoreProvider pendingAckStoreProvider = + (TransactionPendingAckStoreProvider) transactionPendingAckStoreProviderField + .get(pulsarServiceList.get(0)); + TransactionPendingAckStoreProvider mockProvider = mock(pendingAckStoreProvider.getClass()); + // 3. Test retryable exception when checkInitializedBefore: + // The consumer will be built successfully after one time retry. + when(mockProvider.checkInitializedBefore(any())) + // First, the method checkInitializedBefore will fail with a retryable exception. + .thenReturn(FutureUtil.failedFuture(new ManagedLedgerException("mock fail initialize"))) + // Then, the method will be executed successfully. + .thenReturn(CompletableFuture.completedFuture(false)); + transactionPendingAckStoreProviderField.set(pulsarServiceList.get(0), mockProvider); + @Cleanup + Consumer consumer2 = pulsarClient.newConsumer() + .subscriptionName("subName2") + .topic(topic) + .subscribe(); + + // 4. Test retryable exception when newPendingAckStore: + // The consumer will be built successfully after one time retry. + when(mockProvider.checkInitializedBefore(any())) + .thenReturn(CompletableFuture.completedFuture(true)); + + when(mockProvider.newPendingAckStore(any())) + // First, the method newPendingAckStore will fail with a retryable exception. + .thenReturn(FutureUtil.failedFuture(new ManagedLedgerException("mock fail new store"))) + // Then, the method will be executed successfully. + .thenCallRealMethod(); + transactionPendingAckStoreProviderField.set(pulsarServiceList.get(0), mockProvider); + @Cleanup + Consumer consumer3 = pulsarClient.newConsumer() + .subscriptionName("subName3") + .topic(topic) + .subscribe(); + + // 5. Test no-retryable exception: + // The consumer building will be failed without retrying. + when(mockProvider.checkInitializedBefore(any())) + // The method checkInitializedBefore will fail with a no-retryable exception without retrying. + .thenReturn(FutureUtil.failedFuture(new ManagedLedgerException + .NonRecoverableLedgerException("mock fail"))) + .thenReturn(CompletableFuture.completedFuture(false)); + @Cleanup PulsarClient pulsarClient = PulsarClient.builder() + .serviceUrl(pulsarServiceList.get(0).getBrokerServiceUrl()) + .operationTimeout(3, TimeUnit.SECONDS) + .build(); + try { + @Cleanup + Consumer consumer4 = pulsarClient.newConsumer() + .subscriptionName("subName4") + .topic(topic) + .subscribe(); + fail(); + } catch (Exception exception) { + assertTrue(exception.getMessage().contains("Failed to init transaction pending ack.")); + } + } + @Test public void individualPendingAckReplayTest() throws Exception { int messageCount = 1000; From ca62ee3aba102b79ae8b42c03d112efe19791302 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 15 Apr 2024 10:59:47 +0800 Subject: [PATCH 420/699] [fix] [broker] Prevent long deduplication cursor backlog so that topic loading wouldn't timeout (#22479) (cherry picked from commit 837f8bca7ddbbad4354f9a89e36fcd6aea1be85c) (cherry picked from commit 0fbcbb240394b241e940225c2067ccf1ac3d3b32) --- .../pulsar/broker/service/BrokerService.java | 4 +- .../persistent/MessageDeduplication.java | 18 +- .../service/persistent/PersistentTopic.java | 2 +- .../DeduplicationDisabledBrokerLevelTest.java | 163 ++++++++++++++++++ 4 files changed, 179 insertions(+), 8 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DeduplicationDisabledBrokerLevelTest.java 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 97bed8e45553f..442f14e36576d 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 @@ -580,8 +580,10 @@ protected void startStatsUpdater(int statsUpdateInitialDelayInSecs, int statsUpd } 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. int interval = pulsar().getConfiguration().getBrokerDeduplicationSnapshotFrequencyInSeconds(); - if (interval > 0 && pulsar().getConfiguration().isBrokerDeduplicationEnabled()) { + if (interval > 0) { this.deduplicationSnapshotMonitor = OrderedScheduler.newSchedulerBuilder() .name("deduplication-snapshot-monitor") .numThreads(1) 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 802dd91796127..e508661364d74 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 @@ -157,9 +157,14 @@ private CompletableFuture recoverSequenceIdsMap() { // Replay all the entries and apply all the sequence ids updates log.info("[{}] Replaying {} entries for deduplication", topic.getName(), managedCursor.getNumberOfEntries()); - CompletableFuture future = new CompletableFuture<>(); + CompletableFuture future = new CompletableFuture<>(); replayCursor(future); - return future; + return future.thenAccept(lastPosition -> { + if (lastPosition != null && snapshotCounter >= snapshotInterval) { + snapshotCounter = 0; + takeSnapshot(lastPosition); + } + }); } /** @@ -168,11 +173,11 @@ private CompletableFuture recoverSequenceIdsMap() { * * @param future future to trigger when the replay is complete */ - private void replayCursor(CompletableFuture future) { + private void replayCursor(CompletableFuture future) { managedCursor.asyncReadEntries(100, new ReadEntriesCallback() { @Override public void readEntriesComplete(List entries, Object ctx) { - + Position lastPosition = null; for (Entry entry : entries) { ByteBuf messageMetadataAndPayload = entry.getDataBuffer(); MessageMetadata md = Commands.parseMessageMetadata(messageMetadataAndPayload); @@ -182,7 +187,8 @@ public void readEntriesComplete(List entries, Object ctx) { highestSequencedPushed.put(producerName, sequenceId); highestSequencedPersisted.put(producerName, sequenceId); producerRemoved(producerName); - + snapshotCounter++; + lastPosition = entry.getPosition(); entry.release(); } @@ -191,7 +197,7 @@ public void readEntriesComplete(List entries, Object ctx) { pulsar.getExecutor().execute(() -> replayCursor(future)); } else { // Done replaying - future.complete(null); + future.complete(lastPosition); } } 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 bbb24588f3bfd..63136257a9bb3 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 @@ -196,7 +196,7 @@ public class PersistentTopic extends AbstractTopic implements Topic, AddEntryCal private volatile List shadowTopics; private final TopicName shadowSourceTopic; - static final String DEDUPLICATION_CURSOR_NAME = "pulsar.dedup"; + public static final String DEDUPLICATION_CURSOR_NAME = "pulsar.dedup"; public static boolean isDedupCursorName(String name) { return DEDUPLICATION_CURSOR_NAME.equals(name); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DeduplicationDisabledBrokerLevelTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DeduplicationDisabledBrokerLevelTest.java new file mode 100644 index 0000000000000..b06c7db1439a3 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DeduplicationDisabledBrokerLevelTest.java @@ -0,0 +1,163 @@ +/* + * 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 static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import java.time.Duration; +import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.persistent.MessageDeduplication; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +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.awaitility.reflect.WhiteboxImpl; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +public class DeduplicationDisabledBrokerLevelTest extends ProducerConsumerBase { + + private int deduplicationSnapshotFrequency = 5; + private int brokerDeduplicationEntriesInterval = 1000; + + @BeforeClass + @Override + protected void setup() throws Exception { + super.internalSetup(); + producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + protected void doInitConf() throws Exception { + this.conf.setBrokerDeduplicationEnabled(false); + this.conf.setBrokerDeduplicationSnapshotFrequencyInSeconds(deduplicationSnapshotFrequency); + this.conf.setBrokerDeduplicationEntriesInterval(brokerDeduplicationEntriesInterval); + } + + @Test + public void testNoBacklogOnDeduplication() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(topic); + final PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + final ManagedLedgerImpl ml = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); + // deduplication enabled: + // broker level: "false" + // topic level: "true". + // So it is enabled. + admin.topicPolicies().setDeduplicationStatus(topic, true); + Awaitility.await().untilAsserted(() -> { + ManagedCursorImpl cursor = + (ManagedCursorImpl) ml.getCursors().get(PersistentTopic.DEDUPLICATION_CURSOR_NAME); + assertNotNull(cursor); + }); + + // Verify: regarding deduplication cursor, messages will be acknowledged automatically. + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + producer.send("1"); + producer.send("2"); + producer.send("3"); + producer.close(); + ManagedCursorImpl cursor = (ManagedCursorImpl) ml.getCursors().get(PersistentTopic.DEDUPLICATION_CURSOR_NAME); + Awaitility.await().atMost(Duration.ofSeconds(deduplicationSnapshotFrequency * 3)).untilAsserted(() -> { + PositionImpl LAC = (PositionImpl) ml.getLastConfirmedEntry(); + PositionImpl cursorMD = (PositionImpl) cursor.getMarkDeletedPosition(); + assertTrue(LAC.compareTo(cursorMD) <= 0); + }); + + // cleanup. + admin.topics().delete(topic); + } + + @Test + public void testSnapshotCounterAfterUnload() throws Exception { + final int originalDeduplicationSnapshotFrequency = deduplicationSnapshotFrequency; + deduplicationSnapshotFrequency = 3600; + cleanup(); + setup(); + + // Create a topic and wait deduplication is started. + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(topic); + final PersistentTopic persistentTopic1 = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + final ManagedLedgerImpl ml1 = (ManagedLedgerImpl) persistentTopic1.getManagedLedger(); + admin.topicPolicies().setDeduplicationStatus(topic, true); + Awaitility.await().untilAsserted(() -> { + ManagedCursorImpl cursor1 = + (ManagedCursorImpl) ml1.getCursors().get(PersistentTopic.DEDUPLICATION_CURSOR_NAME); + assertNotNull(cursor1); + }); + final MessageDeduplication deduplication1 = persistentTopic1.getMessageDeduplication(); + + // 1. Send 999 messages, it is less than "brokerDeduplicationEntriesIntervaddl". + // 2. Unload topic. + // 3. Send 1 messages, there are 1099 messages have not been snapshot now. + // 4. Verify the snapshot has been taken. + // step 1. + final Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + for (int i = 0; i < brokerDeduplicationEntriesInterval - 1; i++) { + producer.send(i + ""); + } + int snapshotCounter1 = WhiteboxImpl.getInternalState(deduplication1, "snapshotCounter"); + assertEquals(snapshotCounter1, brokerDeduplicationEntriesInterval - 1); + admin.topics().unload(topic); + PersistentTopic persistentTopic2 = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + ManagedLedgerImpl ml2 = (ManagedLedgerImpl) persistentTopic2.getManagedLedger(); + MessageDeduplication deduplication2 = persistentTopic2.getMessageDeduplication(); + admin.topicPolicies().setDeduplicationStatus(topic, true); + Awaitility.await().untilAsserted(() -> { + ManagedCursorImpl cursor = + (ManagedCursorImpl) ml2.getCursors().get(PersistentTopic.DEDUPLICATION_CURSOR_NAME); + assertNotNull(cursor); + }); + // step 3. + producer.send("last message"); + ml2.trimConsumedLedgersInBackground(new CompletableFuture<>()); + // step 4. + Awaitility.await().untilAsserted(() -> { + int snapshotCounter3 = WhiteboxImpl.getInternalState(deduplication2, "snapshotCounter"); + assertTrue(snapshotCounter3 < brokerDeduplicationEntriesInterval); + // Since https://github.com/apache/pulsar/pull/22034 has not been cherry-pick into branch-3.0, there + // should be 2 ledgers. + // Verify: the previous ledger will be removed because all messages have been acked. + assertEquals(ml2.getLedgersInfo().size(), 1); + }); + + // cleanup. + producer.close(); + admin.topics().delete(topic); + deduplicationSnapshotFrequency = originalDeduplicationSnapshotFrequency; + cleanup(); + setup(); + } +} From ba98afe1225ea8433f7fe13620f01e11987bf58e Mon Sep 17 00:00:00 2001 From: Hang Chen Date: Wed, 13 Mar 2024 14:52:43 +0800 Subject: [PATCH 421/699] [improve] [broker] Servlet support response compression (#21667) (cherry picked from commit 7a4e16a8373754a6bc4713dcfe9d06c674ce3758) (cherry picked from commit 93664d769cbf32869daf6a1f216c5a8fed3625dc) --- .../apache/pulsar/broker/web/WebService.java | 16 +++-- .../pulsar/broker/web/WebServiceTest.java | 72 +++++++++++++++++++ 2 files changed, 82 insertions(+), 6 deletions(-) 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 eada0436f4d61..82d2fda14f4d7 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 @@ -44,6 +44,7 @@ import org.eclipse.jetty.server.handler.RequestLogHandler; import org.eclipse.jetty.server.handler.ResourceHandler; import org.eclipse.jetty.server.handler.StatisticsHandler; +import org.eclipse.jetty.server.handler.gzip.GzipHandler; import org.eclipse.jetty.servlet.FilterHolder; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; @@ -259,15 +260,18 @@ public void addFilters(ServletContextHandler context, boolean requiresAuthentica public void addServlet(String path, ServletHolder servletHolder, boolean requiresAuthentication, Map attributeMap) { - ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS); + ServletContextHandler servletContextHandler = new ServletContextHandler(ServletContextHandler.SESSIONS); // Notice: each context path should be unique, but there's nothing here to verify that - context.setContextPath(path); - context.addServlet(servletHolder, MATCH_ALL); + servletContextHandler.setContextPath(path); + servletContextHandler.addServlet(servletHolder, MATCH_ALL); if (attributeMap != null) { - attributeMap.forEach(context::setAttribute); + attributeMap.forEach(servletContextHandler::setAttribute); } - filterInitializer.addFilters(context, requiresAuthentication); - handlers.add(context); + filterInitializer.addFilters(servletContextHandler, requiresAuthentication); + + GzipHandler gzipHandler = new GzipHandler(); + gzipHandler.setHandler(servletContextHandler); + handlers.add(gzipHandler); } public void addStaticResources(String basePath, String resourcePath) { 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 b069d31dc6e0d..733964bee1101 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 @@ -21,11 +21,14 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; + import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import com.google.common.io.CharStreams; import com.google.common.io.Closeables; import io.netty.handler.ssl.util.InsecureTrustManagerFactory; + +import java.io.BufferedReader; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; @@ -42,6 +45,8 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.zip.GZIPInputStream; +import java.util.zip.ZipException; import javax.net.ssl.HttpsURLConnection; import javax.net.ssl.KeyManager; import javax.net.ssl.KeyManagerFactory; @@ -353,6 +358,73 @@ public void testBrokerReady() throws Exception { assertEquals(res.getResponseBody(), "ok"); } + @Test + public void testCompressOutputMetricsInPrometheus() throws Exception { + + setupEnv(true, false, false, false, -1, false); + + String metricsUrl = pulsar.getWebServiceAddress() + "/metrics/"; + + String[] command = {"curl", "-H", "Accept-Encoding: gzip", metricsUrl}; + + ProcessBuilder processBuilder = new ProcessBuilder(command); + Process process = processBuilder.start(); + + InputStream inputStream = process.getInputStream(); + + try { + GZIPInputStream gzipInputStream = new GZIPInputStream(inputStream); + + // Process the decompressed content + StringBuilder content = new StringBuilder(); + int data; + while ((data = gzipInputStream.read()) != -1) { + content.append((char) data); + } + log.info("Response Content: {}", content); + + process.waitFor(); + assertTrue(content.toString().contains("process_cpu_seconds_total")); + } catch (IOException e) { + log.error("Failed to decompress the content, likely the content is not compressed ", e); + fail(); + } + } + + @Test + public void testUnCompressOutputMetricsInPrometheus() throws Exception { + + setupEnv(true, false, false, false, -1, false); + + String metricsUrl = pulsar.getWebServiceAddress() + "/metrics/"; + + String[] command = {"curl", metricsUrl}; + + ProcessBuilder processBuilder = new ProcessBuilder(command); + Process process = processBuilder.start(); + + InputStream inputStream = process.getInputStream(); + try { + GZIPInputStream gzipInputStream = new GZIPInputStream(inputStream); + fail(); + } catch (IOException e) { + log.error("Failed to decompress the content, likely the content is not compressed ", e); + assertTrue(e instanceof ZipException); + } + + BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream)); + StringBuilder content = new StringBuilder(); + String line; + while ((line = reader.readLine()) != null) { + content.append(line + "\n"); + } + + log.info("Response Content: {}", content); + + process.waitFor(); + assertTrue(content.toString().contains("process_cpu_seconds_total")); + } + private String makeHttpRequest(boolean useTls, boolean useAuth) throws Exception { InputStream response = null; try { From 7f05b6e1f3e88a76c7ff101106198930ae4a2147 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 9 Apr 2024 07:48:57 -0700 Subject: [PATCH 422/699] [improve][test] Replace usage of curl in Java test and fix stream leaks (#22463) (cherry picked from commit f3d14a6b0b15f6d3c17509b21b28a586a22e5d89) (cherry picked from commit 976399cb172bef277419980236698018d7dd40d2) --- .../pulsar/broker/web/WebServiceTest.java | 69 +++++++++---------- 1 file changed, 33 insertions(+), 36 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 733964bee1101..00c6a209c13d7 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 @@ -21,18 +21,17 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; - import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import com.google.common.io.CharStreams; import com.google.common.io.Closeables; import io.netty.handler.ssl.util.InsecureTrustManagerFactory; - import java.io.BufferedReader; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; +import java.net.HttpURLConnection; import java.net.URL; import java.security.KeyStore; import java.security.PrivateKey; @@ -360,68 +359,66 @@ public void testBrokerReady() throws Exception { @Test public void testCompressOutputMetricsInPrometheus() throws Exception { - setupEnv(true, false, false, false, -1, false); String metricsUrl = pulsar.getWebServiceAddress() + "/metrics/"; - String[] command = {"curl", "-H", "Accept-Encoding: gzip", metricsUrl}; + URL url = new URL(metricsUrl); + HttpURLConnection connection = (HttpURLConnection) url.openConnection(); + connection.setRequestMethod("GET"); + connection.setRequestProperty("Accept-Encoding", "gzip"); - ProcessBuilder processBuilder = new ProcessBuilder(command); - Process process = processBuilder.start(); - - InputStream inputStream = process.getInputStream(); - - try { - GZIPInputStream gzipInputStream = new GZIPInputStream(inputStream); + StringBuilder content = new StringBuilder(); - // Process the decompressed content - StringBuilder content = new StringBuilder(); - int data; - while ((data = gzipInputStream.read()) != -1) { - content.append((char) data); + try (InputStream inputStream = connection.getInputStream()) { + try (GZIPInputStream gzipInputStream = new GZIPInputStream(inputStream)) { + // Process the decompressed content + int data; + while ((data = gzipInputStream.read()) != -1) { + content.append((char) data); + } } - log.info("Response Content: {}", content); - process.waitFor(); + log.info("Response Content: {}", content); assertTrue(content.toString().contains("process_cpu_seconds_total")); } catch (IOException e) { log.error("Failed to decompress the content, likely the content is not compressed ", e); fail(); + } finally { + connection.disconnect(); } } @Test public void testUnCompressOutputMetricsInPrometheus() throws Exception { - setupEnv(true, false, false, false, -1, false); String metricsUrl = pulsar.getWebServiceAddress() + "/metrics/"; - String[] command = {"curl", metricsUrl}; + URL url = new URL(metricsUrl); + HttpURLConnection connection = (HttpURLConnection) url.openConnection(); + connection.setRequestMethod("GET"); - ProcessBuilder processBuilder = new ProcessBuilder(command); - Process process = processBuilder.start(); + StringBuilder content = new StringBuilder(); - InputStream inputStream = process.getInputStream(); - try { - GZIPInputStream gzipInputStream = new GZIPInputStream(inputStream); - fail(); - } catch (IOException e) { - log.error("Failed to decompress the content, likely the content is not compressed ", e); - assertTrue(e instanceof ZipException); - } + try (InputStream inputStream = connection.getInputStream()) { + try (GZIPInputStream gzipInputStream = new GZIPInputStream(inputStream)) { + fail(); + } catch (IOException e) { + assertTrue(e instanceof ZipException); + } - BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream)); - StringBuilder content = new StringBuilder(); - String line; - while ((line = reader.readLine()) != null) { - content.append(line + "\n"); + BufferedReader reader = new BufferedReader(new InputStreamReader(inputStream)); + String line; + while ((line = reader.readLine()) != null) { + content.append(line + "\n"); + } + } finally { + connection.disconnect(); } log.info("Response Content: {}", content); - process.waitFor(); assertTrue(content.toString().contains("process_cpu_seconds_total")); } From 961ecce291dea9a58dd55b54b916846c98765696 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 12 Apr 2024 10:35:09 -0700 Subject: [PATCH 423/699] [improve][broker] Improve Gzip compression, allow excluding specific paths or disabling it (#22370) (cherry picked from commit 15ed6595af5489a007db82002ed3391589bad54d) (cherry picked from commit cdc5af157754bbcb7a44559189e16a5438d294cd) --- .../pulsar/broker/ServiceConfiguration.java | 13 +++++ .../pulsar/broker/web/GzipHandlerUtil.java | 48 +++++++++++++++++++ .../apache/pulsar/broker/web/WebService.java | 10 ++-- .../proxy/server/AdminProxyHandler.java | 1 + 4 files changed, 66 insertions(+), 6 deletions(-) create mode 100644 pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.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 f4337eb18f64c..b833223cb1e3e 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 @@ -331,6 +331,19 @@ public class ServiceConfiguration implements PulsarConfiguration { + "(0 to disable limiting)") private int maxHttpServerConnections = 2048; + @FieldContext(category = CATEGORY_SERVER, doc = + "Gzip compression is enabled by default. Specific paths can be excluded from compression.\n" + + "There are 2 syntaxes supported, Servlet url-pattern based, and Regex based.\n" + + "If the spec starts with '^' the spec is assumed to be a regex based path spec and will match " + + "with normal Java regex rules.\n" + + "If the spec starts with '/' then spec is assumed to be a Servlet url-pattern rules path spec " + + "for either an exact match or prefix based match.\n" + + "If the spec starts with '*.' then spec is assumed to be a Servlet url-pattern rules path spec " + + "for a suffix based match.\n" + + "All other syntaxes are unsupported.\n" + + "Disable all compression with ^.* or ^.*$") + private List httpServerGzipCompressionExcludedPaths = new ArrayList<>(); + @FieldContext(category = CATEGORY_SERVER, doc = "Whether to enable the delayed delivery for messages.") private boolean delayedDeliveryEnabled = true; diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java new file mode 100644 index 0000000000000..37c9c05e5d53c --- /dev/null +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.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.web; + +import java.util.List; +import org.eclipse.jetty.server.Handler; +import org.eclipse.jetty.server.handler.gzip.GzipHandler; + +public class GzipHandlerUtil { + public static Handler wrapWithGzipHandler(Handler innerHandler, List gzipCompressionExcludedPaths) { + Handler wrappedHandler; + if (isGzipCompressionCompletelyDisabled(gzipCompressionExcludedPaths)) { + // no need to add GZIP handler if it's disabled by setting the excluded path to "^.*" or "^.*$" + wrappedHandler = innerHandler; + } else { + // add GZIP handler which is active when the request contains "Accept-Encoding: gzip" header + GzipHandler gzipHandler = new GzipHandler(); + gzipHandler.setHandler(innerHandler); + if (gzipCompressionExcludedPaths != null && gzipCompressionExcludedPaths.size() > 0) { + gzipHandler.setExcludedPaths(gzipCompressionExcludedPaths.toArray(new String[0])); + } + wrappedHandler = gzipHandler; + } + return wrappedHandler; + } + + public static boolean isGzipCompressionCompletelyDisabled(List gzipCompressionExcludedPaths) { + return gzipCompressionExcludedPaths != null && gzipCompressionExcludedPaths.size() == 1 + && (gzipCompressionExcludedPaths.get(0).equals("^.*") + || gzipCompressionExcludedPaths.get(0).equals("^.*$")); + } +} 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 82d2fda14f4d7..7395144167af9 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 @@ -44,7 +44,6 @@ import org.eclipse.jetty.server.handler.RequestLogHandler; import org.eclipse.jetty.server.handler.ResourceHandler; import org.eclipse.jetty.server.handler.StatisticsHandler; -import org.eclipse.jetty.server.handler.gzip.GzipHandler; import org.eclipse.jetty.servlet.FilterHolder; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; @@ -269,9 +268,7 @@ public void addServlet(String path, ServletHolder servletHolder, boolean require } filterInitializer.addFilters(servletContextHandler, requiresAuthentication); - GzipHandler gzipHandler = new GzipHandler(); - gzipHandler.setHandler(servletContextHandler); - handlers.add(gzipHandler); + handlers.add(servletContextHandler); } public void addStaticResources(String basePath, String resourcePath) { @@ -295,8 +292,10 @@ public void start() throws PulsarServerException { ContextHandlerCollection contexts = new ContextHandlerCollection(); contexts.setHandlers(handlers.toArray(new Handler[handlers.size()])); + Handler handlerForContexts = GzipHandlerUtil.wrapWithGzipHandler(contexts, + pulsar.getConfig().getHttpServerGzipCompressionExcludedPaths()); HandlerCollection handlerCollection = new HandlerCollection(); - handlerCollection.setHandlers(new Handler[] { contexts, new DefaultHandler(), requestLogHandler }); + handlerCollection.setHandlers(new Handler[] {handlerForContexts, new DefaultHandler(), requestLogHandler}); // Metrics handler StatisticsHandler stats = new StatisticsHandler(); @@ -307,7 +306,6 @@ public void start() throws PulsarServerException { } catch (IllegalArgumentException e) { // Already registered. Eg: in unit tests } - handlers.add(stats); server.setHandler(stats); server.start(); 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 c528ceb2cf5b7..caaa99c5d40cc 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 @@ -158,6 +158,7 @@ protected HttpClient createHttpClient() throws ServletException { client.start(); // Content must not be decoded, otherwise the client gets confused. + // Allow encoded content, such as "Content-Encoding: gzip", to pass through without decoding it. client.getContentDecoderFactories().clear(); // Pass traffic to the client, only intercept what's necessary. From 0b001837598852b38f9406847fedb189528640e0 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 13 Apr 2024 10:00:23 -0700 Subject: [PATCH 424/699] [fix][broker] Optimize /metrics, fix unbounded request queue issue and fix race conditions in metricsBufferResponse mode (#22494) (cherry picked from commit 7009071b6d53bbc3d740ea99cdc0c010692679ab) (cherry picked from commit 5f9d7c5d6234f7a51f2a8412ad96ae7a0547632b) --- conf/proxy.conf | 6 +- .../prometheus/PrometheusMetricsServlet.java | 149 +++++--- .../pulsar/broker/stats/TimeWindow.java | 94 ----- .../pulsar/broker/stats/WindowWrap.java | 56 --- .../stats/prometheus/MetricsExports.java | 68 ++++ .../PrometheusMetricsGenerator.java | 328 +++++++++++------- .../PulsarPrometheusMetricsServlet.java | 140 +++++++- .../broker/stats/prometheus/TopicStats.java | 12 +- .../pulsar/PrometheusMetricsTestUtil.java | 84 +++++ .../persistent/BucketDelayedDeliveryTest.java | 6 +- .../persistent/PersistentTopicTest.java | 4 +- .../service/schema/SchemaServiceTest.java | 4 +- .../broker/stats/ConsumerStatsTest.java | 4 +- .../broker/stats/MetadataStoreStatsTest.java | 6 +- .../broker/stats/PrometheusMetricsTest.java | 114 +++--- .../broker/stats/SubscriptionStatsTest.java | 4 +- .../pulsar/broker/stats/TimeWindowTest.java | 83 ----- .../broker/stats/TransactionMetricsTest.java | 18 +- .../buffer/TransactionBufferClientTest.java | 4 +- .../pendingack/PendingAckPersistentTest.java | 4 +- .../pulsar/broker/web/WebServiceTest.java | 4 +- .../common/util/SimpleTextOutputStream.java | 16 +- .../proxy/server/ProxyConfiguration.java | 6 + .../pulsar/proxy/server/ProxyService.java | 3 +- .../proxy/server/ProxyServiceStarter.java | 40 ++- 25 files changed, 734 insertions(+), 523 deletions(-) delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/TimeWindow.java delete mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/WindowWrap.java create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/MetricsExports.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java delete mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TimeWindowTest.java diff --git a/conf/proxy.conf b/conf/proxy.conf index 8285e1cb75320..5a9d433f39ceb 100644 --- a/conf/proxy.conf +++ b/conf/proxy.conf @@ -376,5 +376,7 @@ zooKeeperCacheExpirySeconds=-1 enableProxyStatsEndpoints=true # Whether the '/metrics' endpoint requires authentication. Defaults to true authenticateMetricsEndpoint=true -# Enable cache metrics data, default value is false -metricsBufferResponse=false +# Time in milliseconds that metrics endpoint would time out. Default is 30s. +# Set it to 0 to disable timeout. +metricsServletTimeoutMs=30000 + diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java index 64d1fcdab6f14..8a41bed29d44f 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java @@ -25,9 +25,13 @@ import java.util.List; 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.AtomicBoolean; import javax.servlet.AsyncContext; +import javax.servlet.AsyncEvent; +import javax.servlet.AsyncListener; import javax.servlet.ServletException; -import javax.servlet.ServletOutputStream; import javax.servlet.http.HttpServlet; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; @@ -35,67 +39,132 @@ import org.slf4j.LoggerFactory; public class PrometheusMetricsServlet extends HttpServlet { - private static final long serialVersionUID = 1L; - private static final int HTTP_STATUS_OK_200 = 200; - private static final int HTTP_STATUS_INTERNAL_SERVER_ERROR_500 = 500; - - private final long metricsServletTimeoutMs; - private final String cluster; + static final int HTTP_STATUS_OK_200 = 200; + static final int HTTP_STATUS_INTERNAL_SERVER_ERROR_500 = 500; + protected final long metricsServletTimeoutMs; + protected final String cluster; protected List metricsProviders; - private ExecutorService executor = null; + protected ExecutorService executor = null; + protected final int executorMaxThreads; public PrometheusMetricsServlet(long metricsServletTimeoutMs, String cluster) { + this(metricsServletTimeoutMs, cluster, 1); + } + + public PrometheusMetricsServlet(long metricsServletTimeoutMs, String cluster, int executorMaxThreads) { this.metricsServletTimeoutMs = metricsServletTimeoutMs; this.cluster = cluster; + this.executorMaxThreads = executorMaxThreads; } @Override public void init() throws ServletException { - executor = Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("prometheus-stats")); + if (executorMaxThreads > 0) { + executor = + Executors.newScheduledThreadPool(executorMaxThreads, new DefaultThreadFactory("prometheus-stats")); + } } @Override protected void doGet(HttpServletRequest request, HttpServletResponse response) { AsyncContext context = request.startAsync(); - context.setTimeout(metricsServletTimeoutMs); - executor.execute(() -> { - long start = System.currentTimeMillis(); - HttpServletResponse res = (HttpServletResponse) context.getResponse(); - try { - res.setStatus(HTTP_STATUS_OK_200); - res.setContentType("text/plain;charset=utf-8"); - generateMetrics(cluster, res.getOutputStream()); - } catch (Exception e) { - long end = System.currentTimeMillis(); - long time = end - start; - if (e instanceof EOFException) { - // NO STACKTRACE - log.error("Failed to send metrics, " - + "likely the client or this server closed " - + "the connection due to a timeout ({} ms elapsed): {}", time, e + ""); - } else { - log.error("Failed to generate prometheus stats, {} ms elapsed", time, e); + // set hard timeout to 2 * timeout + if (metricsServletTimeoutMs > 0) { + context.setTimeout(metricsServletTimeoutMs * 2); + } + long startNanos = System.nanoTime(); + AtomicBoolean taskStarted = new AtomicBoolean(false); + Future future = executor.submit(() -> { + taskStarted.set(true); + long elapsedNanos = System.nanoTime() - startNanos; + // check if the request has been timed out, implement a soft timeout + // so that response writing can continue to up to 2 * timeout + if (metricsServletTimeoutMs > 0 && elapsedNanos > TimeUnit.MILLISECONDS.toNanos(metricsServletTimeoutMs)) { + log.warn("Prometheus metrics request was too long in queue ({}ms). Skipping sending metrics.", + TimeUnit.NANOSECONDS.toMillis(elapsedNanos)); + if (!response.isCommitted()) { + response.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); } - res.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); - } finally { - long end = System.currentTimeMillis(); - long time = end - start; - try { - context.complete(); - } catch (IllegalStateException e) { - // this happens when metricsServletTimeoutMs expires - // java.lang.IllegalStateException: AsyncContext completed and/or Request lifecycle recycled - log.error("Failed to generate prometheus stats, " - + "this is likely due to metricsServletTimeoutMs: {} ms elapsed: {}", time, e + ""); + context.complete(); + return; + } + handleAsyncMetricsRequest(context); + }); + context.addListener(new AsyncListener() { + @Override + public void onComplete(AsyncEvent asyncEvent) throws IOException { + if (!taskStarted.get()) { + future.cancel(false); } } + + @Override + public void onTimeout(AsyncEvent asyncEvent) throws IOException { + if (!taskStarted.get()) { + future.cancel(false); + } + log.warn("Prometheus metrics request timed out"); + HttpServletResponse res = (HttpServletResponse) context.getResponse(); + if (!res.isCommitted()) { + res.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + } + context.complete(); + } + + @Override + public void onError(AsyncEvent asyncEvent) throws IOException { + if (!taskStarted.get()) { + future.cancel(false); + } + } + + @Override + public void onStartAsync(AsyncEvent asyncEvent) throws IOException { + + } }); + + } + + private void handleAsyncMetricsRequest(AsyncContext context) { + long start = System.currentTimeMillis(); + HttpServletResponse res = (HttpServletResponse) context.getResponse(); + try { + generateMetricsSynchronously(res); + } catch (Exception e) { + long end = System.currentTimeMillis(); + long time = end - start; + if (e instanceof EOFException) { + // NO STACKTRACE + log.error("Failed to send metrics, " + + "likely the client or this server closed " + + "the connection due to a timeout ({} ms elapsed): {}", time, e + ""); + } else { + log.error("Failed to generate prometheus stats, {} ms elapsed", time, e); + } + if (!res.isCommitted()) { + res.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + } + } finally { + long end = System.currentTimeMillis(); + long time = end - start; + try { + context.complete(); + } catch (IllegalStateException e) { + // this happens when metricsServletTimeoutMs expires + // java.lang.IllegalStateException: AsyncContext completed and/or Request lifecycle recycled + log.error("Failed to generate prometheus stats, " + + "this is likely due to metricsServletTimeoutMs: {} ms elapsed: {}", time, e + ""); + } + } } - protected void generateMetrics(String cluster, ServletOutputStream outputStream) throws IOException { - PrometheusMetricsGeneratorUtils.generate(cluster, outputStream, metricsProviders); + private void generateMetricsSynchronously(HttpServletResponse res) throws IOException { + res.setStatus(HTTP_STATUS_OK_200); + res.setContentType("text/plain;charset=utf-8"); + PrometheusMetricsGeneratorUtils.generate(cluster, res.getOutputStream(), metricsProviders); } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/TimeWindow.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/TimeWindow.java deleted file mode 100644 index 08730189322ee..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/TimeWindow.java +++ /dev/null @@ -1,94 +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.broker.stats; - -import java.util.concurrent.atomic.AtomicReferenceArray; -import java.util.function.Function; - -public final class TimeWindow { - private final int interval; - private final int sampleCount; - private final AtomicReferenceArray> array; - - public TimeWindow(int sampleCount, int interval) { - this.sampleCount = sampleCount; - this.interval = interval; - this.array = new AtomicReferenceArray<>(sampleCount); - } - - /** - * return current time window data. - * - * @param function generate data. - * @return - */ - public synchronized WindowWrap current(Function function) { - long millis = System.currentTimeMillis(); - - if (millis < 0) { - return null; - } - int idx = calculateTimeIdx(millis); - long windowStart = calculateWindowStart(millis); - while (true) { - WindowWrap old = array.get(idx); - if (old == null) { - WindowWrap window = new WindowWrap<>(interval, windowStart, null); - if (array.compareAndSet(idx, null, window)) { - T value = null == function ? null : function.apply(null); - window.value(value); - return window; - } else { - Thread.yield(); - } - } else if (windowStart == old.start()) { - return old; - } else if (windowStart > old.start()) { - T value = null == function ? null : function.apply(old.value()); - old.value(value); - old.resetWindowStart(windowStart); - return old; - } else { - //it should never goes here - throw new IllegalStateException(); - } - } - } - - private int calculateTimeIdx(long timeMillis) { - long timeId = timeMillis / this.interval; - return (int) (timeId % sampleCount); - } - - private long calculateWindowStart(long timeMillis) { - return timeMillis - timeMillis % this.interval; - } - - public int sampleCount() { - return sampleCount; - } - - public int interval() { - return interval; - } - - public long currentWindowStart(long millis) { - return this.calculateWindowStart(millis); - } -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/WindowWrap.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/WindowWrap.java deleted file mode 100644 index 12869b82921e5..0000000000000 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/WindowWrap.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.broker.stats; - -public final class WindowWrap { - private final long interval; - private long start; - private T value; - - public WindowWrap(long interval, long windowStart, T value) { - this.interval = interval; - this.start = windowStart; - this.value = value; - } - - public long interval() { - return this.interval; - } - - public long start() { - return this.start; - } - - public T value() { - return value; - } - - public void value(T value) { - this.value = value; - } - - public WindowWrap resetWindowStart(long startTime) { - this.start = startTime; - return this; - } - - public boolean isTimeInWindow(long timeMillis) { - return start <= timeMillis && timeMillis < start + interval; - } -} diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/MetricsExports.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/MetricsExports.java new file mode 100644 index 0000000000000..b80e5747d8a5a --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/MetricsExports.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.stats.prometheus; + +import static org.apache.pulsar.common.stats.JvmMetrics.getJvmDirectMemoryUsed; +import io.prometheus.client.CollectorRegistry; +import io.prometheus.client.Gauge; +import io.prometheus.client.hotspot.DefaultExports; +import org.apache.pulsar.PulsarVersion; +import org.apache.pulsar.common.util.DirectMemoryUtils; + +public class MetricsExports { + private static boolean initialized = false; + + private MetricsExports() { + } + + public static synchronized void initialize() { + if (!initialized) { + DefaultExports.initialize(); + register(CollectorRegistry.defaultRegistry); + initialized = true; + } + } + + public static void register(CollectorRegistry registry) { + Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Gauge.Child() { + @Override + public double get() { + return getJvmDirectMemoryUsed(); + } + }).register(registry); + + Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Gauge.Child() { + @Override + public double get() { + return DirectMemoryUtils.jvmMaxDirectMemory(); + } + }).register(registry); + + // metric to export pulsar version info + Gauge.build("pulsar_version_info", "-") + .labelNames("version", "commit").create() + .setChild(new Gauge.Child() { + @Override + public double get() { + return 1.0; + } + }, PulsarVersion.getVersion(), PulsarVersion.getGitSha()) + .register(registry); + } +} 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 293ac30ecd123..da77fe9b3341d 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 @@ -20,41 +20,40 @@ import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGeneratorUtils.generateSystemMetrics; import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGeneratorUtils.getTypeStr; -import static org.apache.pulsar.common.stats.JvmMetrics.getJvmDirectMemoryUsed; import io.netty.buffer.ByteBuf; -import io.netty.buffer.UnpooledByteBufAllocator; +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.CompositeByteBuf; +import io.netty.buffer.PooledByteBufAllocator; import io.prometheus.client.Collector; -import io.prometheus.client.CollectorRegistry; -import io.prometheus.client.Gauge; -import io.prometheus.client.Gauge.Child; -import io.prometheus.client.hotspot.DefaultExports; +import java.io.BufferedOutputStream; import java.io.IOException; import java.io.OutputStream; -import java.io.StringWriter; +import java.io.OutputStreamWriter; import java.io.Writer; -import java.nio.ByteBuffer; +import java.nio.CharBuffer; +import java.nio.charset.StandardCharsets; +import java.time.Clock; import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.stats.NullStatsProvider; import org.apache.bookkeeper.stats.StatsProvider; -import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.broker.PulsarService; -import org.apache.pulsar.broker.stats.TimeWindow; -import org.apache.pulsar.broker.stats.WindowWrap; 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.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.stats.Metrics; -import org.apache.pulsar.common.util.DirectMemoryUtils; import org.apache.pulsar.common.util.SimpleTextOutputStream; -import org.eclipse.jetty.server.HttpOutput; /** * Generate metrics aggregated at the namespace level and optionally at a topic level and formats them out @@ -63,123 +62,80 @@ * href="https://prometheus.io/docs/instrumenting/exposition_formats/">Exposition Formats */ @Slf4j -public class PrometheusMetricsGenerator { - private static volatile TimeWindow timeWindow; - private static final int MAX_COMPONENTS = 64; - - static { - DefaultExports.initialize(); - - Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() { - @Override - public double get() { - return getJvmDirectMemoryUsed(); - } - }).register(CollectorRegistry.defaultRegistry); - - Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() { - @Override - public double get() { - return DirectMemoryUtils.jvmMaxDirectMemory(); - } - }).register(CollectorRegistry.defaultRegistry); - - // metric to export pulsar version info - Gauge.build("pulsar_version_info", "-") - .labelNames("version", "commit").create() - .setChild(new Child() { - @Override - public double get() { - return 1.0; - } - }, PulsarVersion.getVersion(), PulsarVersion.getGitSha()) - .register(CollectorRegistry.defaultRegistry); - } - - public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, - boolean includeProducerMetrics, OutputStream out) throws IOException { - generate(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, false, out, null); - } - - public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, - boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, - OutputStream out) throws IOException { - generate(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, - splitTopicAndPartitionIndexLabel, out, null); - } - - public static synchronized void generate(PulsarService pulsar, boolean includeTopicMetrics, - boolean includeConsumerMetrics, boolean includeProducerMetrics, - boolean splitTopicAndPartitionIndexLabel, OutputStream out, - List metricsProviders) throws IOException { - ByteBuf buffer; - boolean exposeBufferMetrics = pulsar.getConfiguration().isMetricsBufferResponse(); +public class PrometheusMetricsGenerator implements AutoCloseable { + private static final int DEFAULT_INITIAL_BUFFER_SIZE = 1024 * 1024; // 1MB + private static final int MINIMUM_FOR_MAX_COMPONENTS = 64; + + private volatile MetricsBuffer metricsBuffer; + private static AtomicReferenceFieldUpdater metricsBufferFieldUpdater = + AtomicReferenceFieldUpdater.newUpdater(PrometheusMetricsGenerator.class, MetricsBuffer.class, + "metricsBuffer"); + private volatile boolean closed; + + public static class MetricsBuffer { + private final CompletableFuture bufferFuture; + private final long createTimeslot; + private final AtomicInteger refCnt = new AtomicInteger(2); + + MetricsBuffer(long timeslot) { + bufferFuture = new CompletableFuture<>(); + createTimeslot = timeslot; + } - if (!exposeBufferMetrics) { - buffer = generate0(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, - splitTopicAndPartitionIndexLabel, metricsProviders); - } else { - if (null == timeWindow) { - int period = pulsar.getConfiguration().getManagedLedgerStatsPeriodSeconds(); - timeWindow = new TimeWindow<>(1, (int) TimeUnit.SECONDS.toMillis(period)); - } - WindowWrap window = timeWindow.current(oldBuf -> { - // release expired buffer, in case of memory leak - if (oldBuf != null && oldBuf.refCnt() > 0) { - oldBuf.release(); - log.debug("Cached metrics buffer released"); - } + public CompletableFuture getBufferFuture() { + return bufferFuture; + } - try { - ByteBuf buf = generate0(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, - splitTopicAndPartitionIndexLabel, metricsProviders); - log.debug("Generated metrics buffer size {}", buf.readableBytes()); - return buf; - } catch (IOException e) { - log.error("Generate metrics failed", e); - //return empty buffer if exception happens - return PulsarByteBufAllocator.DEFAULT.heapBuffer(0); - } - }); + long getCreateTimeslot() { + return createTimeslot; + } - if (null == window || null == window.value()) { - return; - } - buffer = window.value(); - log.debug("Current window start {}, current cached buf size {}", window.start(), buffer.readableBytes()); + /** + * Retain the buffer. This is allowed, only when the buffer is not already released. + * + * @return true if the buffer is retained successfully, false otherwise. + */ + boolean retain() { + return refCnt.updateAndGet(x -> x > 0 ? x + 1 : x) > 0; } - try { - if (out instanceof HttpOutput) { - HttpOutput output = (HttpOutput) out; - //no mem_copy and memory allocations here - ByteBuffer[] buffers = buffer.nioBuffers(); - for (ByteBuffer buffer0 : buffers) { - output.write(buffer0); - } - } else { - //read data from buffer and write it to output stream, with no more heap buffer(byte[]) allocation. - //not modify buffer readIndex/writeIndex here. - int readIndex = buffer.readerIndex(); - int readableBytes = buffer.readableBytes(); - for (int i = 0; i < readableBytes; i++) { - out.write(buffer.getByte(readIndex + i)); - } - } - } finally { - if (!exposeBufferMetrics && buffer.refCnt() > 0) { - buffer.release(); - log.debug("Metrics buffer released."); + /** + * Release the buffer. + */ + public void release() { + int newValue = refCnt.decrementAndGet(); + if (newValue == 0) { + bufferFuture.whenComplete((byteBuf, throwable) -> { + if (byteBuf != null) { + byteBuf.release(); + } + }); } } } - private static ByteBuf generate0(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, - boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, - List metricsProviders) throws IOException { - //Use unpooled buffers here to avoid direct buffer usage increasing. - //when write out 200MB data, MAX_COMPONENTS = 64 needn't mem_copy. see: CompositeByteBuf#consolidateIfNeeded() - ByteBuf buf = UnpooledByteBufAllocator.DEFAULT.compositeDirectBuffer(MAX_COMPONENTS); + private final PulsarService pulsar; + private final boolean includeTopicMetrics; + private final boolean includeConsumerMetrics; + private final boolean includeProducerMetrics; + private final boolean splitTopicAndPartitionIndexLabel; + private final Clock clock; + + private volatile int initialBufferSize = DEFAULT_INITIAL_BUFFER_SIZE; + + public PrometheusMetricsGenerator(PulsarService pulsar, boolean includeTopicMetrics, + boolean includeConsumerMetrics, boolean includeProducerMetrics, + boolean splitTopicAndPartitionIndexLabel, Clock clock) { + this.pulsar = pulsar; + this.includeTopicMetrics = includeTopicMetrics; + this.includeConsumerMetrics = includeConsumerMetrics; + this.includeProducerMetrics = includeProducerMetrics; + this.splitTopicAndPartitionIndexLabel = splitTopicAndPartitionIndexLabel; + this.clock = clock; + } + + private ByteBuf generate0(List metricsProviders) { + ByteBuf buf = allocateMultipartCompositeDirectBuffer(); boolean exceptionHappens = false; //Used in namespace/topic and transaction aggregators as share metric names PrometheusMetricStreams metricStreams = new PrometheusMetricStreams(); @@ -221,10 +177,34 @@ private static ByteBuf generate0(PulsarService pulsar, boolean includeTopicMetri //if exception happens, release buffer if (exceptionHappens) { buf.release(); + } else { + // for the next time, the initial buffer size will be suggested by the last buffer size + initialBufferSize = Math.max(DEFAULT_INITIAL_BUFFER_SIZE, buf.readableBytes()); } } } + private 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; + if (byteBufAllocator instanceof PooledByteBufAllocator) { + PooledByteBufAllocator pooledByteBufAllocator = (PooledByteBufAllocator) byteBufAllocator; + chunkSize = Math.max(pooledByteBufAllocator.metric().chunkSize(), DEFAULT_INITIAL_BUFFER_SIZE); + } else { + chunkSize = DEFAULT_INITIAL_BUFFER_SIZE; + } + CompositeByteBuf buf = byteBufAllocator.compositeDirectBuffer( + Math.max(MINIMUM_FOR_MAX_COMPONENTS, (initialBufferSize / chunkSize) + 1)); + int totalLen = 0; + while (totalLen < initialBufferSize) { + totalLen += chunkSize; + buf.addComponent(false, byteBufAllocator.directBuffer(chunkSize)); + } + return buf; + } + private static void generateBrokerBasicMetrics(PulsarService pulsar, SimpleTextOutputStream stream) { String clusterName = pulsar.getConfiguration().getClusterName(); // generate managedLedgerCache metrics @@ -270,12 +250,13 @@ private static void parseMetricsToPrometheusMetrics(Collection metrics, String name = key.substring(0, nameIndex); value = key.substring(nameIndex + 1); if (!names.contains(name)) { - stream.write("# TYPE ").write(name.replace("brk_", "pulsar_")).write(' ') - .write(getTypeStr(metricType)).write("\n"); + stream.write("# TYPE "); + writeNameReplacingBrkPrefix(stream, name); + stream.write(' ').write(getTypeStr(metricType)).write("\n"); names.add(name); } - stream.write(name.replace("brk_", "pulsar_")) - .write("{cluster=\"").write(cluster).write('"'); + writeNameReplacingBrkPrefix(stream, name); + stream.write("{cluster=\"").write(cluster).write('"'); } catch (Exception e) { continue; } @@ -284,12 +265,13 @@ private static void parseMetricsToPrometheusMetrics(Collection metrics, String name = entry.getKey(); if (!names.contains(name)) { - stream.write("# TYPE ").write(entry.getKey().replace("brk_", "pulsar_")).write(' ') - .write(getTypeStr(metricType)).write('\n'); + stream.write("# TYPE "); + writeNameReplacingBrkPrefix(stream, entry.getKey()); + stream.write(' ').write(getTypeStr(metricType)).write('\n'); names.add(name); } - stream.write(name.replace("brk_", "pulsar_")) - .write("{cluster=\"").write(cluster).write('"'); + writeNameReplacingBrkPrefix(stream, name); + stream.write("{cluster=\"").write(cluster).write('"'); } //to avoid quantile label duplicated @@ -312,18 +294,98 @@ private static void parseMetricsToPrometheusMetrics(Collection metrics, } } + private static SimpleTextOutputStream writeNameReplacingBrkPrefix(SimpleTextOutputStream stream, String name) { + if (name.startsWith("brk_")) { + return stream.write("pulsar_").write(CharBuffer.wrap(name).position("brk_".length())); + } else { + return stream.write(name); + } + } + private static void generateManagedLedgerBookieClientMetrics(PulsarService pulsar, SimpleTextOutputStream stream) { StatsProvider statsProvider = pulsar.getManagedLedgerClientFactory().getStatsProvider(); if (statsProvider instanceof NullStatsProvider) { return; } - try { - Writer writer = new StringWriter(); + try (Writer writer = new OutputStreamWriter(new BufferedOutputStream(new OutputStream() { + @Override + public void write(int b) throws IOException { + stream.writeByte(b); + } + + @Override + public void write(byte b[], int off, int len) throws IOException { + stream.write(b, off, len); + } + }), StandardCharsets.UTF_8)) { statsProvider.writeAllMetrics(writer); - stream.write(writer.toString()); } catch (IOException e) { - // nop + log.error("Failed to write managed ledger bookie client metrics", e); + } + } + + public MetricsBuffer renderToBuffer(Executor executor, List metricsProviders) { + boolean cacheMetricsResponse = pulsar.getConfiguration().isMetricsBufferResponse(); + while (!closed && !Thread.currentThread().isInterrupted()) { + long currentTimeSlot = cacheMetricsResponse ? calculateCurrentTimeSlot() : 0; + MetricsBuffer currentMetricsBuffer = metricsBuffer; + if (currentMetricsBuffer == null || currentMetricsBuffer.getBufferFuture().isCompletedExceptionally() + || (currentMetricsBuffer.getBufferFuture().isDone() + && (currentMetricsBuffer.getCreateTimeslot() != 0 + && currentTimeSlot > currentMetricsBuffer.getCreateTimeslot()))) { + MetricsBuffer newMetricsBuffer = new MetricsBuffer(currentTimeSlot); + if (metricsBufferFieldUpdater.compareAndSet(this, currentMetricsBuffer, newMetricsBuffer)) { + if (currentMetricsBuffer != null) { + currentMetricsBuffer.release(); + } + CompletableFuture bufferFuture = newMetricsBuffer.getBufferFuture(); + executor.execute(() -> { + try { + bufferFuture.complete(generate0(metricsProviders)); + } catch (Exception e) { + bufferFuture.completeExceptionally(e); + } finally { + if (currentTimeSlot == 0) { + // if the buffer is not cached, release it after the future is completed + metricsBufferFieldUpdater.compareAndSet(this, newMetricsBuffer, null); + newMetricsBuffer.release(); + } + } + }); + // no need to retain before returning since the new buffer starts with refCnt 2 + return newMetricsBuffer; + } else { + currentMetricsBuffer = metricsBuffer; + } + } + // retain the buffer before returning + // if the buffer is already released, retaining won't succeed, retry in that case + if (currentMetricsBuffer != null && currentMetricsBuffer.retain()) { + return currentMetricsBuffer; + } + } + return null; + } + + /** + * Calculate the current time slot based on the current time. + * This is to ensure that cached metrics are refreshed consistently at a fixed interval regardless of the request + * time. + */ + private long calculateCurrentTimeSlot() { + long cacheTimeoutMillis = + TimeUnit.SECONDS.toMillis(Math.max(1, pulsar.getConfiguration().getManagedLedgerStatsPeriodSeconds())); + long now = clock.millis(); + return now / cacheTimeoutMillis; + } + + @Override + public void close() { + closed = true; + MetricsBuffer buffer = metricsBufferFieldUpdater.getAndSet(this, null); + if (buffer != null) { + buffer.release(); } } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java index 42bd2652883b6..7fcc74e965c24 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java @@ -18,34 +18,142 @@ */ package org.apache.pulsar.broker.stats.prometheus; +import java.io.EOFException; import java.io.IOException; +import java.nio.ByteBuffer; +import java.time.Clock; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import javax.servlet.AsyncContext; +import javax.servlet.AsyncEvent; +import javax.servlet.AsyncListener; import javax.servlet.ServletOutputStream; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.PulsarService; +import org.eclipse.jetty.server.HttpOutput; +@Slf4j public class PulsarPrometheusMetricsServlet extends PrometheusMetricsServlet { - private static final long serialVersionUID = 1L; + private static final int EXECUTOR_MAX_THREADS = 4; - private final PulsarService pulsar; - private final boolean shouldExportTopicMetrics; - private final boolean shouldExportConsumerMetrics; - private final boolean shouldExportProducerMetrics; - private final boolean splitTopicAndPartitionLabel; + private final PrometheusMetricsGenerator prometheusMetricsGenerator; public PulsarPrometheusMetricsServlet(PulsarService pulsar, boolean includeTopicMetrics, - boolean includeConsumerMetrics, boolean shouldExportProducerMetrics, + boolean includeConsumerMetrics, boolean includeProducerMetrics, boolean splitTopicAndPartitionLabel) { - super(pulsar.getConfiguration().getMetricsServletTimeoutMs(), pulsar.getConfiguration().getClusterName()); - this.pulsar = pulsar; - this.shouldExportTopicMetrics = includeTopicMetrics; - this.shouldExportConsumerMetrics = includeConsumerMetrics; - this.shouldExportProducerMetrics = shouldExportProducerMetrics; - this.splitTopicAndPartitionLabel = splitTopicAndPartitionLabel; + super(pulsar.getConfiguration().getMetricsServletTimeoutMs(), pulsar.getConfiguration().getClusterName(), + EXECUTOR_MAX_THREADS); + MetricsExports.initialize(); + prometheusMetricsGenerator = + new PrometheusMetricsGenerator(pulsar, includeTopicMetrics, includeConsumerMetrics, + includeProducerMetrics, splitTopicAndPartitionLabel, Clock.systemUTC()); } + @Override - protected void generateMetrics(String cluster, ServletOutputStream outputStream) throws IOException { - PrometheusMetricsGenerator.generate(pulsar, shouldExportTopicMetrics, shouldExportConsumerMetrics, - shouldExportProducerMetrics, splitTopicAndPartitionLabel, outputStream, metricsProviders); + public void destroy() { + super.destroy(); + prometheusMetricsGenerator.close(); + } + + protected void doGet(HttpServletRequest request, HttpServletResponse response) { + AsyncContext context = request.startAsync(); + // set hard timeout to 2 * timeout + if (metricsServletTimeoutMs > 0) { + context.setTimeout(metricsServletTimeoutMs * 2); + } + long startNanos = System.nanoTime(); + AtomicBoolean skipWritingResponse = new AtomicBoolean(false); + context.addListener(new AsyncListener() { + @Override + public void onComplete(AsyncEvent event) throws IOException { + } + + @Override + public void onTimeout(AsyncEvent event) throws IOException { + log.warn("Prometheus metrics request timed out"); + skipWritingResponse.set(true); + HttpServletResponse res = (HttpServletResponse) context.getResponse(); + if (!res.isCommitted()) { + res.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + } + context.complete(); + } + + @Override + public void onError(AsyncEvent event) throws IOException { + skipWritingResponse.set(true); + } + + @Override + public void onStartAsync(AsyncEvent event) throws IOException { + } + }); + PrometheusMetricsGenerator.MetricsBuffer metricsBuffer = + prometheusMetricsGenerator.renderToBuffer(executor, metricsProviders); + if (metricsBuffer == null) { + log.info("Service is closing, skip writing metrics."); + response.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + context.complete(); + return; + } + metricsBuffer.getBufferFuture().whenComplete((buffer, ex) -> executor.execute(() -> { + try { + long elapsedNanos = System.nanoTime() - startNanos; + // check if the request has been timed out, implement a soft timeout + // so that response writing can continue to up to 2 * timeout + if (metricsServletTimeoutMs > 0 && elapsedNanos > TimeUnit.MILLISECONDS.toNanos( + metricsServletTimeoutMs)) { + log.warn("Prometheus metrics request was too long in queue ({}ms). Skipping sending metrics.", + TimeUnit.NANOSECONDS.toMillis(elapsedNanos)); + if (!response.isCommitted() && !skipWritingResponse.get()) { + response.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + } + return; + } + if (skipWritingResponse.get()) { + log.warn("Response has timed or failed, skip writing metrics."); + return; + } + if (response.isCommitted()) { + log.warn("Response is already committed, cannot write metrics"); + return; + } + if (ex != null) { + log.error("Failed to generate metrics", ex); + response.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + return; + } + if (buffer == null) { + log.error("Failed to generate metrics, buffer is null"); + response.setStatus(HTTP_STATUS_INTERNAL_SERVER_ERROR_500); + } else { + response.setStatus(HTTP_STATUS_OK_200); + response.setContentType("text/plain;charset=utf-8"); + ServletOutputStream outputStream = response.getOutputStream(); + if (outputStream instanceof HttpOutput) { + HttpOutput output = (HttpOutput) outputStream; + for (ByteBuffer nioBuffer : buffer.nioBuffers()) { + output.write(nioBuffer); + } + } else { + int length = buffer.readableBytes(); + if (length > 0) { + buffer.duplicate().readBytes(outputStream, length); + } + } + } + } catch (EOFException e) { + log.error("Failed to write metrics to response due to EOFException"); + } catch (IOException e) { + log.error("Failed to write metrics to response", e); + } finally { + metricsBuffer.release(); + context.complete(); + } + })); } } 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 dda03e3e59dd4..33ef05df9ebc4 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 @@ -475,7 +475,9 @@ private static void writeConsumerMetric(PrometheusMetricStreams stream, String m static void writeTopicMetric(PrometheusMetricStreams stream, String metricName, Number value, String cluster, String namespace, String topic, boolean splitTopicAndPartitionIndexLabel, String... extraLabelsAndValues) { - String[] labelsAndValues = new String[splitTopicAndPartitionIndexLabel ? 8 : 6]; + int baseLabelCount = splitTopicAndPartitionIndexLabel ? 8 : 6; + String[] labelsAndValues = + new String[baseLabelCount + (extraLabelsAndValues != null ? extraLabelsAndValues.length : 0)]; labelsAndValues[0] = "cluster"; labelsAndValues[1] = cluster; labelsAndValues[2] = "namespace"; @@ -495,7 +497,11 @@ static void writeTopicMetric(PrometheusMetricStreams stream, String metricName, } else { labelsAndValues[5] = topic; } - String[] labels = ArrayUtils.addAll(labelsAndValues, extraLabelsAndValues); - stream.writeSample(metricName, value, labels); + if (extraLabelsAndValues != null) { + for (int i = 0; i < extraLabelsAndValues.length; i++) { + labelsAndValues[baseLabelCount + i] = extraLabelsAndValues[i]; + } + } + stream.writeSample(metricName, value, labelsAndValues); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java new file mode 100644 index 0000000000000..fcc3b6aa88fb4 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java @@ -0,0 +1,84 @@ +/* + * 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; + +import com.google.common.util.concurrent.MoreExecutors; +import io.netty.buffer.ByteBuf; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.time.Clock; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; +import org.apache.pulsar.broker.stats.prometheus.PrometheusRawMetricsProvider; +import org.eclipse.jetty.server.HttpOutput; + +public class PrometheusMetricsTestUtil { + public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, + boolean includeProducerMetrics, OutputStream out) throws IOException { + generate(new PrometheusMetricsGenerator(pulsar, includeTopicMetrics, includeConsumerMetrics, + includeProducerMetrics, false, Clock.systemUTC()), out, null); + } + + public static void generate(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, + boolean includeProducerMetrics, boolean splitTopicAndPartitionIndexLabel, + OutputStream out) throws IOException { + generate(new PrometheusMetricsGenerator(pulsar, includeTopicMetrics, includeConsumerMetrics, + includeProducerMetrics, splitTopicAndPartitionIndexLabel, Clock.systemUTC()), out, null); + } + + public static void generate(PrometheusMetricsGenerator metricsGenerator, OutputStream out, + List metricsProviders) throws IOException { + PrometheusMetricsGenerator.MetricsBuffer metricsBuffer = + metricsGenerator.renderToBuffer(MoreExecutors.directExecutor(), metricsProviders); + try { + ByteBuf buffer = null; + try { + buffer = metricsBuffer.getBufferFuture().get(5, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException(e); + } catch (ExecutionException | TimeoutException e) { + throw new IOException(e); + } + if (buffer == null) { + return; + } + if (out instanceof HttpOutput) { + HttpOutput output = (HttpOutput) out; + ByteBuffer[] nioBuffers = buffer.nioBuffers(); + for (ByteBuffer nioBuffer : nioBuffers) { + output.write(nioBuffer); + } + } else { + int length = buffer.readableBytes(); + if (length > 0) { + buffer.duplicate().readBytes(out, length); + } + } + } finally { + metricsBuffer.release(); + } + } +} 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 54fec3934ddbc..981feb002871e 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 @@ -38,11 +38,11 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.commons.lang3.mutable.MutableInt; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.stats.PrometheusMetricsTest; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; @@ -217,7 +217,7 @@ public void testBucketDelayedIndexMetrics() throws Exception { Thread.sleep(2000); ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, true, true, output); + PrometheusMetricsTestUtil.generate(pulsar, true, true, true, output); String metricsStr = output.toString(StandardCharsets.UTF_8); Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); @@ -303,7 +303,7 @@ public void testBucketDelayedIndexMetrics() throws Exception { assertEquals(opLatencyMetricsSum.intValue(), opLatencyTopicMetrics.get().value); ByteArrayOutputStream namespaceOutput = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, true, true, namespaceOutput); + PrometheusMetricsTestUtil.generate(pulsar, false, true, true, namespaceOutput); Multimap namespaceMetricsMap = PrometheusMetricsTest.parseMetrics(namespaceOutput.toString(StandardCharsets.UTF_8)); Optional namespaceMetric = 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 4c0d8eb6a49ac..5c49b4723033b 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 @@ -63,12 +63,12 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.pulsar.PrometheusMetricsTestUtil; 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.TopicPoliciesService; import org.apache.pulsar.broker.stats.PrometheusMetricsTest; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -366,7 +366,7 @@ public void testDelayedDeliveryTrackerMemoryUsageMetric(String topic, boolean ex latch.await(10, TimeUnit.SECONDS); ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, exposeTopicLevelMetrics, true, true, output); + PrometheusMetricsTestUtil.generate(pulsar, exposeTopicLevelMetrics, true, true, output); String metricsStr = output.toString(StandardCharsets.UTF_8); Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); 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 a7d33737bdc3c..3623a47d9ef67 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 @@ -41,12 +41,12 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.schema.SchemaRegistry.SchemaAndMetadata; import org.apache.pulsar.broker.stats.PrometheusMetricsTest; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo; import org.apache.pulsar.common.naming.TopicName; @@ -120,7 +120,7 @@ public void testSchemaRegistryMetrics() throws Exception { deleteSchema(schemaId, version(1)); ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, output); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, output); output.flush(); String metricsStr = output.toString(StandardCharsets.UTF_8); Multimap metrics = PrometheusMetricsTest.parseMetrics(metricsStr); 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 f29c643a8f50b..de65d5db56475 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 @@ -43,6 +43,7 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; @@ -50,7 +51,6 @@ import org.apache.pulsar.broker.service.plugin.EntryFilter; import org.apache.pulsar.broker.service.plugin.EntryFilterProducerTest; import org.apache.pulsar.broker.service.plugin.EntryFilterWithClassLoader; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -333,7 +333,7 @@ private void testMessageAckRateMetric(String topicName, boolean exposeTopicLevel consumer2.updateRates(); ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, exposeTopicLevelMetrics, true, true, output); + PrometheusMetricsTestUtil.generate(pulsar, exposeTopicLevelMetrics, true, true, output); String metricStr = output.toString(StandardCharsets.UTF_8); Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java index 8ae0242c6232a..7368d42355c1b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java @@ -28,10 +28,10 @@ import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; import org.apache.pulsar.broker.service.BrokerTestBase; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Producer; @@ -99,7 +99,7 @@ public void testMetadataStoreStats() throws Exception { } ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, false, output); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, false, output); String metricsStr = output.toString(); Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); @@ -189,7 +189,7 @@ public void testBatchMetadataStoreMetrics() throws Exception { } ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, false, output); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, false, output); String metricsStr = output.toString(); Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); 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 455da11f4fdf0..244d9eaa81274 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 @@ -19,7 +19,10 @@ package org.apache.pulsar.broker.stats; import static com.google.common.base.Preconditions.checkArgument; +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 static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; @@ -35,6 +38,7 @@ import java.math.RoundingMode; import java.nio.charset.StandardCharsets; import java.text.NumberFormat; +import java.time.Clock; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -52,6 +56,7 @@ import java.util.TreeMap; import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -62,6 +67,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; @@ -84,7 +90,6 @@ import org.apache.pulsar.compaction.PulsarCompactionServiceFactory; import org.apache.zookeeper.CreateMode; import org.awaitility.Awaitility; -import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -167,7 +172,7 @@ private void checkPublishRateLimitedTimes(boolean preciseRateLimit) throws Excep }); @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_publish_rate_limit_times")); @@ -203,7 +208,7 @@ private void checkPublishRateLimitedTimes(boolean preciseRateLimit) throws Excep @Cleanup ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut2); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut2); String metricsStr2 = statsOut2.toString(); Multimap metrics2 = parseMetrics(metricsStr2); assertTrue(metrics2.containsKey("pulsar_publish_rate_limit_times")); @@ -235,7 +240,7 @@ public void testMetricsTopicCount() throws Exception { Thread.sleep(ASYNC_EVENT_COMPLETION_WAIT); @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); Collection metric = metrics.get("pulsar_topics_count"); @@ -272,7 +277,7 @@ public void testMetricsAvgMsgSize2() throws Exception { producerInServer.getStats().msgThroughputIn = 100; @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, true, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, true, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_average_msg_size")); @@ -315,7 +320,7 @@ public void testPerTopicStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -413,7 +418,7 @@ public void testPerBrokerStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -522,7 +527,7 @@ public void testPerTopicStatsReconnect() throws Exception { c2.close(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -600,7 +605,7 @@ public void testStorageReadCacheMissesRate(boolean cacheEnable) throws Exception // includeTopicMetric true ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -632,7 +637,7 @@ public void testStorageReadCacheMissesRate(boolean cacheEnable) throws Exception // includeTopicMetric false ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut2); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut2); String metricsStr2 = statsOut2.toString(); Multimap metrics2 = parseMetrics(metricsStr2); @@ -716,7 +721,7 @@ public void testPerTopicExpiredStat() throws Exception { Awaitility.await().until(() -> sub2.getExpiredMessageRate() != 0.0); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); // There should be 2 metrics with different tags for each topic @@ -797,7 +802,7 @@ public void testBundlesMetrics() throws Exception { loadManager.getLoadManager().updateLocalBrokerData(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_bundle_msg_rate_in")); @@ -842,7 +847,7 @@ public void testNonPersistentSubMetrics() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); assertTrue(metrics.containsKey("pulsar_subscription_back_log")); @@ -889,7 +894,7 @@ public void testPerNamespaceStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -962,7 +967,7 @@ public void testPerProducerStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, true, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, true, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1030,7 +1035,7 @@ public void testPerConsumerStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, true, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, true, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1117,7 +1122,7 @@ public void testDuplicateMetricTypeDefinitions() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Map typeDefs = new HashMap<>(); @@ -1221,7 +1226,7 @@ public void testManagedLedgerCacheStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1257,7 +1262,7 @@ public void testManagedLedgerStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1335,7 +1340,7 @@ public void testManagedLedgerBookieClientStats() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1416,7 +1421,7 @@ public String getCommandData() { }); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); List cm = (List) metrics.get("pulsar_authentication_success_total"); @@ -1477,7 +1482,7 @@ public String getCommandData() { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); List cm = (List) metrics.get("pulsar_expired_token_total"); @@ -1518,7 +1523,7 @@ public String getCommandData() { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); Metric countMetric = ((List) metrics.get("pulsar_expiring_token_minutes_count")).get(0); @@ -1592,7 +1597,7 @@ public void testManagedCursorPersistStats() throws Exception { // enable ExposeManagedCursorMetricsInPrometheus pulsar.getConfiguration().setExposeManagedCursorMetricsInPrometheus(true); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -1605,7 +1610,7 @@ public void testManagedCursorPersistStats() throws Exception { // disable ExposeManagedCursorMetricsInPrometheus pulsar.getConfiguration().setExposeManagedCursorMetricsInPrometheus(false); ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut2); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut2); String metricsStr2 = statsOut2.toString(); Multimap metrics2 = parseMetrics(metricsStr2); List cm2 = (List) metrics2.get("pulsar_ml_cursor_persistLedgerSucceed"); @@ -1624,7 +1629,7 @@ public void testBrokerConnection() throws Exception { .create(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); List cm = (List) metrics.get("pulsar_connection_created_total_count"); @@ -1641,7 +1646,7 @@ public void testBrokerConnection() throws Exception { pulsarClient.close(); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); @@ -1664,7 +1669,7 @@ public void testBrokerConnection() throws Exception { pulsarClient.close(); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); @@ -1708,7 +1713,7 @@ public void testCompaction() throws Exception { .messageRoutingMode(MessageRoutingMode.SinglePartition) .create(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); List cm = (List) metrics.get("pulsar_compaction_removed_event_count"); @@ -1743,7 +1748,7 @@ public void testCompaction() throws Exception { Compactor compactor = ((PulsarCompactionServiceFactory)pulsar.getCompactionServiceFactory()).getCompactor(); compactor.compact(topicName).get(); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); cm = (List) metrics.get("pulsar_compaction_removed_event_count"); @@ -1776,31 +1781,36 @@ public void testCompaction() throws Exception { @Test public void testMetricsWithCache() throws Throwable { - ServiceConfiguration configuration = Mockito.mock(ServiceConfiguration.class); - Mockito.when(configuration.getManagedLedgerStatsPeriodSeconds()).thenReturn(2); - Mockito.when(configuration.isMetricsBufferResponse()).thenReturn(true); - Mockito.when(configuration.getClusterName()).thenReturn(configClusterName); - Mockito.when(pulsar.getConfiguration()).thenReturn(configuration); + ServiceConfiguration configuration = pulsar.getConfiguration(); + configuration.setManagedLedgerStatsPeriodSeconds(2); + configuration.setMetricsBufferResponse(true); + configuration.setClusterName(configClusterName); - int period = pulsar.getConfiguration().getManagedLedgerStatsPeriodSeconds(); - TimeWindow timeWindow = new TimeWindow<>(2, (int) TimeUnit.SECONDS.toMillis(period)); + // create a mock clock to control the time + AtomicLong currentTimeMillis = new AtomicLong(System.currentTimeMillis()); + Clock clock = mock(Clock.class); + when(clock.millis()).thenAnswer(invocation -> currentTimeMillis.get()); + PrometheusMetricsGenerator prometheusMetricsGenerator = + new PrometheusMetricsGenerator(pulsar, true, false, false, + false, clock); + + String previousMetrics = null; for (int a = 0; a < 4; a++) { - long start = System.currentTimeMillis(); ByteArrayOutputStream statsOut1 = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, false, statsOut1, null); + PrometheusMetricsTestUtil.generate(prometheusMetricsGenerator, statsOut1, null); ByteArrayOutputStream statsOut2 = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, false, statsOut2, null); - long end = System.currentTimeMillis(); - - if (timeWindow.currentWindowStart(start) == timeWindow.currentWindowStart(end)) { - String metricsStr1 = statsOut1.toString(); - String metricsStr2 = statsOut2.toString(); - assertEquals(metricsStr1, metricsStr2); - Multimap metrics = parseMetrics(metricsStr1); - } + PrometheusMetricsTestUtil.generate(prometheusMetricsGenerator, statsOut2, null); + + String metricsStr1 = statsOut1.toString(); + String metricsStr2 = statsOut2.toString(); + assertTrue(metricsStr1.length() > 1000); + assertEquals(metricsStr1, metricsStr2); + assertNotEquals(metricsStr1, previousMetrics); + previousMetrics = metricsStr1; - Thread.sleep(TimeUnit.SECONDS.toMillis(period / 2)); + // move time forward + currentTimeMillis.addAndGet(TimeUnit.SECONDS.toMillis(2)); } } @@ -1828,7 +1838,7 @@ public void testSplitTopicAndPartitionLabel() throws Exception { .subscribe(); @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, true, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, true, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); Collection metric = metrics.get("pulsar_consumers_count"); @@ -1864,7 +1874,7 @@ public void testMetricsGroupedByTypeDefinitions() throws Exception { } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Pattern typePattern = Pattern.compile("^#\\s+TYPE\\s+(\\w+)\\s+(\\w+)"); @@ -1984,7 +1994,7 @@ public void testEscapeLabelValue() throws Exception { .subscribe(); @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); final List subCountLines = metricsStr.lines() 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 d5e0066a86f15..83e6f43cbafcc 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 @@ -29,13 +29,13 @@ import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.Dispatcher; import org.apache.pulsar.broker.service.EntryFilterSupport; 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.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -231,7 +231,7 @@ public void testSubscriptionStats(final String topic, final String subName, bool } ByteArrayOutputStream output = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, enableTopicStats, false, false, output); + PrometheusMetricsTestUtil.generate(pulsar, enableTopicStats, false, false, output); String metricsStr = output.toString(); Multimap metrics = PrometheusMetricsTest.parseMetrics(metricsStr); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TimeWindowTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TimeWindowTest.java deleted file mode 100644 index 89528c1965397..0000000000000 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TimeWindowTest.java +++ /dev/null @@ -1,83 +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.broker.stats; - -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNotNull; -import org.testng.annotations.Test; - -public class TimeWindowTest { - - @Test - public void windowTest() throws Exception { - int intervalInMs = 1000; - int sampleCount = 2; - TimeWindow timeWindow = new TimeWindow<>(sampleCount, intervalInMs); - - WindowWrap expect1 = timeWindow.current(oldValue -> 1); - WindowWrap expect2 = timeWindow.current(oldValue -> null); - assertNotNull(expect1); - assertNotNull(expect2); - - if (expect1.start() == expect2.start()) { - assertEquals((int) expect1.value(), 1); - assertEquals(expect1, expect2); - assertEquals(expect1.value(), expect2.value()); - } - - Thread.sleep(intervalInMs); - - WindowWrap expect3 = timeWindow.current(oldValue -> 2); - WindowWrap expect4 = timeWindow.current(oldValue -> null); - assertNotNull(expect3); - assertNotNull(expect4); - - if (expect3.start() == expect4.start()) { - assertEquals((int) expect3.value(), 2); - assertEquals(expect3, expect4); - assertEquals(expect3.value(), expect4.value()); - } - - Thread.sleep(intervalInMs); - - WindowWrap expect5 = timeWindow.current(oldValue -> 3); - WindowWrap expect6 = timeWindow.current(oldValue -> null); - assertNotNull(expect5); - assertNotNull(expect6); - - if (expect5.start() == expect6.start()) { - assertEquals((int) expect5.value(), 3); - assertEquals(expect5, expect6); - assertEquals(expect5.value(), expect6.value()); - } - - Thread.sleep(intervalInMs); - - WindowWrap expect7 = timeWindow.current(oldValue -> 4); - WindowWrap expect8 = timeWindow.current(oldValue -> null); - assertNotNull(expect7); - assertNotNull(expect8); - - if (expect7.start() == expect8.start()) { - assertEquals((int) expect7.value(), 4); - assertEquals(expect7, expect8); - assertEquals(expect7.value(), expect8.value()); - } - } -} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java index 4d38f5fad5141..85c6dd795d759 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java @@ -37,9 +37,9 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerTestBase; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; @@ -117,7 +117,7 @@ public void testTransactionCoordinatorMetrics() throws Exception { pulsar.getTransactionMetadataStoreService().getStores() .get(transactionCoordinatorIDTwo).newTransaction(timeout, null).get(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); Collection metric = metrics.get("pulsar_txn_active_count"); @@ -185,7 +185,7 @@ public void testTransactionCoordinatorRateMetrics() throws Exception { pulsar.getBrokerService().updateRates(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -215,7 +215,7 @@ public void testTransactionCoordinatorRateMetrics() throws Exception { }); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); @@ -271,7 +271,7 @@ public void testManagedLedgerMetrics() throws Exception { producer.send("hello pulsar".getBytes()); consumer.acknowledgeAsync(consumer.receive().getMessageId(), transaction).get(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -289,7 +289,7 @@ public void testManagedLedgerMetrics() throws Exception { checkManagedLedgerMetrics(MLTransactionLogImpl.TRANSACTION_SUBSCRIPTION_NAME, 126, metric); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); metric = metrics.get("pulsar_storage_size"); @@ -333,7 +333,7 @@ public void testManagedLedgerMetricsWhenPendingAckNotInit() throws Exception { producer.send("hello pulsar".getBytes()); consumer.acknowledgeAsync(consumer.receive().getMessageId(), transaction).get(); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = parseMetrics(metricsStr); @@ -358,7 +358,7 @@ public void testManagedLedgerMetricsWhenPendingAckNotInit() throws Exception { consumer.acknowledgeAsync(consumer.receive().getMessageId(), transaction).get(); statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); metricsStr = statsOut.toString(); metrics = parseMetrics(metricsStr); metric = metrics.get("pulsar_storage_size"); @@ -392,7 +392,7 @@ public void testDuplicateMetricTypeDefinitions() throws Exception { .send(); } ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Map typeDefs = new HashMap<>(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java index 1dc086dbe3470..0211b860cc4fe 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java @@ -36,10 +36,10 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.stats.PrometheusMetricsTest; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.broker.transaction.TransactionTestBase; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferClientImpl; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferHandlerImpl; @@ -226,7 +226,7 @@ public void testTransactionBufferMetrics() throws Exception { @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsarServiceList.get(0), true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsarServiceList.get(0), true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); 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 2e154715ac9d8..f5ac68950b49d 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 @@ -47,6 +47,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.map.LinkedMap; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.AbstractTopic; import org.apache.pulsar.broker.service.BrokerService; @@ -54,7 +55,6 @@ import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.stats.PrometheusMetricsTest; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.broker.transaction.TransactionTestBase; import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore; import org.apache.pulsar.broker.transaction.pendingack.impl.PendingAckHandleImpl; @@ -336,7 +336,7 @@ public void testPendingAckMetrics() throws Exception { @Cleanup ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsarServiceList.get(0), true, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsarServiceList.get(0), true, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); 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 00c6a209c13d7..999f22dc7f92b 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 @@ -53,10 +53,10 @@ import javax.net.ssl.TrustManager; import lombok.Cleanup; import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.stats.PrometheusMetricsTest; -import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsGenerator; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminBuilder; @@ -106,7 +106,7 @@ public class WebServiceTest { public void testWebExecutorMetrics() throws Exception { setupEnv(true, false, false, false, -1, false); ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); - PrometheusMetricsGenerator.generate(pulsar, false, false, false, statsOut); + PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); Multimap metrics = PrometheusMetricsTest.parseMetrics(metricsStr); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java index c8c639606aa3e..9bf6302f50f02 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java @@ -20,6 +20,7 @@ import io.netty.buffer.ByteBuf; import io.netty.util.CharsetUtil; +import java.nio.CharBuffer; /** * Format strings and numbers into a ByteBuf without any memory allocation. @@ -28,6 +29,7 @@ public class SimpleTextOutputStream { private final ByteBuf buffer; private static final char[] hexChars = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f'}; + private final CharBuffer singleCharBuffer = CharBuffer.allocate(1); public SimpleTextOutputStream(ByteBuf buffer) { this.buffer = buffer; @@ -44,11 +46,17 @@ public SimpleTextOutputStream write(byte[] a, int offset, int len) { } public SimpleTextOutputStream write(char c) { - write(String.valueOf(c)); + // In UTF-8, any character from U+0000 to U+007F is encoded in one byte + if (c <= '\u007F') { + buffer.writeByte((byte) c); + return this; + } + singleCharBuffer.put(0, c); + buffer.writeCharSequence(singleCharBuffer, CharsetUtil.UTF_8); return this; } - public SimpleTextOutputStream write(String s) { + public SimpleTextOutputStream write(CharSequence s) { if (s == null) { return this; } @@ -136,4 +144,8 @@ public void write(ByteBuf byteBuf) { public ByteBuf getBuffer() { return buffer; } + + public void writeByte(int b) { + buffer.writeByte(b); + } } 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 4ec5b3f77a3e1..1ff141ec13a08 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 @@ -398,6 +398,12 @@ public class ProxyConfiguration implements PulsarConfiguration { ) private boolean authenticateMetricsEndpoint = true; + @FieldContext( + category = CATEGORY_HTTP, + doc = "Time in milliseconds that metrics endpoint would time out. Default is 30s.\n" + + " Set it to 0 to disable timeout." + ) + private long metricsServletTimeoutMs = 30000; @FieldContext( category = CATEGORY_SASL_AUTH, 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 b360820ad007c..a3917f2bc6708 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 @@ -292,7 +292,8 @@ public void start() throws Exception { } private synchronized void createMetricsServlet() { - this.metricsServlet = new PrometheusMetricsServlet(-1L, proxyConfig.getClusterName()); + this.metricsServlet = + new PrometheusMetricsServlet(proxyConfig.getMetricsServletTimeoutMs(), proxyConfig.getClusterName()); if (pendingMetricsProviders != null) { pendingMetricsProviders.forEach(provider -> metricsServlet.addRawMetricsProvider(provider)); this.pendingMetricsProviders = null; diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java index e623d4b85aa09..aa612f12b1ba9 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java @@ -26,6 +26,7 @@ import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; import com.google.common.annotations.VisibleForTesting; +import io.prometheus.client.Collector; import io.prometheus.client.CollectorRegistry; import io.prometheus.client.Gauge; import io.prometheus.client.Gauge.Child; @@ -231,21 +232,36 @@ public void start() throws Exception { if (!metricsInitialized) { // Setup metrics DefaultExports.initialize(); + CollectorRegistry registry = CollectorRegistry.defaultRegistry; // Report direct memory from Netty counters - Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() { - @Override - public double get() { - return getJvmDirectMemoryUsed(); - } - }).register(CollectorRegistry.defaultRegistry); + Collector jvmMemoryDirectBytesUsed = + Gauge.build("jvm_memory_direct_bytes_used", "-").create().setChild(new Child() { + @Override + public double get() { + return getJvmDirectMemoryUsed(); + } + }); + try { + registry.register(jvmMemoryDirectBytesUsed); + } catch (IllegalArgumentException e) { + // workaround issue in tests where the metric is already registered + log.debug("Failed to register jvm_memory_direct_bytes_used metric: {}", e.getMessage()); + } - Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() { - @Override - public double get() { - return DirectMemoryUtils.jvmMaxDirectMemory(); - } - }).register(CollectorRegistry.defaultRegistry); + Collector jvmMemoryDirectBytesMax = + Gauge.build("jvm_memory_direct_bytes_max", "-").create().setChild(new Child() { + @Override + public double get() { + return DirectMemoryUtils.jvmMaxDirectMemory(); + } + }); + try { + registry.register(jvmMemoryDirectBytesMax); + } catch (IllegalArgumentException e) { + // workaround issue in tests where the metric is already registered + log.debug("Failed to register jvm_memory_direct_bytes_max metric: {}", e.getMessage()); + } metricsInitialized = true; } From 7d1d0bb61a58317135727d24c1c8a6c8a08f5a17 Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Fri, 22 Mar 2024 11:52:47 +0800 Subject: [PATCH 425/699] [fix][broker] Create new ledger after the current ledger is closed (#22034) (cherry picked from commit d0ca9835cf972ce156bd4a1fc5d109482330857d) (cherry picked from commit 54042df2ca5594325b9c038fd853c264559fbbac) --- .../mledger/impl/ManagedCursorImpl.java | 2 +- .../mledger/impl/ManagedLedgerImpl.java | 22 ++-- .../mledger/impl/ManagedCursorTest.java | 33 ++++-- .../impl/ManagedLedgerFactoryTest.java | 2 +- .../mledger/impl/ManagedLedgerTest.java | 111 +++++++++++++---- .../mledger/impl/NonDurableCursorTest.java | 17 +-- .../impl/ShadowManagedLedgerImplTest.java | 5 +- .../service/BacklogQuotaManagerTest.java | 13 +- .../service/BrokerBkEnsemblesTests.java | 12 +- .../service/BrokerBookieIsolationTest.java | 112 ++++++++++++------ .../service/ConsumedLedgersTrimTest.java | 6 +- .../impl/ProducerConsumerInternalTest.java | 44 +++++++ 12 files changed, 275 insertions(+), 104 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 972abcfea983d..ccea125ef83f6 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 @@ -1278,7 +1278,7 @@ protected void internalResetCursor(PositionImpl proposedReadPosition, if (proposedReadPosition.equals(PositionImpl.EARLIEST)) { newReadPosition = ledger.getFirstPosition(); } else if (proposedReadPosition.equals(PositionImpl.LATEST)) { - newReadPosition = ledger.getLastPosition().getNext(); + newReadPosition = ledger.getNextValidPosition(ledger.getLastPosition()); } else { newReadPosition = proposedReadPosition; } 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 dce4860e3e6f0..32ac345629d32 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 @@ -1757,10 +1757,7 @@ synchronized void ledgerClosed(final LedgerHandle lh) { maybeOffloadInBackground(NULL_OFFLOAD_PROMISE); - if (!pendingAddEntries.isEmpty()) { - // Need to create a new ledger to write pending entries - createLedgerAfterClosed(); - } + createLedgerAfterClosed(); } @Override @@ -1815,7 +1812,6 @@ public void closeComplete(int rc, LedgerHandle lh, Object o) { } ledgerClosed(lh); - createLedgerAfterClosed(); } }, null); } @@ -2696,7 +2692,16 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture promise) { } else { PositionImpl slowestReaderPosition = cursors.getSlowestReaderPosition(); if (slowestReaderPosition != null) { - slowestReaderLedgerId = slowestReaderPosition.getLedgerId(); + // The slowest reader position is the mark delete position. + // If the slowest reader position point the last entry in the ledger x, + // the slowestReaderLedgerId should be x + 1 and the ledger x could be deleted. + LedgerInfo ledgerInfo = ledgers.get(slowestReaderPosition.getLedgerId()); + if (ledgerInfo != null && ledgerInfo.getLedgerId() != currentLedger.getId() + && ledgerInfo.getEntries() == slowestReaderPosition.getEntryId() + 1) { + slowestReaderLedgerId = slowestReaderPosition.getLedgerId() + 1; + } else { + slowestReaderLedgerId = slowestReaderPosition.getLedgerId(); + } } else { promise.completeExceptionally(new ManagedLedgerException("Couldn't find reader position")); trimmerMutex.unlock(); @@ -3740,7 +3745,11 @@ public PositionImpl getValidPositionAfterSkippedEntries(final PositionImpl posit PositionImpl skippedPosition = position.getPositionAfterEntries(skippedEntryNum); while (!isValidPosition(skippedPosition)) { Long nextLedgerId = ledgers.ceilingKey(skippedPosition.getLedgerId() + 1); + // This means it has jumped to the last position if (nextLedgerId == null) { + if (currentLedgerEntries == 0) { + return PositionImpl.get(currentLedger.getId(), 0); + } return lastConfirmedEntry.getNext(); } skippedPosition = PositionImpl.get(nextLedgerId, 0); @@ -4532,7 +4541,6 @@ public boolean checkInactiveLedgerAndRollOver() { } ledgerClosed(lh); - createLedgerAfterClosed(); // we do not create ledger here, since topic is inactive for a long time. }, null); return true; 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 644f53c3a522d..c9bd64171c15a 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 @@ -234,15 +234,16 @@ void readTwice() throws Exception { @Test void testPersistentMarkDeleteIfCreateCursorLedgerFailed() throws Exception { - final int entryCount = 10; + final int entryCount = 9; final String cursorName = "c1"; final String mlName = "ml_test"; - final ManagedLedgerConfig mlConfig = new ManagedLedgerConfig().setMaxEntriesPerLedger(1); + // Avoid creating new empty ledger after the last ledger is full and remove fail future. + final ManagedLedgerConfig mlConfig = new ManagedLedgerConfig().setMaxEntriesPerLedger(2); ManagedLedgerImpl ml = (ManagedLedgerImpl) factory.open(mlName, mlConfig); ManagedCursor cursor = ml.openCursor("c1"); Position lastEntry = null; - for (int i = 0; i < 10; i++) { + for (int i = 0; i < entryCount; i++) { lastEntry = ml.addEntry(("entry-" + i).getBytes(Encoding)); } @@ -809,7 +810,7 @@ void testResetCursor1() throws Exception { assertEquals(firstInNext, cursor.getReadPosition()); moveStatus.set(false); - // reset to a non exist larger ledger should point to the first non-exist entry in the last ledger + // reset to a non exist larger ledger should point to the first non-exist entry in the next ledger PositionImpl latest = new PositionImpl(last.getLedgerId() + 2, 0); try { cursor.resetCursor(latest); @@ -818,11 +819,13 @@ void testResetCursor1() throws Exception { log.warn("error in reset cursor", e.getCause()); } assertTrue(moveStatus.get()); - PositionImpl lastPos = new PositionImpl(last.getLedgerId(), last.getEntryId() + 1); - assertEquals(lastPos, cursor.getReadPosition()); + PositionImpl lastPos = new PositionImpl(last.getLedgerId() + 1, 0); + Awaitility.await().untilAsserted(() -> { + assertEquals(lastPos, cursor.getReadPosition()); + }); moveStatus.set(false); - // reset to latest should point to the first non-exist entry in the last ledger + // reset to latest should point to the first non-exist entry in the next ledger PositionImpl anotherLast = PositionImpl.LATEST; try { cursor.resetCursor(anotherLast); @@ -1701,7 +1704,7 @@ void testMarkDeleteTwice(boolean useOpenRangeSet) throws Exception { @Test(timeOut = 20000, dataProvider = "useOpenRangeSet") void testSkipEntries(boolean useOpenRangeSet) throws Exception { - ManagedLedger ledger = factory.open("my_test_ledger", new ManagedLedgerConfig() + ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("my_test_ledger", new ManagedLedgerConfig() .setUnackedRangesOpenCacheSetEnabled(useOpenRangeSet).setMaxEntriesPerLedger(2)); Position pos; @@ -1715,6 +1718,11 @@ void testSkipEntries(boolean useOpenRangeSet) throws Exception { pos = ledger.addEntry("dummy-entry-1".getBytes(Encoding)); pos = ledger.addEntry("dummy-entry-2".getBytes(Encoding)); + // Wait new empty ledger created completely. + Awaitility.await().untilAsserted(() -> { + assertEquals(ledger.ledgers.size(), 2); + }); + // skip entries in same ledger c1.skipEntries(1, IndividualDeletedEntries.Exclude); assertEquals(c1.getNumberOfEntries(), 1); @@ -1722,7 +1730,7 @@ void testSkipEntries(boolean useOpenRangeSet) throws Exception { // skip entries until end of ledger c1.skipEntries(1, IndividualDeletedEntries.Exclude); assertEquals(c1.getNumberOfEntries(), 0); - assertEquals(c1.getReadPosition(), pos.getNext()); + assertEquals(c1.getReadPosition(), new PositionImpl(ledger.currentLedger.getId(), 0)); assertEquals(c1.getMarkDeletedPosition(), pos); // skip entries across ledgers @@ -1737,7 +1745,10 @@ void testSkipEntries(boolean useOpenRangeSet) throws Exception { c1.skipEntries(10, IndividualDeletedEntries.Exclude); assertEquals(c1.getNumberOfEntries(), 0); assertFalse(c1.hasMoreEntries()); - assertEquals(c1.getReadPosition(), pos.getNext()); + // We can not check the ledger id because a cursor leger can be created. + Awaitility.await().untilAsserted(() -> { + assertEquals(c1.getReadPosition().getEntryId(), 0); + }); assertEquals(c1.getMarkDeletedPosition(), pos); } @@ -1759,7 +1770,7 @@ void testSkipEntriesWithIndividualDeletedMessages(boolean useOpenRangeSet) throw c1.skipEntries(3, IndividualDeletedEntries.Exclude); assertEquals(c1.getNumberOfEntries(), 0); - assertEquals(c1.getReadPosition(), pos5.getNext()); + assertEquals(c1.getReadPosition(), new PositionImpl(pos5.getLedgerId() + 1, 0)); assertEquals(c1.getMarkDeletedPosition(), pos5); pos1 = ledger.addEntry("dummy-entry-1".getBytes(Encoding)); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryTest.java index 4f2c3e178773e..a953b140aba63 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerFactoryTest.java @@ -54,7 +54,7 @@ public void testGetManagedLedgerInfoWithClose() throws Exception { ManagedLedgerInfo info = factory.getManagedLedgerInfo("testGetManagedLedgerInfo"); - assertEquals(info.ledgers.size(), 4); + assertEquals(info.ledgers.size(), 5); assertEquals(info.ledgers.get(0).ledgerId, 3); assertEquals(info.ledgers.get(1).ledgerId, 4); 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 f918ffdc755d6..c28a32d9ec2e1 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 @@ -36,6 +36,7 @@ import static org.mockito.Mockito.when; 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.assertSame; @@ -1120,9 +1121,13 @@ public void testTrimmer() throws Exception { cursor.markDelete(lastPosition); - while (ledger.getNumberOfEntries() != 2) { - Thread.sleep(10); - } + Awaitility.await().untilAsserted(() -> { + // The number of entries in the ledger should not contain the entry in the mark delete position. + // last position is the position of entry-3. + // cursor.markDelete(lastPosition); + // only entry-4 is left in the ledger. + assertEquals(ledger.getNumberOfEntries(), 1); + }); } @Test(timeOut = 20000) @@ -2436,7 +2441,7 @@ public void testRetentionSize() throws Exception { Awaitility.await().untilAsserted(() -> { assertTrue(ml.getTotalSize() <= retentionSizeInMB * 1024 * 1024); - assertEquals(ml.getLedgersInfoAsList().size(), 5); + assertEquals(ml.getLedgersInfoAsList().size(), 6); }); } @@ -2694,9 +2699,17 @@ public void testGetNextValidPosition() throws Exception { assertEquals(ledger.getNextValidPosition((PositionImpl) c1.getMarkDeletedPosition()), p1); assertEquals(ledger.getNextValidPosition(p1), p2); - assertEquals(ledger.getNextValidPosition(p3), PositionImpl.get(p3.getLedgerId(), p3.getEntryId() + 1)); - assertEquals(ledger.getNextValidPosition(PositionImpl.get(p3.getLedgerId(), p3.getEntryId() + 1)), PositionImpl.get(p3.getLedgerId(), p3.getEntryId() + 1)); - assertEquals(ledger.getNextValidPosition(PositionImpl.get(p3.getLedgerId() + 1, p3.getEntryId() + 1)), PositionImpl.get(p3.getLedgerId(), p3.getEntryId() + 1)); + Awaitility.await().untilAsserted(() -> { + assertEquals(ledger.getNextValidPosition(p3), PositionImpl.get(p3.getLedgerId() + 1, 0)); + }); + Awaitility.await().untilAsserted(() -> { + assertEquals(ledger.getNextValidPosition(PositionImpl.get(p3.getLedgerId(), p3.getEntryId() + 1)), + PositionImpl.get(p3.getLedgerId() + 1, 0)); + }); + Awaitility.await().untilAsserted(() -> { + assertEquals(ledger.getNextValidPosition(PositionImpl.get(p3.getLedgerId() + 1, p3.getEntryId() + 1)), + PositionImpl.get(p3.getLedgerId() + 1, 0)); + }); } /** @@ -3035,19 +3048,22 @@ public void testConsumerSubscriptionInitializePosition() throws Exception{ String content = "entry" + i; // 5 bytes ledger.addEntry(content.getBytes()); } + Awaitility.await().untilAsserted(() -> { + assertEquals(ledger.currentLedgerSize, 0); + assertEquals(ledger.ledgers.size(), 1); + }); // Open Cursor also adds cursor into activeCursor-container ManagedCursor latestCursor = ledger.openCursor("c1", InitialPosition.Latest); ManagedCursor earliestCursor = ledger.openCursor("c2", InitialPosition.Earliest); // Since getReadPosition returns the next position, we decrease the entryId by 1 - PositionImpl p1 = (PositionImpl) latestCursor.getReadPosition(); PositionImpl p2 = (PositionImpl) earliestCursor.getReadPosition(); Pair latestPositionAndCounter = ledger.getLastPositionAndCounter(); Pair earliestPositionAndCounter = ledger.getFirstPositionAndCounter(); - - assertEquals(latestPositionAndCounter.getLeft().getNext(), p1); - assertEquals(earliestPositionAndCounter.getLeft().getNext(), p2); + // The read position is the valid next position of the last position instead of the next position. + assertEquals(ledger.getNextValidPosition(latestPositionAndCounter.getLeft()), latestCursor.getReadPosition()); + assertEquals(ledger.getNextValidPosition(earliestPositionAndCounter.getLeft()), p2); assertEquals(latestPositionAndCounter.getRight().longValue(), totalInsertedEntries); assertEquals(earliestPositionAndCounter.getRight().longValue(), totalInsertedEntries - earliestCursor.getNumberOfEntriesInBacklog(false)); @@ -3471,7 +3487,8 @@ public void testManagedLedgerRollOverIfFull() throws Exception { ledger.addEntry(new byte[1024 * 1024]); } - Awaitility.await().untilAsserted(() -> Assert.assertEquals(ledger.getLedgersInfoAsList().size(), msgNum / 2)); + Awaitility.await().untilAsserted(() -> Assert.assertEquals(ledger.getLedgersInfoAsList().size(), + msgNum / 2 + 1)); List entries = cursor.readEntries(msgNum); Assert.assertEquals(msgNum, entries.size()); @@ -3486,6 +3503,9 @@ public void testManagedLedgerRollOverIfFull() throws Exception { stateUpdater.setAccessible(true); stateUpdater.set(ledger, ManagedLedgerImpl.State.LedgerOpened); ledger.rollCurrentLedgerIfFull(); + CompletableFuture completableFuture = new CompletableFuture<>(); + ledger.trimConsumedLedgersInBackground(completableFuture); + completableFuture.get(); Awaitility.await().untilAsserted(() -> Assert.assertEquals(ledger.getLedgersInfoAsList().size(), 1)); Awaitility.await().untilAsserted(() -> Assert.assertEquals(ledger.getTotalSize(), 0)); } @@ -3651,8 +3671,12 @@ public void testInvalidateReadHandleWhenDeleteLedger() throws Exception { } List entryList = cursor.readEntries(3); assertEquals(entryList.size(), 3); - assertEquals(ledger.ledgers.size(), 3); - assertEquals(ledger.ledgerCache.size(), 2); + Awaitility.await().untilAsserted(() -> { + log.error("ledger.ledgerCache.size() : " + ledger.ledgerCache.size()); + assertEquals(ledger.ledgerCache.size(), 3); + assertEquals(ledger.ledgers.size(), 4); + }); + cursor.clearBacklog(); cursor2.clearBacklog(); ledger.trimConsumedLedgersInBackground(Futures.NULL_PROMISE); @@ -3681,15 +3705,15 @@ public void testLockReleaseWhenTrimLedger() throws Exception { } List entryList = cursor.readEntries(entries); assertEquals(entryList.size(), entries); - assertEquals(ledger.ledgers.size(), entries); - assertEquals(ledger.ledgerCache.size(), entries - 1); + assertEquals(ledger.ledgers.size() - 1, entries); + assertEquals(ledger.ledgerCache.size() - 1, entries - 1); cursor.clearBacklog(); ledger.trimConsumedLedgersInBackground(Futures.NULL_PROMISE); ledger.trimConsumedLedgersInBackground(Futures.NULL_PROMISE); // Cleanup fails because ManagedLedgerNotFoundException is thrown Awaitility.await().untilAsserted(() -> { - assertEquals(ledger.ledgers.size(), entries); - assertEquals(ledger.ledgerCache.size(), entries - 1); + assertEquals(ledger.ledgers.size() - 1, entries); + assertEquals(ledger.ledgerCache.size() - 1, entries - 1); }); // The lock is released even if an ManagedLedgerNotFoundException occurs, so it can be called repeatedly Awaitility.await().untilAsserted(() -> @@ -3715,13 +3739,13 @@ public void testInvalidateReadHandleWhenConsumed() throws Exception { } List entryList = cursor.readEntries(3); assertEquals(entryList.size(), 3); - assertEquals(ledger.ledgers.size(), 3); - assertEquals(ledger.ledgerCache.size(), 2); + assertEquals(ledger.ledgers.size(), 4); + assertEquals(ledger.ledgerCache.size(), 3); cursor.clearBacklog(); cursor2.clearBacklog(); ledger.trimConsumedLedgersInBackground(Futures.NULL_PROMISE); Awaitility.await().untilAsserted(() -> { - assertEquals(ledger.ledgers.size(), 3); + assertEquals(ledger.ledgers.size(), 4); assertEquals(ledger.ledgerCache.size(), 0); }); @@ -3729,11 +3753,11 @@ public void testInvalidateReadHandleWhenConsumed() throws Exception { ManagedCursor cursor3 = ledger.openCursor("test-cursor3", InitialPosition.Earliest); entryList = cursor3.readEntries(3); assertEquals(entryList.size(), 3); - assertEquals(ledger.ledgerCache.size(), 2); + assertEquals(ledger.ledgerCache.size(), 3); cursor3.clearBacklog(); ledger.trimConsumedLedgersInBackground(Futures.NULL_PROMISE); Awaitility.await().untilAsserted(() -> { - assertEquals(ledger.ledgers.size(), 3); + assertEquals(ledger.ledgers.size(), 4); assertEquals(ledger.ledgerCache.size(), 0); }); @@ -4255,4 +4279,45 @@ public void testNoCleanupOffloadLedgerWhenMetadataExceptionHappens() throws Exce verify(ledgerOffloader, times(0)) .deleteOffloaded(eq(ledgerInfo.getLedgerId()), any(), anyMap()); } + + + @DataProvider(name = "closeLedgerByAddEntry") + public Object[][] closeLedgerByAddEntry() { + return new Object[][] {{Boolean.TRUE}, {Boolean.FALSE}}; + } + + @Test(dataProvider = "closeLedgerByAddEntry") + public void testDeleteCurrentLedgerWhenItIsClosed(boolean closeLedgerByAddEntry) throws Exception { + // Setup: Open a manageLedger with one initial entry. + ManagedLedgerConfig config = new ManagedLedgerConfig(); + config.setMaxEntriesPerLedger(10); + ManagedLedgerImpl ml = spy((ManagedLedgerImpl) factory.open("testDeleteCurrentLedgerWhenItIsClosed", + config)); + assertEquals(ml.ledgers.size(), 1); + ml.addEntry(new byte[4]); + // Act: Trigger the rollover of the current ledger. + long currentLedgerID = ml.currentLedger.getId(); + ml.config.setMaximumRolloverTime(10, TimeUnit.MILLISECONDS); + Thread.sleep(10); + if (closeLedgerByAddEntry) { + // Detect the current ledger is full before written entry and close the ledger after writing completely. + ml.addEntry(new byte[4]); + } else { + // Detect the current ledger is full by the timed task. (Imitate: the timed task `checkLedgerRollTask` call + // `rollCurrentLedgerIfFull` periodically). + ml.rollCurrentLedgerIfFull(); + // the ledger closing in the `rollCurrentLedgerIfFull` is async, so the wait is needed. + Awaitility.await().untilAsserted(() -> assertEquals(ml.ledgers.size(), 2)); + } + // Act: Trigger trimming to delete the previous current ledger. + ml.internalTrimLedgers(false, Futures.NULL_PROMISE); + // Verify: A new ledger will be opened after the current ledger is closed and the previous current ledger can be + // deleted. + Awaitility.await().untilAsserted(() -> { + assertEquals(ml.state, ManagedLedgerImpl.State.LedgerOpened); + assertEquals(ml.ledgers.size(), 1); + assertNotEquals(currentLedgerID, ml.currentLedger.getId()); + assertEquals(ml.currentLedgerEntries, 0); + }); + } } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java index 1e1f7df0a46d5..82141bfd0eeeb 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java @@ -589,12 +589,12 @@ void subscribeToEarliestPositionWithImmediateDeletion() throws Exception { /* Position p1 = */ ledger.addEntry("entry-1".getBytes()); /* Position p2 = */ ledger.addEntry("entry-2".getBytes()); - Position p3 = ledger.addEntry("entry-3".getBytes()); + /* Position p3 = */ ledger.addEntry("entry-3".getBytes()); Thread.sleep(300); ManagedCursor c1 = ledger.newNonDurableCursor(PositionImpl.EARLIEST); - assertEquals(c1.getReadPosition(), p3); - assertEquals(c1.getMarkDeletedPosition(), new PositionImpl(5, -1)); + assertEquals(c1.getReadPosition(), new PositionImpl(6, 0)); + assertEquals(c1.getMarkDeletedPosition(), new PositionImpl(6, -1)); } @Test // (timeOut = 20000) @@ -723,9 +723,10 @@ public void testBacklogStatsWhenDroppingData() throws Exception { CompletableFuture promise = new CompletableFuture<>(); ledger.internalTrimConsumedLedgers(promise); promise.join(); - - assertEquals(nonDurableCursor.getNumberOfEntries(), 6); - assertEquals(nonDurableCursor.getNumberOfEntriesInBacklog(true), 6); + // The mark delete position has moved to position 4:1, and the ledger 4 only has one entry, + // so the ledger 4 can be deleted. nonDurableCursor should has the same backlog with durable cursor. + assertEquals(nonDurableCursor.getNumberOfEntries(), 5); + assertEquals(nonDurableCursor.getNumberOfEntriesInBacklog(true), 5); c1.close(); ledger.deleteCursor(c1.getName()); @@ -733,8 +734,8 @@ public void testBacklogStatsWhenDroppingData() throws Exception { ledger.internalTrimConsumedLedgers(promise); promise.join(); - assertEquals(nonDurableCursor.getNumberOfEntries(), 1); - assertEquals(nonDurableCursor.getNumberOfEntriesInBacklog(true), 1); + assertEquals(nonDurableCursor.getNumberOfEntries(), 0); + assertEquals(nonDurableCursor.getNumberOfEntriesInBacklog(true), 0); ledger.close(); } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java index 4482e9944c0ce..cc4b3f2481152 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java @@ -148,7 +148,10 @@ public void addFailed(ManagedLedgerException exception, Object ctx) { newPos = sourceML.addEntry(data); // new ledger rolled. newPos = sourceML.addEntry(data); - Awaitility.await().untilAsserted(() -> assertEquals(shadowML.ledgers.size(), 5)); + Awaitility.await().untilAsserted(() -> { + assertEquals(shadowML.ledgers.size(), 6); + assertEquals(shadowML.currentLedgerEntries, 0); + }); assertEquals(future.get(), fakePos); // LCE should be updated. log.info("3.Source.LCE={},Shadow.LCE={}", sourceML.lastConfirmedEntry, shadowML.lastConfirmedEntry); 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 0ac5fdaef1599..3918dcbe86d66 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 @@ -228,7 +228,7 @@ public void testBacklogQuotaWithReader() throws Exception { // non-durable mes should still assertEquals(stats.getSubscriptions().size(), 1); long nonDurableSubscriptionBacklog = stats.getSubscriptions().values().iterator().next().getMsgBacklog(); - assertEquals(nonDurableSubscriptionBacklog, MAX_ENTRIES_PER_LEDGER, + assertEquals(nonDurableSubscriptionBacklog, 0, "non-durable subscription backlog is [" + nonDurableSubscriptionBacklog + "]"); MessageIdImpl msgId = null; @@ -254,9 +254,6 @@ public void testBacklogQuotaWithReader() throws Exception { // check there is only one ledger left assertEquals(internalStats.ledgers.size(), 1); - - // check if its the expected ledger id given MAX_ENTRIES_PER_LEDGER - assertEquals(internalStats.ledgers.get(0).ledgerId, finalMsgId.getLedgerId()); }); // check reader can still read with out error @@ -303,10 +300,10 @@ public void testTriggerBacklogQuotaSizeWithReader() throws Exception { TopicStats stats = getTopicStats(topic1); // overall backlogSize should be zero because we only have readers assertEquals(stats.getBacklogSize(), 0, "backlog size is [" + stats.getBacklogSize() + "]"); - // non-durable mes should still assertEquals(stats.getSubscriptions().size(), 1); long nonDurableSubscriptionBacklog = stats.getSubscriptions().values().iterator().next().getMsgBacklog(); - assertEquals(nonDurableSubscriptionBacklog, MAX_ENTRIES_PER_LEDGER, + // All the full ledgers should be deleted. + assertEquals(nonDurableSubscriptionBacklog, 0, "non-durable subscription backlog is [" + nonDurableSubscriptionBacklog + "]"); MessageIdImpl messageId = null; try { @@ -327,8 +324,8 @@ public void testTriggerBacklogQuotaSizeWithReader() throws Exception { // check there is only one ledger left assertEquals(internalStats.ledgers.size(), 1); - // check if its the expected ledger id given MAX_ENTRIES_PER_LEDGER - assertEquals(internalStats.ledgers.get(0).ledgerId, finalMessageId.getLedgerId()); + // check if it's the expected ledger id given MAX_ENTRIES_PER_LEDGER + assertEquals(internalStats.ledgers.get(0).ledgerId, finalMessageId.getLedgerId() + 1); }); // check reader can still read with out error 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 40649a4164047..42b9358911a69 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 @@ -191,9 +191,9 @@ public void testSkipCorruptDataLedger() throws Exception { .build(); final String ns1 = "prop/usc/crash-broker"; - final int totalMessages = 100; + final int totalMessages = 99; final int totalDataLedgers = 5; - final int entriesPerLedger = totalMessages / totalDataLedgers; + final int entriesPerLedger = 20; try { admin.namespaces().createNamespace(ns1); @@ -273,9 +273,9 @@ public void testSkipCorruptDataLedger() throws Exception { retryStrategically((test) -> config.isAutoSkipNonRecoverableData(), 5, 100); - // (5) consumer will be able to consume 20 messages from last non-deleted ledger + // (5) consumer will be able to consume 19 messages from last non-deleted ledger consumer = client.newConsumer().topic(topic1).subscriptionName("my-subscriber-name").subscribe(); - for (int i = 0; i < entriesPerLedger; i++) { + for (int i = 0; i < entriesPerLedger - 1; i++) { msg = consumer.receive(); System.out.println(i); consumer.acknowledge(msg); @@ -296,9 +296,9 @@ public void testTruncateCorruptDataLedger() throws Exception { .statsInterval(0, TimeUnit.SECONDS) .build(); - final int totalMessages = 100; + final int totalMessages = 99; final int totalDataLedgers = 5; - final int entriesPerLedger = totalMessages / totalDataLedgers; + final int entriesPerLedger = 20; final String tenant = "prop"; try { 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 5252407892eea..3b2f3cf215ea3 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 @@ -231,28 +231,43 @@ public void testBookieIsolation() throws Exception { LedgerManager ledgerManager = getLedgerManager(bookie1); // namespace: ns1 - ManagedLedgerImpl ml = (ManagedLedgerImpl) topic1.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml1 = (ManagedLedgerImpl) topic1.getManagedLedger(); + // totalLedgers = totalPublish / totalEntriesPerLedger. (totalPublish = 100, totalEntriesPerLedger = 20.) + // The last ledger is full, a new empty ledger will be created. + // The ledger is created async, so adding a wait is needed. + Awaitility.await().untilAsserted(() -> { + assertEquals(ml1.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml1.getCurrentLedgerEntries(), 0); + }); // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), defaultBookies); + assertAffinityBookies(ledgerManager, ml1.getLedgersInfoAsList(), defaultBookies); // namespace: ns2 - ml = (ManagedLedgerImpl) topic2.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml2 = (ManagedLedgerImpl) topic2.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml2.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml2.getCurrentLedgerEntries(), 0); + }); // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), isolatedBookies); + assertAffinityBookies(ledgerManager, ml2.getLedgersInfoAsList(), isolatedBookies); // namespace: ns3 - ml = (ManagedLedgerImpl) topic3.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml3 = (ManagedLedgerImpl) topic3.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml3.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml3.getCurrentLedgerEntries(), 0); + }); // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), isolatedBookies); + assertAffinityBookies(ledgerManager, ml3.getLedgersInfoAsList(), isolatedBookies); // namespace: ns4 - ml = (ManagedLedgerImpl) topic4.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml4 = (ManagedLedgerImpl) topic4.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml4.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml4.getCurrentLedgerEntries(), 0); + }); // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), isolatedBookies); + assertAffinityBookies(ledgerManager, ml4.getLedgersInfoAsList(), isolatedBookies); ManagedLedgerClientFactory mlFactory = (ManagedLedgerClientFactory) pulsarService.getManagedLedgerClientFactory(); @@ -388,11 +403,14 @@ public void testSetRackInfoAndAffinityGroupDuringProduce() throws Exception { ManagedLedgerImpl ml2 = (ManagedLedgerImpl) topic2.getManagedLedger(); // namespace: ns2 - assertEquals(ml2.getLedgersInfoAsList().size(), totalLedgers); - + Awaitility.await().untilAsserted(() -> { + assertEquals(ml2.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml2.getCurrentLedgerEntries(), 0); + }); List ledgers = ml2.getLedgersInfoAsList(); // validate ledgers' ensemble with affinity bookies - for (int i=1; i> ledgerMetaFuture = ledgerManager.readLedgerMetadata(ledgerId); @@ -529,28 +547,40 @@ public void testStrictBookieIsolation() throws Exception { LedgerManager ledgerManager = getLedgerManager(bookie1); // namespace: ns1 - ManagedLedgerImpl ml = (ManagedLedgerImpl) topic1.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml1 = (ManagedLedgerImpl) topic1.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml1.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml1.getCurrentLedgerEntries(), 0); + }); // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), defaultBookies); + assertAffinityBookies(ledgerManager, ml1.getLedgersInfoAsList(), defaultBookies); // namespace: ns2 - ml = (ManagedLedgerImpl) topic2.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml2 = (ManagedLedgerImpl) topic2.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml2.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml2.getCurrentLedgerEntries(), 0); + }); // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), isolatedBookies); + assertAffinityBookies(ledgerManager, ml2.getLedgersInfoAsList(), isolatedBookies); // namespace: ns3 - ml = (ManagedLedgerImpl) topic3.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml3 = (ManagedLedgerImpl) topic3.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml3.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml3.getCurrentLedgerEntries(), 0); + }); // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), isolatedBookies); + assertAffinityBookies(ledgerManager, ml3.getLedgersInfoAsList(), isolatedBookies); // namespace: ns4 - ml = (ManagedLedgerImpl) topic4.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml4 = (ManagedLedgerImpl) topic4.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml4.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml4.getCurrentLedgerEntries(), 0); + }); // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), isolatedBookies); + assertAffinityBookies(ledgerManager, ml4.getLedgersInfoAsList(), isolatedBookies); ManagedLedgerClientFactory mlFactory = (ManagedLedgerClientFactory) pulsarService.getManagedLedgerClientFactory(); @@ -687,22 +717,32 @@ public void testBookieIsolationWithSecondaryGroup() throws Exception { LedgerManager ledgerManager = getLedgerManager(bookie1); // namespace: ns1 - ManagedLedgerImpl ml = (ManagedLedgerImpl) topic1.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml1 = (ManagedLedgerImpl) topic1.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml1.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml1.getCurrentLedgerEntries(), 0); + }); + // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), defaultBookies); + assertAffinityBookies(ledgerManager, ml1.getLedgersInfoAsList(), defaultBookies); // namespace: ns2 - ml = (ManagedLedgerImpl) topic2.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml2 = (ManagedLedgerImpl) topic2.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml2.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml2.getCurrentLedgerEntries(), 0); + }); // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), isolatedBookies); + assertAffinityBookies(ledgerManager, ml2.getLedgersInfoAsList(), isolatedBookies); // namespace: ns3 - ml = (ManagedLedgerImpl) topic3.getManagedLedger(); - assertEquals(ml.getLedgersInfoAsList().size(), totalLedgers); + ManagedLedgerImpl ml3 = (ManagedLedgerImpl) topic3.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + assertEquals(ml3.getLedgersInfoAsList().size(), totalLedgers + 1); + assertEquals(ml3.getCurrentLedgerEntries(), 0); + }); // validate ledgers' ensemble with affinity bookies - assertAffinityBookies(ledgerManager, ml.getLedgersInfoAsList(), isolatedBookies); + assertAffinityBookies(ledgerManager, ml3.getLedgersInfoAsList(), isolatedBookies); ManagedLedgerClientFactory mlFactory = (ManagedLedgerClientFactory) pulsarService.getManagedLedgerClientFactory(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumedLedgersTrimTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumedLedgersTrimTest.java index 80db4c30f454d..30867dd2cb44d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumedLedgersTrimTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ConsumedLedgersTrimTest.java @@ -97,11 +97,13 @@ public void TestConsumedLedgersTrim() throws Exception { } ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); - Assert.assertEquals(managedLedger.getLedgersInfoAsList().size(), msgNum / 2); + Awaitility.await().untilAsserted(() -> { + Assert.assertEquals(managedLedger.getLedgersInfoAsList().size() - 1, msgNum / 2); + }); //no traffic, unconsumed ledger will be retained Thread.sleep(1200); - Assert.assertEquals(managedLedger.getLedgersInfoAsList().size(), msgNum / 2); + Assert.assertEquals(managedLedger.getLedgersInfoAsList().size() - 1, msgNum / 2); for (int i = 0; i < msgNum; i++) { Message msg = consumer.receive(2, TimeUnit.SECONDS); 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 4ec8107030600..a06085d3d4626 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 @@ -20,6 +20,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; @@ -29,11 +30,14 @@ 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.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.SubscriptionType; import org.apache.pulsar.common.api.proto.CommandCloseProducer; +import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.awaitility.Awaitility; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -186,4 +190,44 @@ public void testSendTimerCheckForBatchContainer(BatcherBuilder batcherBuilder) t future.thenAccept(msgId -> log.info("msg-1 done: {} (msgId: {})", System.nanoTime(), msgId)); future.get(); } + + + @Test + public void testRetentionPolicyByProducingMessages() throws Exception { + // Setup: configure the entries per ledger and retention polices. + final int maxEntriesPerLedger = 10, messagesCount = 10; + final String topicName = BrokerTestUtil.newUniqueName("persistent://my-property/my-ns/tp_"); + pulsar.getConfiguration().setManagedLedgerMaxEntriesPerLedger(maxEntriesPerLedger); + pulsar.getConfiguration().setManagedLedgerMinLedgerRolloverTimeMinutes(0); + pulsar.getConfiguration().setDefaultRetentionTimeInMinutes(0); + pulsar.getConfiguration().setDefaultRetentionSizeInMB(0); + + @Cleanup + Producer producer = pulsarClient.newProducer().topic(topicName) + .sendTimeout(1, TimeUnit.SECONDS) + .enableBatching(false) + .create(); + + @Cleanup + Consumer consumer = pulsarClient.newConsumer().topic(topicName) + .subscriptionName("my-sub") + .subscribe(); + // Act: prepare a full ledger data and ack them. + for (int i = 0; i < messagesCount; i++) { + producer.newMessage().sendAsync(); + } + for (int i = 0; i < messagesCount; i++) { + Message message = consumer.receive(); + assertNotNull(message); + consumer.acknowledge(message); + } + // Verify: a new empty ledger will be created after the current ledger is fulled. + // And the previous consumed ledgers will be deleted + Awaitility.await().untilAsserted(() -> { + admin.topics().trimTopic(topicName); + PersistentTopicInternalStats internalStats = admin.topics().getInternalStatsAsync(topicName).get(); + assertEquals(internalStats.currentLedgerEntries, 0); + assertEquals(internalStats.ledgers.size(), 1); + }); + } } From ce3b816b6910930fdea5df8f8f618e20bbb85e80 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Mon, 15 Apr 2024 13:55:34 +0800 Subject: [PATCH 426/699] [fix][test] Flaky-test: testMessageExpiryWithTimestampNonRecoverableException and testIncorrectClientClock (#22489) (cherry picked from commit d9a43dd21605930e16bb038095e36fceff3a4a40) (cherry picked from commit c590198b8735887f867a15d84b93227f653ac905) --- .../service/PersistentMessageFinderTest.java | 47 +++++++++---------- 1 file changed, 23 insertions(+), 24 deletions(-) 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 1f4aafe58f62d..a062847b56cfc 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 @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.service; -import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.retryStrategically; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.clearInvocations; import static org.mockito.Mockito.doAnswer; @@ -378,7 +377,7 @@ public static Set getBrokerEntryMetadataIntercep * * @throws Exception */ - @Test(groups = "flaky") + @Test void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { final String ledgerAndCursorName = "testPersistentMessageExpiryWithNonRecoverableLedgers"; @@ -397,11 +396,15 @@ void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { for (int i = 0; i < totalEntries; i++) { ledger.addEntry(createMessageWrittenToLedger("msg" + i)); } + Awaitility.await().untilAsserted(() -> + assertEquals(ledger.getState(), ManagedLedgerImpl.State.LedgerOpened)); List ledgers = ledger.getLedgersInfoAsList(); LedgerInfo lastLedgerInfo = ledgers.get(ledgers.size() - 1); - - assertEquals(ledgers.size(), totalEntries / entriesPerLedger); + // The `lastLedgerInfo` should be newly opened, and it does not contain any entries. + // Please refer to: https://github.com/apache/pulsar/pull/22034 + assertEquals(lastLedgerInfo.getEntries(), 0); + assertEquals(ledgers.size(), totalEntries / entriesPerLedger + 1); // this will make sure that all entries should be deleted Thread.sleep(TimeUnit.SECONDS.toMillis(ttlSeconds)); @@ -411,19 +414,13 @@ void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { bkc.deleteLedger(ledgers.get(2).getLedgerId()); PersistentMessageExpiryMonitor monitor = new PersistentMessageExpiryMonitor("topicname", c1.getName(), c1, null); - Position previousMarkDelete = null; - for (int i = 0; i < totalEntries; i++) { - monitor.expireMessages(1); - Position previousPos = previousMarkDelete; - retryStrategically( - (test) -> c1.getMarkDeletedPosition() != null && !c1.getMarkDeletedPosition().equals(previousPos), - 5, 100); - previousMarkDelete = c1.getMarkDeletedPosition(); - } - - PositionImpl markDeletePosition = (PositionImpl) c1.getMarkDeletedPosition(); - assertEquals(lastLedgerInfo.getLedgerId(), markDeletePosition.getLedgerId()); - assertEquals(lastLedgerInfo.getEntries() - 1, markDeletePosition.getEntryId()); + assertTrue(monitor.expireMessages(ttlSeconds)); + Awaitility.await().untilAsserted(() -> { + PositionImpl markDeletePosition = (PositionImpl) c1.getMarkDeletedPosition(); + // The markDeletePosition points to the last entry of the previous ledger in lastLedgerInfo. + assertEquals(markDeletePosition.getLedgerId(), lastLedgerInfo.getLedgerId() - 1); + assertEquals(markDeletePosition.getEntryId(), entriesPerLedger - 1); + }); c1.close(); ledger.close(); @@ -431,24 +428,26 @@ void testMessageExpiryWithTimestampNonRecoverableException() throws Exception { } - @Test(groups = "flaky") + @Test public void testIncorrectClientClock() throws Exception { final String ledgerAndCursorName = "testIncorrectClientClock"; int maxTTLSeconds = 1; + int entriesNum = 10; ManagedLedgerConfig config = new ManagedLedgerConfig(); config.setMaxEntriesPerLedger(1); ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open(ledgerAndCursorName, config); ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor(ledgerAndCursorName); // set client clock to 10 days later long incorrectPublishTimestamp = System.currentTimeMillis() + TimeUnit.DAYS.toMillis(10); - for (int i = 0; i < 10; i++) { + for (int i = 0; i < entriesNum; i++) { ledger.addEntry(createMessageWrittenToLedger("msg" + i, incorrectPublishTimestamp)); } - assertEquals(ledger.getLedgersInfoAsList().size(), 10); - PersistentTopic mock = mock(PersistentTopic.class); - when(mock.getName()).thenReturn("topicname"); - when(mock.getLastPosition()).thenReturn(PositionImpl.EARLIEST); - PersistentMessageExpiryMonitor monitor = new PersistentMessageExpiryMonitor(mock, c1.getName(), c1, null); + Awaitility.await().untilAsserted(() -> + assertEquals(ledger.getState(), ManagedLedgerImpl.State.LedgerOpened)); + // The number of ledgers should be (entriesNum / MaxEntriesPerLedger) + 1 + // Please refer to: https://github.com/apache/pulsar/pull/22034 + assertEquals(ledger.getLedgersInfoAsList().size(), entriesNum + 1); + PersistentMessageExpiryMonitor monitor = new PersistentMessageExpiryMonitor("topicname", c1.getName(), c1, null); Thread.sleep(TimeUnit.SECONDS.toMillis(maxTTLSeconds)); monitor.expireMessages(maxTTLSeconds); assertEquals(c1.getNumberOfEntriesInBacklog(true), 0); From f3740572aaaf07e27e02d7fee1c6d26104bc0a6b Mon Sep 17 00:00:00 2001 From: Hideaki Oguni <22386882+izumo27@users.noreply.github.com> Date: Tue, 16 Apr 2024 00:21:04 +0900 Subject: [PATCH 427/699] [fix][sec] Upgrade Bouncycastle to 1.78 (#22509) Co-authored-by: hoguni (cherry picked from commit 20915d1c438783c05778d98a3b77ec485b79d79d) # Conflicts: # pom.xml (cherry picked from commit ef9b28fc27d1f8c8afdcec8ca8ae1c221092adc0) --- bouncy-castle/bc/LICENSE | 6 +++--- distribution/server/src/assemble/LICENSE.bin.txt | 8 ++++---- distribution/shell/src/assemble/LICENSE.bin.txt | 8 ++++---- pom.xml | 2 +- pulsar-sql/presto-distribution/LICENSE | 8 ++++---- 5 files changed, 16 insertions(+), 16 deletions(-) diff --git a/bouncy-castle/bc/LICENSE b/bouncy-castle/bc/LICENSE index dae8f16df5b82..14f4e76e921d3 100644 --- a/bouncy-castle/bc/LICENSE +++ b/bouncy-castle/bc/LICENSE @@ -205,6 +205,6 @@ This projects includes binary packages with the following licenses: Bouncy Castle License * Bouncy Castle -- licenses/LICENSE-bouncycastle.txt - - org.bouncycastle-bcpkix-jdk18on-1.75.jar - - org.bouncycastle-bcprov-jdk18on-1.75.jar - - org.bouncycastle-bcprov-ext-jdk18on-1.75.jar + - org.bouncycastle-bcpkix-jdk18on-1.78.jar + - org.bouncycastle-bcprov-jdk18on-1.78.jar + - org.bouncycastle-bcprov-ext-jdk18on-1.78.jar diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index ea88f50937441..f3881ce74cc45 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -579,10 +579,10 @@ Creative Commons Attribution License Bouncy Castle License * Bouncy Castle -- ../licenses/LICENSE-bouncycastle.txt - - org.bouncycastle-bcpkix-jdk18on-1.75.jar - - org.bouncycastle-bcprov-ext-jdk18on-1.75.jar - - org.bouncycastle-bcprov-jdk18on-1.75.jar - - org.bouncycastle-bcutil-jdk18on-1.75.jar + - org.bouncycastle-bcpkix-jdk18on-1.78.jar + - org.bouncycastle-bcprov-ext-jdk18on-1.78.jar + - org.bouncycastle-bcprov-jdk18on-1.78.jar + - org.bouncycastle-bcutil-jdk18on-1.78.jar Datastax - com.datastax.oss-managed-ledger-3.1.3.2-SNAPSHOT.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 86e94fca32e7b..96d515c2da165 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -466,10 +466,10 @@ Creative Commons Attribution License Bouncy Castle License * Bouncy Castle -- ../licenses/LICENSE-bouncycastle.txt - - bcpkix-jdk18on-1.75.jar - - bcprov-ext-jdk18on-1.75.jar - - bcprov-jdk18on-1.75.jar - - bcutil-jdk18on-1.75.jar + - bcpkix-jdk18on-1.78.jar + - bcprov-ext-jdk18on-1.78.jar + - bcprov-jdk18on-1.78.jar + - bcutil-jdk18on-1.78.jar ------------------------ diff --git a/pom.xml b/pom.xml index 12d1c20cb1410..4a4bd6429ec59 100644 --- a/pom.xml +++ b/pom.xml @@ -155,7 +155,7 @@ flexible messaging model and an intuitive client API. 1.7.32 4.4 2.18.0 - 1.75 + 1.78 1.0.6 1.0.2.4 2.14.2 diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index fee68f159b93c..03187ff390064 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -588,7 +588,7 @@ Creative Commons Attribution License Bouncy Castle License * Bouncy Castle -- licenses/LICENSE-bouncycastle.txt - - bcpkix-jdk18on-1.75.jar - - bcprov-ext-jdk18on-1.75.jar - - bcprov-jdk18on-1.75.jar - - bcutil-jdk18on-1.75.jar + - bcpkix-jdk18on-1.78.jar + - bcprov-ext-jdk18on-1.78.jar + - bcprov-jdk18on-1.78.jar + - bcutil-jdk18on-1.78.jar From c87fa4ea1c46dc9f6547e564e6c44864af4f5261 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Tue, 16 Apr 2024 08:04:11 +0800 Subject: [PATCH 428/699] [fix][io] Kafka Source connector maybe stuck (#22511) (cherry picked from commit bbff29d8ecc2f6c7ec91e0a48085fe14c8ffd6b8) (cherry picked from commit a0120d00cada882dd82976a548942ffd8a55be52) --- .../pulsar/io/kafka/KafkaAbstractSource.java | 28 +++++- .../kafka/source/KafkaAbstractSourceTest.java | 89 +++++++++++++++++++ 2 files changed, 116 insertions(+), 1 deletion(-) diff --git a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java index bd243974a0fb7..b081bddefe144 100644 --- a/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java +++ b/pulsar-io/kafka/src/main/java/org/apache/pulsar/io/kafka/KafkaAbstractSource.java @@ -27,6 +27,7 @@ import java.util.Optional; import java.util.Properties; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; @@ -62,6 +63,7 @@ public abstract class KafkaAbstractSource extends KafkaPushSource { private volatile boolean running = false; private KafkaSourceConfig kafkaSourceConfig; private Thread runnerThread; + private long maxPollIntervalMs; @Override public void open(Map config, SourceContext sourceContext) throws Exception { @@ -125,6 +127,13 @@ public void open(Map config, SourceContext sourceContext) throws props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, kafkaSourceConfig.getAutoOffsetReset()); props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, kafkaSourceConfig.getKeyDeserializationClass()); props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, kafkaSourceConfig.getValueDeserializationClass()); + if (props.containsKey(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG)) { + maxPollIntervalMs = Long.parseLong(props.get(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG).toString()); + } else { + maxPollIntervalMs = Long.parseLong( + ConsumerConfig.configDef().defaultValues().get(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG) + .toString()); + } try { consumer = new KafkaConsumer<>(beforeCreateConsumer(props)); } catch (Exception ex) { @@ -174,7 +183,9 @@ public void start() { index++; } if (!kafkaSourceConfig.isAutoCommitEnabled()) { - CompletableFuture.allOf(futures).get(); + // Wait about 2/3 of the time of maxPollIntervalMs. + // so as to avoid waiting for the timeout to be kicked out of the consumer group. + CompletableFuture.allOf(futures).get(maxPollIntervalMs * 2 / 3, TimeUnit.MILLISECONDS); consumer.commitSync(); } } catch (Exception e) { @@ -252,6 +263,21 @@ public void ack() { completableFuture.complete(null); } + @Override + public void fail() { + completableFuture.completeExceptionally( + new RuntimeException( + String.format( + "Failed to process record with kafka topic: %s partition: %d offset: %d key: %s", + record.topic(), + record.partition(), + record.offset(), + getKey() + ) + ) + ); + } + @Override public Schema getSchema() { return schema; diff --git a/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java b/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java index 3f5a310d1bab5..6897711730f9f 100644 --- a/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java +++ b/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java @@ -22,12 +22,18 @@ import java.time.Duration; import java.util.Collection; import java.util.Collections; +import java.util.Arrays; import java.lang.reflect.Field; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.io.core.SourceContext; import org.apache.pulsar.io.kafka.KafkaAbstractSource; import org.apache.pulsar.io.kafka.KafkaSourceConfig; @@ -47,6 +53,7 @@ import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; import static org.testng.Assert.expectThrows; import static org.testng.Assert.fail; @@ -219,6 +226,88 @@ public final void throwExceptionByPoll() throws Exception { source.read(); } + @Test + public final void throwExceptionBySendFail() throws Exception { + KafkaAbstractSource source = new DummySource(); + + KafkaSourceConfig kafkaSourceConfig = new KafkaSourceConfig(); + kafkaSourceConfig.setTopic("test-topic"); + kafkaSourceConfig.setAutoCommitEnabled(false); + Field kafkaSourceConfigField = KafkaAbstractSource.class.getDeclaredField("kafkaSourceConfig"); + kafkaSourceConfigField.setAccessible(true); + kafkaSourceConfigField.set(source, kafkaSourceConfig); + + Field defaultMaxPollIntervalMsField = KafkaAbstractSource.class.getDeclaredField("maxPollIntervalMs"); + defaultMaxPollIntervalMsField.setAccessible(true); + defaultMaxPollIntervalMsField.set(source, 300000); + + Consumer consumer = mock(Consumer.class); + ConsumerRecord consumerRecord = new ConsumerRecord<>("topic", 0, 0, + "t-key", "t-value".getBytes(StandardCharsets.UTF_8)); + ConsumerRecords consumerRecords = new ConsumerRecords<>(Collections.singletonMap( + new TopicPartition("topic", 0), + Arrays.asList(consumerRecord))); + Mockito.doReturn(consumerRecords).when(consumer).poll(Mockito.any(Duration.class)); + + Field consumerField = KafkaAbstractSource.class.getDeclaredField("consumer"); + consumerField.setAccessible(true); + consumerField.set(source, consumer); + source.start(); + + // Mock send message fail + Record record = source.read(); + record.fail(); + + // read again will throw RuntimeException. + try { + source.read(); + fail("Should throw exception"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof RuntimeException); + assertTrue(e.getCause().getMessage().contains("Failed to process record with kafka topic")); + } + } + + @Test + public final void throwExceptionBySendTimeOut() throws Exception { + KafkaAbstractSource source = new DummySource(); + + KafkaSourceConfig kafkaSourceConfig = new KafkaSourceConfig(); + kafkaSourceConfig.setTopic("test-topic"); + kafkaSourceConfig.setAutoCommitEnabled(false); + Field kafkaSourceConfigField = KafkaAbstractSource.class.getDeclaredField("kafkaSourceConfig"); + kafkaSourceConfigField.setAccessible(true); + kafkaSourceConfigField.set(source, kafkaSourceConfig); + + Field defaultMaxPollIntervalMsField = KafkaAbstractSource.class.getDeclaredField("maxPollIntervalMs"); + defaultMaxPollIntervalMsField.setAccessible(true); + defaultMaxPollIntervalMsField.set(source, 1); + + Consumer consumer = mock(Consumer.class); + ConsumerRecord consumerRecord = new ConsumerRecord<>("topic", 0, 0, + "t-key", "t-value".getBytes(StandardCharsets.UTF_8)); + ConsumerRecords consumerRecords = new ConsumerRecords<>(Collections.singletonMap( + new TopicPartition("topic", 0), + Arrays.asList(consumerRecord))); + Mockito.doReturn(consumerRecords).when(consumer).poll(Mockito.any(Duration.class)); + + Field consumerField = KafkaAbstractSource.class.getDeclaredField("consumer"); + consumerField.setAccessible(true); + consumerField.set(source, consumer); + source.start(); + + // Mock send message fail, just read do noting. + source.read(); + + // read again will throw TimeOutException. + try { + source.read(); + fail("Should throw exception"); + } catch (Exception e) { + assertTrue(e instanceof TimeoutException); + } + } + private File getFile(String name) { ClassLoader classLoader = getClass().getClassLoader(); return new File(classLoader.getResource(name).getFile()); From f8d509852cdedc1e1edef1978405dd5b93a39898 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 17 Apr 2024 03:15:01 -0700 Subject: [PATCH 429/699] [improve][broker] Optimize gzip compression for /metrics endpoint by sharing/caching compressed result (#22521) (cherry picked from commit 94f6c7ccd2bf8bc261d45ab41f6c7f123359fa47) # Conflicts: # pulsar-broker/src/main/java/org/apache/pulsar/broker/PulsarService.java (cherry picked from commit 212b85f6bf90f2e0b8232567993e43ea86525457) --- .../prometheus/PrometheusMetricsServlet.java | 1 + .../pulsar/broker/web/GzipHandlerUtil.java | 21 +++ .../broker/web/GzipHandlerUtilTest.java | 36 ++++ .../apache/pulsar/broker/PulsarService.java | 3 +- .../PrometheusMetricsGenerator.java | 176 ++++++++++++++++-- .../PulsarPrometheusMetricsServlet.java | 28 ++- .../pulsar/PrometheusMetricsTestUtil.java | 2 +- 7 files changed, 253 insertions(+), 14 deletions(-) create mode 100644 pulsar-broker-common/src/test/java/org/apache/pulsar/broker/web/GzipHandlerUtilTest.java diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java index 8a41bed29d44f..8685348174cd6 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsServlet.java @@ -39,6 +39,7 @@ import org.slf4j.LoggerFactory; public class PrometheusMetricsServlet extends HttpServlet { + public static final String DEFAULT_METRICS_PATH = "/metrics"; private static final long serialVersionUID = 1L; static final int HTTP_STATUS_OK_200 = 200; static final int HTTP_STATUS_INTERNAL_SERVER_ERROR_500 = 500; diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java index 37c9c05e5d53c..9e980cecb791f 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/GzipHandlerUtil.java @@ -19,8 +19,10 @@ package org.apache.pulsar.broker.web; import java.util.List; +import org.eclipse.jetty.http.pathmap.PathSpecSet; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.handler.gzip.GzipHandler; +import org.eclipse.jetty.util.IncludeExclude; public class GzipHandlerUtil { public static Handler wrapWithGzipHandler(Handler innerHandler, List gzipCompressionExcludedPaths) { @@ -45,4 +47,23 @@ public static boolean isGzipCompressionCompletelyDisabled(List gzipCompr && (gzipCompressionExcludedPaths.get(0).equals("^.*") || gzipCompressionExcludedPaths.get(0).equals("^.*$")); } + + /** + * Check if GZIP compression is enabled for the given endpoint. + * @param gzipCompressionExcludedPaths list of paths that should not be compressed + * @param endpoint the endpoint to check + * @return true if GZIP compression is enabled for the endpoint, false otherwise + */ + public static boolean isGzipCompressionEnabledForEndpoint(List gzipCompressionExcludedPaths, + String endpoint) { + if (gzipCompressionExcludedPaths == null || gzipCompressionExcludedPaths.isEmpty()) { + return true; + } + if (isGzipCompressionCompletelyDisabled(gzipCompressionExcludedPaths)) { + return false; + } + IncludeExclude paths = new IncludeExclude<>(PathSpecSet.class); + paths.exclude(gzipCompressionExcludedPaths.toArray(new String[0])); + return paths.test(endpoint); + } } diff --git a/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/web/GzipHandlerUtilTest.java b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/web/GzipHandlerUtilTest.java new file mode 100644 index 0000000000000..d6958695dec9f --- /dev/null +++ b/pulsar-broker-common/src/test/java/org/apache/pulsar/broker/web/GzipHandlerUtilTest.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.broker.web; + +import static org.testng.Assert.*; +import java.util.Arrays; +import org.testng.annotations.Test; + +public class GzipHandlerUtilTest { + + @Test + public void testIsGzipCompressionEnabledForEndpoint() { + assertTrue(GzipHandlerUtil.isGzipCompressionEnabledForEndpoint(null, "/metrics")); + assertFalse(GzipHandlerUtil.isGzipCompressionEnabledForEndpoint(Arrays.asList("^.*"), "/metrics")); + assertFalse(GzipHandlerUtil.isGzipCompressionEnabledForEndpoint(Arrays.asList("^.*$"), "/metrics")); + assertFalse(GzipHandlerUtil.isGzipCompressionEnabledForEndpoint(Arrays.asList("/metrics"), "/metrics")); + assertTrue(GzipHandlerUtil.isGzipCompressionEnabledForEndpoint(Arrays.asList("/metrics"), "/metrics2")); + assertTrue(GzipHandlerUtil.isGzipCompressionEnabledForEndpoint(Arrays.asList("/admin", "/custom"), "/metrics")); + } +} \ No newline at end of file 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 5410bacbe78eb..af89b4b99c57a 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 @@ -107,6 +107,7 @@ import org.apache.pulsar.broker.service.schema.SchemaRegistryService; import org.apache.pulsar.broker.service.schema.SchemaStorageFactory; import org.apache.pulsar.broker.stats.MetricsGenerator; +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.ManagedLedgerStorage; @@ -1024,7 +1025,7 @@ private void addWebServerHandlers(WebService webService, true, attributeMap, true, Topics.class); // Add metrics servlet - webService.addServlet("/metrics", + webService.addServlet(PrometheusMetricsServlet.DEFAULT_METRICS_PATH, new ServletHolder(metricsServlet), config.isAuthenticateMetricsEndpoint(), attributeMap); 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 da77fe9b3341d..364fd59664cc5 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 @@ -30,6 +30,8 @@ import java.io.OutputStream; import java.io.OutputStreamWriter; import java.io.Writer; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.nio.CharBuffer; import java.nio.charset.StandardCharsets; import java.time.Clock; @@ -44,6 +46,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.zip.CRC32; +import java.util.zip.Deflater; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.stats.NullStatsProvider; import org.apache.bookkeeper.stats.StatsProvider; @@ -73,7 +77,7 @@ public class PrometheusMetricsGenerator implements AutoCloseable { private volatile boolean closed; public static class MetricsBuffer { - private final CompletableFuture bufferFuture; + private final CompletableFuture bufferFuture; private final long createTimeslot; private final AtomicInteger refCnt = new AtomicInteger(2); @@ -82,7 +86,7 @@ public static class MetricsBuffer { createTimeslot = timeslot; } - public CompletableFuture getBufferFuture() { + public CompletableFuture getBufferFuture() { return bufferFuture; } @@ -114,6 +118,151 @@ public void release() { } } + /** + * A wraps the response buffer and asynchronously provides a gzip compressed buffer when requested. + */ + public static class ResponseBuffer { + private final ByteBuf uncompressedBuffer; + private boolean released = false; + private CompletableFuture compressedBuffer; + + private ResponseBuffer(final ByteBuf uncompressedBuffer) { + this.uncompressedBuffer = uncompressedBuffer; + } + + public ByteBuf getUncompressedBuffer() { + return uncompressedBuffer; + } + + public synchronized CompletableFuture getCompressedBuffer(Executor executor) { + if (released) { + throw new IllegalStateException("Already released!"); + } + if (compressedBuffer == null) { + compressedBuffer = new CompletableFuture<>(); + ByteBuf retainedDuplicate = uncompressedBuffer.retainedDuplicate(); + executor.execute(() -> { + try { + compressedBuffer.complete(compress(retainedDuplicate)); + } catch (Exception e) { + compressedBuffer.completeExceptionally(e); + } finally { + retainedDuplicate.release(); + } + }); + } + return compressedBuffer; + } + + private ByteBuf compress(ByteBuf uncompressedBuffer) { + GzipByteBufferWriter gzipByteBufferWriter = new GzipByteBufferWriter(uncompressedBuffer.alloc(), + uncompressedBuffer.readableBytes()); + return gzipByteBufferWriter.compress(uncompressedBuffer); + } + + public synchronized void release() { + released = true; + uncompressedBuffer.release(); + if (compressedBuffer != null) { + compressedBuffer.whenComplete((byteBuf, throwable) -> { + if (byteBuf != null) { + byteBuf.release(); + } + }); + } + } + } + + /** + * Compress input nio buffers into gzip format with output in a Netty composite ByteBuf. + */ + private static class GzipByteBufferWriter { + private static final byte[] GZIP_HEADER = + new byte[] {(byte) 0x1f, (byte) 0x8b, Deflater.DEFLATED, 0, 0, 0, 0, 0, 0, 0}; + private final ByteBufAllocator bufAllocator; + private final Deflater deflater; + private final CRC32 crc; + private final int bufferSize; + private final CompositeByteBuf resultBuffer; + private ByteBuf backingCompressBuffer; + private ByteBuffer compressBuffer; + + GzipByteBufferWriter(ByteBufAllocator bufAllocator, int readableBytes) { + deflater = new Deflater(Deflater.DEFAULT_COMPRESSION, true); + crc = new CRC32(); + this.bufferSize = Math.max(Math.min(resolveChunkSize(bufAllocator), readableBytes), 8192); + this.bufAllocator = bufAllocator; + this.resultBuffer = bufAllocator.compositeDirectBuffer(readableBytes / bufferSize + 1); + allocateBuffer(); + } + + /** + * Compress the input Netty buffer and append it to the result buffer in gzip format. + * @param uncompressedBuffer + */ + public ByteBuf compress(ByteBuf uncompressedBuffer) { + try { + ByteBuffer[] nioBuffers = uncompressedBuffer.nioBuffers(); + for (int i = 0, nioBuffersLength = nioBuffers.length; i < nioBuffersLength; i++) { + ByteBuffer nioBuffer = nioBuffers[i]; + compressAndAppend(nioBuffer, i == 0, i == nioBuffersLength - 1); + } + return resultBuffer; + } finally { + close(); + } + } + + private void compressAndAppend(ByteBuffer nioBuffer, boolean isFirst, boolean isLast) { + if (isFirst) { + // write gzip header + compressBuffer.put(GZIP_HEADER); + } + nioBuffer.mark(); + crc.update(nioBuffer); + nioBuffer.reset(); + deflater.setInput(nioBuffer); + if (isLast) { + deflater.finish(); + } + while (!deflater.needsInput() && !deflater.finished()) { + int written = deflater.deflate(compressBuffer); + if (written == 0 && !compressBuffer.hasRemaining()) { + backingCompressBuffer.setIndex(0, compressBuffer.position()); + resultBuffer.addComponent(true, backingCompressBuffer); + allocateBuffer(); + } + } + if (isLast) { + // write gzip footer, integer values are in little endian byte order + compressBuffer.order(ByteOrder.LITTLE_ENDIAN); + // write CRC32 checksum + compressBuffer.putInt((int) crc.getValue()); + // write uncompressed size + compressBuffer.putInt(deflater.getTotalIn()); + // append the last compressed buffer + backingCompressBuffer.setIndex(0, compressBuffer.position()); + resultBuffer.addComponent(true, backingCompressBuffer); + backingCompressBuffer = null; + compressBuffer = null; + } + } + + private void allocateBuffer() { + backingCompressBuffer = bufAllocator.directBuffer(bufferSize); + compressBuffer = backingCompressBuffer.nioBuffer(0, bufferSize); + } + + private void close() { + if (deflater != null) { + deflater.end(); + } + if (backingCompressBuffer != null) { + backingCompressBuffer.release(); + } + } + } + private final PulsarService pulsar; private final boolean includeTopicMetrics; private final boolean includeConsumerMetrics; @@ -188,13 +337,7 @@ private 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; - if (byteBufAllocator instanceof PooledByteBufAllocator) { - PooledByteBufAllocator pooledByteBufAllocator = (PooledByteBufAllocator) byteBufAllocator; - chunkSize = Math.max(pooledByteBufAllocator.metric().chunkSize(), DEFAULT_INITIAL_BUFFER_SIZE); - } else { - chunkSize = DEFAULT_INITIAL_BUFFER_SIZE; - } + int chunkSize = resolveChunkSize(byteBufAllocator); CompositeByteBuf buf = byteBufAllocator.compositeDirectBuffer( Math.max(MINIMUM_FOR_MAX_COMPONENTS, (initialBufferSize / chunkSize) + 1)); int totalLen = 0; @@ -205,6 +348,17 @@ private ByteBuf allocateMultipartCompositeDirectBuffer() { return buf; } + private static int resolveChunkSize(ByteBufAllocator byteBufAllocator) { + int chunkSize; + if (byteBufAllocator instanceof PooledByteBufAllocator) { + PooledByteBufAllocator pooledByteBufAllocator = (PooledByteBufAllocator) byteBufAllocator; + chunkSize = Math.max(pooledByteBufAllocator.metric().chunkSize(), DEFAULT_INITIAL_BUFFER_SIZE); + } else { + chunkSize = DEFAULT_INITIAL_BUFFER_SIZE; + } + return chunkSize; + } + private static void generateBrokerBasicMetrics(PulsarService pulsar, SimpleTextOutputStream stream) { String clusterName = pulsar.getConfiguration().getClusterName(); // generate managedLedgerCache metrics @@ -339,10 +493,10 @@ public MetricsBuffer renderToBuffer(Executor executor, List bufferFuture = newMetricsBuffer.getBufferFuture(); + CompletableFuture bufferFuture = newMetricsBuffer.getBufferFuture(); executor.execute(() -> { try { - bufferFuture.complete(generate0(metricsProviders)); + bufferFuture.complete(new ResponseBuffer(generate0(metricsProviders))); } catch (Exception e) { bufferFuture.completeExceptionally(e); } finally { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java index 7fcc74e965c24..43514d481dcab 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/PulsarPrometheusMetricsServlet.java @@ -18,10 +18,13 @@ */ package org.apache.pulsar.broker.stats.prometheus; +import static org.apache.pulsar.broker.web.GzipHandlerUtil.isGzipCompressionEnabledForEndpoint; import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; import java.time.Clock; +import java.util.Arrays; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import javax.servlet.AsyncContext; @@ -40,6 +43,7 @@ public class PulsarPrometheusMetricsServlet extends PrometheusMetricsServlet { private static final int EXECUTOR_MAX_THREADS = 4; private final PrometheusMetricsGenerator prometheusMetricsGenerator; + private final boolean gzipCompressionEnabledForMetrics; public PulsarPrometheusMetricsServlet(PulsarService pulsar, boolean includeTopicMetrics, boolean includeConsumerMetrics, boolean includeProducerMetrics, @@ -50,6 +54,8 @@ public PulsarPrometheusMetricsServlet(PulsarService pulsar, boolean includeTopic prometheusMetricsGenerator = new PrometheusMetricsGenerator(pulsar, includeTopicMetrics, includeConsumerMetrics, includeProducerMetrics, splitTopicAndPartitionLabel, Clock.systemUTC()); + gzipCompressionEnabledForMetrics = isGzipCompressionEnabledForEndpoint( + pulsar.getConfiguration().getHttpServerGzipCompressionExcludedPaths(), DEFAULT_METRICS_PATH); } @@ -100,7 +106,14 @@ public void onStartAsync(AsyncEvent event) throws IOException { context.complete(); return; } - metricsBuffer.getBufferFuture().whenComplete((buffer, ex) -> executor.execute(() -> { + boolean compressOutput = gzipCompressionEnabledForMetrics && isGzipAccepted(request); + metricsBuffer.getBufferFuture().thenCompose(responseBuffer -> { + if (compressOutput) { + return responseBuffer.getCompressedBuffer(executor); + } else { + return CompletableFuture.completedFuture(responseBuffer.getUncompressedBuffer()); + } + }).whenComplete((buffer, ex) -> executor.execute(() -> { try { long elapsedNanos = System.nanoTime() - startNanos; // check if the request has been timed out, implement a soft timeout @@ -133,6 +146,9 @@ public void onStartAsync(AsyncEvent event) throws IOException { } else { response.setStatus(HTTP_STATUS_OK_200); response.setContentType("text/plain;charset=utf-8"); + if (compressOutput) { + response.setHeader("Content-Encoding", "gzip"); + } ServletOutputStream outputStream = response.getOutputStream(); if (outputStream instanceof HttpOutput) { HttpOutput output = (HttpOutput) outputStream; @@ -156,4 +172,14 @@ public void onStartAsync(AsyncEvent event) throws IOException { } })); } + + private boolean isGzipAccepted(HttpServletRequest request) { + String acceptEncoding = request.getHeader("Accept-Encoding"); + if (acceptEncoding != null) { + return Arrays.stream(acceptEncoding.split(",")) + .map(String::trim) + .anyMatch(str -> "gzip".equalsIgnoreCase(str)); + } + return false; + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java b/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java index fcc3b6aa88fb4..68826372b7bd6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/PrometheusMetricsTestUtil.java @@ -55,7 +55,7 @@ public static void generate(PrometheusMetricsGenerator metricsGenerator, OutputS try { ByteBuf buffer = null; try { - buffer = metricsBuffer.getBufferFuture().get(5, TimeUnit.SECONDS); + buffer = metricsBuffer.getBufferFuture().get(5, TimeUnit.SECONDS).getUncompressedBuffer(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new IOException(e); From 984d3c06748d8719b8cc6a8862ca7f4b05c5c717 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Wed, 17 Apr 2024 23:59:36 -0700 Subject: [PATCH 430/699] [fix][broker] Fix a deadlock in SystemTopicBasedTopicPoliciesService during NamespaceEventsSystemTopicFactory init (#22528) (cherry picked from commit 72474d7a2dabdf7acf0b158bd07f1bc8b69b790e) (cherry picked from commit 404bcdf9da2f668f25b12a7a005462052d74ca3d) --- .../SystemTopicBasedTopicPoliciesService.java | 49 ++++++++++++------- 1 file changed, 32 insertions(+), 17 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 4e9e875bcf4c3..0449e5c885cd3 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 @@ -34,6 +34,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import javax.annotation.Nonnull; +import org.apache.commons.lang3.concurrent.ConcurrentInitializer; +import org.apache.commons.lang3.concurrent.LazyInitializer; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; @@ -70,7 +72,19 @@ public class SystemTopicBasedTopicPoliciesService implements TopicPoliciesServic private final PulsarService pulsarService; private final HashSet localCluster; private final String clusterName; - private volatile NamespaceEventsSystemTopicFactory namespaceEventsSystemTopicFactory; + + private final ConcurrentInitializer + namespaceEventsSystemTopicFactoryLazyInitializer = new LazyInitializer<>() { + @Override + protected NamespaceEventsSystemTopicFactory initialize() { + try { + return new NamespaceEventsSystemTopicFactory(pulsarService.getClient()); + } catch (PulsarServerException e) { + log.error("Create namespace event system topic factory error.", e); + throw new RuntimeException(e); + } + } + }; @VisibleForTesting final Map policiesCache = new ConcurrentHashMap<>(); @@ -102,7 +116,7 @@ public SystemTopicBasedTopicPoliciesService(PulsarService pulsarService) { }); }) .buildAsync((namespaceName, executor) -> { - SystemTopicClient systemTopicClient = namespaceEventsSystemTopicFactory + SystemTopicClient systemTopicClient = getNamespaceEventsSystemTopicFactory() .createTopicPoliciesSystemTopicClient(namespaceName); return systemTopicClient.newWriterAsync(); }); @@ -301,7 +315,7 @@ public CompletableFuture getTopicPoliciesBypassCacheAsync(TopicNa result.complete(null); return result; } - SystemTopicClient systemTopicClient = namespaceEventsSystemTopicFactory + SystemTopicClient systemTopicClient = getNamespaceEventsSystemTopicFactory() .createTopicPoliciesSystemTopicClient(topicName.getNamespaceObject()); systemTopicClient.newReaderAsync().thenAccept(r -> fetchTopicPoliciesAsyncAndCloseReader(r, topicName, null, result)); @@ -373,7 +387,7 @@ protected CompletableFuture> createSystemT } catch (PulsarServerException ex) { return FutureUtil.failedFuture(ex); } - final SystemTopicClient systemTopicClient = namespaceEventsSystemTopicFactory + final SystemTopicClient systemTopicClient = getNamespaceEventsSystemTopicFactory() .createTopicPoliciesSystemTopicClient(namespace); return systemTopicClient.newReaderAsync(); } @@ -561,7 +575,7 @@ private void refreshTopicPoliciesCache(Message msg) { log.error("Failed to create system topic factory"); break; } - SystemTopicClient systemTopicClient = namespaceEventsSystemTopicFactory + SystemTopicClient systemTopicClient = getNamespaceEventsSystemTopicFactory() .createTopicPoliciesSystemTopicClient(topicName.getNamespaceObject()); systemTopicClient.newWriterAsync().thenAccept(writer -> writer.deleteAsync(getEventKey(topicName), @@ -595,18 +609,19 @@ private boolean hasReplicateTo(Message message) { } private void createSystemTopicFactoryIfNeeded() throws PulsarServerException { - if (namespaceEventsSystemTopicFactory == null) { - synchronized (this) { - if (namespaceEventsSystemTopicFactory == null) { - try { - namespaceEventsSystemTopicFactory = - new NamespaceEventsSystemTopicFactory(pulsarService.getClient()); - } catch (PulsarServerException e) { - log.error("Create namespace event system topic factory error.", e); - throw e; - } - } - } + try { + getNamespaceEventsSystemTopicFactory(); + } catch (Exception e) { + throw new PulsarServerException(e); + } + } + + private NamespaceEventsSystemTopicFactory getNamespaceEventsSystemTopicFactory() { + try { + return namespaceEventsSystemTopicFactoryLazyInitializer.get(); + } catch (Exception e) { + log.error("Create namespace event system topic factory error.", e); + throw new RuntimeException(e); } } From 3008d35a0790031b640c423073753d918cf53ab8 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 18 Apr 2024 07:48:55 -0700 Subject: [PATCH 431/699] [improve][build] Upgrade OWASP Dependency check version to 9.1.0 (#22530) (cherry picked from commit 46e0871725dcc420fb36913dfcf15d32f6c44f89) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 4a4bd6429ec59..368a973c0dec9 100644 --- a/pom.xml +++ b/pom.xml @@ -302,7 +302,7 @@ flexible messaging model and an intuitive client API. 0.1.4 1.3 0.4 - 9.0.7 + 9.1.0 0.9.44 1.6.1 6.4.0 From f87ef7487dd742fbb95cf7010b04fc98ad90b84a Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 19 Apr 2024 09:13:19 +0800 Subject: [PATCH 432/699] [fix][broker] Fix NPE causing dispatching to stop when using Key_Shared mode and allowOutOfOrderDelivery=true (#22533) (cherry picked from commit 2badcf6bd0be1aad2a5ec6da552185b4ef5b745b) (cherry picked from commit 53d784815c3027f3fce456240a28694ee2730f4b) --- ...ntStickyKeyDispatcherMultipleConsumers.java | 10 ++++++++++ .../client/api/KeySharedSubscriptionTest.java | 18 ++++++++++++++++-- 2 files changed, 26 insertions(+), 2 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 ee2ebd7ca867e..2df9f38531f5d 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 @@ -457,6 +457,11 @@ private int getAvailablePermits(Consumer c) { @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; } @@ -501,6 +506,11 @@ protected synchronized NavigableSet filterOutEntriesWillBeDiscarde */ @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; 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 7219555050839..27aa98597ec12 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 @@ -1741,6 +1741,14 @@ public void testNoRepeatedReadAndDiscard() throws Exception { admin.topics().delete(topic, false); } + @DataProvider(name = "allowKeySharedOutOfOrder") + public Object[][] allowKeySharedOutOfOrder() { + return new Object[][]{ + {true}, + {false} + }; + } + /** * This test is in order to guarantee the feature added by https://github.com/apache/pulsar/pull/7105. * 1. Start 3 consumers: @@ -1755,8 +1763,8 @@ public void testNoRepeatedReadAndDiscard() throws Exception { * - no repeated Read-and-discard. * - at last, all messages will be received. */ - @Test(timeOut = 180 * 1000) // the test will be finished in 60s. - public void testRecentJoinedPosWillNotStuckOtherConsumer() throws Exception { + @Test(timeOut = 180 * 1000, dataProvider = "allowKeySharedOutOfOrder") // the test will be finished in 60s. + 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"); @@ -1775,6 +1783,8 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer() throws Exception { log.info("Published message :{}", messageId); } + KeySharedPolicy keySharedPolicy = KeySharedPolicy.autoSplitHashRange() + .setAllowOutOfOrderDelivery(allowKeySharedOutOfOrder); // 1. Start 3 consumers and make ack holes. // - one consumer will be closed and trigger a messages redeliver. // - one consumer will not ack any messages to make the new consumer joined late will be stuck due to the @@ -1785,18 +1795,21 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer() throws Exception { .subscriptionName(subName) .receiverQueueSize(10) .subscriptionType(SubscriptionType.Key_Shared) + .keySharedPolicy(keySharedPolicy) .subscribe(); Consumer consumer2 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) .subscriptionName(subName) .receiverQueueSize(10) .subscriptionType(SubscriptionType.Key_Shared) + .keySharedPolicy(keySharedPolicy) .subscribe(); Consumer consumer3 = pulsarClient.newConsumer(Schema.INT32) .topic(topic) .subscriptionName(subName) .receiverQueueSize(10) .subscriptionType(SubscriptionType.Key_Shared) + .keySharedPolicy(keySharedPolicy) .subscribe(); List msgList1 = new ArrayList<>(); List msgList2 = new ArrayList<>(); @@ -1845,6 +1858,7 @@ public void testRecentJoinedPosWillNotStuckOtherConsumer() throws Exception { .subscriptionName(subName) .receiverQueueSize(1000) .subscriptionType(SubscriptionType.Key_Shared) + .keySharedPolicy(keySharedPolicy) .subscribe(); consumerWillBeClose.close(); From b356f1be1f34c6bc9a5c8a8ea68b8a31511cfddb Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Fri, 19 Apr 2024 21:51:10 +0800 Subject: [PATCH 433/699] [improve][offload] Apply autoSkipNonRecoverableData configuration to tiered storage (#22531) (cherry picked from commit fbf4cb71a3f3ed08786205dc5e60b810f3d62605) (cherry picked from commit ff8d3b7343781fbb874d55635df27dd13f48b68e) --- .../impl/BlobStoreBackedInputStreamImpl.java | 8 +++ .../impl/BlobStoreBackedReadHandleImpl.java | 13 ++++- .../impl/BlobStoreBackedReadHandleImplV2.java | 13 ++++- .../BlobStoreBackedInputStreamTest.java | 5 +- ...reManagedLedgerOffloaderStreamingTest.java | 54 +++++++++++++++++++ .../BlobStoreManagedLedgerOffloaderTest.java | 23 ++++++++ 6 files changed, 110 insertions(+), 6 deletions(-) diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java index 0dea46726f50a..6cb60e14984f9 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedInputStreamImpl.java @@ -28,6 +28,7 @@ import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.naming.TopicName; import org.jclouds.blobstore.BlobStore; +import org.jclouds.blobstore.KeyNotFoundException; import org.jclouds.blobstore.domain.Blob; import org.jclouds.blobstore.options.GetOptions; import org.slf4j.Logger; @@ -95,6 +96,9 @@ private boolean refillBufferIfNeeded() throws IOException { try { long startReadTime = System.nanoTime(); Blob blob = blobStore.getBlob(bucket, key, new GetOptions().range(startRange, endRange)); + if (blob == null) { + throw new KeyNotFoundException(bucket, key, ""); + } versionCheck.check(key, blob); try (InputStream stream = blob.getPayload().openStream()) { @@ -121,6 +125,10 @@ private boolean refillBufferIfNeeded() throws IOException { if (null != this.offloaderStats) { this.offloaderStats.recordReadOffloadError(this.topicName); } + // If the blob is not found, the original exception is thrown and handled by the caller. + if (e instanceof KeyNotFoundException) { + throw e; + } throw new IOException("Error reading from BlobStore", e); } } diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java index ef39f7f50d07e..fd7195673fad2 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java @@ -50,6 +50,7 @@ import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.naming.TopicName; import org.jclouds.blobstore.BlobStore; +import org.jclouds.blobstore.KeyNotFoundException; import org.jclouds.blobstore.domain.Blob; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -206,7 +207,11 @@ public CompletableFuture readAsync(long firstEntry, long lastEntr } catch (Throwable t) { log.error("Failed to read entries {} - {} from the offloader in ledger {}", firstEntry, lastEntry, ledgerId, t); - promise.completeExceptionally(t); + if (t instanceof KeyNotFoundException) { + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); + } else { + promise.completeExceptionally(t); + } entries.forEach(LedgerEntry::close); } }); @@ -270,7 +275,7 @@ public static ReadHandle open(ScheduledExecutorService executor, VersionCheck versionCheck, long ledgerId, int readBufferSize, LedgerOffloaderStats offloaderStats, String managedLedgerName) - throws IOException { + throws IOException, BKException.BKNoSuchLedgerExistsException { int retryCount = 3; OffloadIndexBlock index = null; IOException lastException = null; @@ -283,6 +288,10 @@ public static ReadHandle open(ScheduledExecutorService executor, while (retryCount-- > 0) { long readIndexStartTime = System.nanoTime(); Blob blob = blobStore.getBlob(bucket, indexKey); + if (blob == null) { + log.error("{} not found in container {}", indexKey, bucket); + throw new BKException.BKNoSuchLedgerExistsException(); + } offloaderStats.recordReadOffloadIndexLatency(topicName, System.nanoTime() - readIndexStartTime, TimeUnit.NANOSECONDS); versionCheck.check(indexKey, blob); diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImplV2.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImplV2.java index 53d96e08abf5e..502f475174cee 100644 --- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImplV2.java +++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImplV2.java @@ -49,6 +49,7 @@ import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.naming.TopicName; import org.jclouds.blobstore.BlobStore; +import org.jclouds.blobstore.KeyNotFoundException; import org.jclouds.blobstore.domain.Blob; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -224,7 +225,11 @@ public CompletableFuture readAsync(long firstEntry, long lastEntr } } } catch (Throwable t) { - promise.completeExceptionally(t); + if (t instanceof KeyNotFoundException) { + promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsException()); + } else { + promise.completeExceptionally(t); + } entries.forEach(LedgerEntry::close); } @@ -303,7 +308,7 @@ public static ReadHandle open(ScheduledExecutorService executor, VersionCheck versionCheck, long ledgerId, int readBufferSize, LedgerOffloaderStats offloaderStats, String managedLedgerName) - throws IOException { + throws IOException, BKException.BKNoSuchLedgerExistsException { List inputStreams = new LinkedList<>(); List indice = new LinkedList<>(); String topicName = TopicName.fromPersistenceNamingEncoding(managedLedgerName); @@ -313,6 +318,10 @@ public static ReadHandle open(ScheduledExecutorService executor, log.debug("open bucket: {} index key: {}", bucket, indexKey); long startTime = System.nanoTime(); Blob blob = blobStore.getBlob(bucket, indexKey); + if (blob == null) { + log.error("{} not found in container {}", indexKey, bucket); + throw new BKException.BKNoSuchLedgerExistsException(); + } offloaderStats.recordReadOffloadIndexLatency(topicName, System.nanoTime() - startTime, TimeUnit.NANOSECONDS); log.debug("indexKey blob: {} {}", indexKey, blob); diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/BlobStoreBackedInputStreamTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/BlobStoreBackedInputStreamTest.java index 775310925a1a3..3e5c4b609dfec 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/BlobStoreBackedInputStreamTest.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/BlobStoreBackedInputStreamTest.java @@ -32,6 +32,7 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.offload.jcloud.impl.BlobStoreBackedInputStreamImpl; import org.jclouds.blobstore.BlobStore; +import org.jclouds.blobstore.KeyNotFoundException; import org.jclouds.blobstore.domain.Blob; import org.jclouds.io.Payload; import org.jclouds.io.Payloads; @@ -142,8 +143,8 @@ public void testReadingFullObjectByBytes() throws Exception { assertStreamsMatchByBytes(toTest, toCompare); } - @Test(expectedExceptions = IOException.class) - public void testErrorOnRead() throws Exception { + @Test(expectedExceptions = KeyNotFoundException.class) + public void testNotFoundOnRead() throws Exception { BackedInputStream toTest = new BlobStoreBackedInputStreamImpl(blobStore, BUCKET, "doesn't exist", (key, md) -> {}, 1234, 1000); diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderStreamingTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderStreamingTest.java index 9056281a308f2..ad1529072f813 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderStreamingTest.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderStreamingTest.java @@ -18,16 +18,19 @@ */ package org.apache.bookkeeper.mledger.offload.jcloud.impl; +import static org.apache.bookkeeper.client.api.BKException.Code.NoSuchLedgerExistsException; import static org.mockito.AdditionalAnswers.delegatesTo; import static org.mockito.Mockito.mock; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.fail; import java.io.IOException; import java.util.HashMap; import java.util.LinkedList; import java.util.Map; import java.util.Random; import java.util.UUID; +import org.apache.bookkeeper.client.BKException; import org.apache.bookkeeper.client.api.LedgerEntries; import org.apache.bookkeeper.client.api.LedgerEntry; import org.apache.bookkeeper.client.api.ReadHandle; @@ -445,4 +448,55 @@ public void testInvalidEntryIds() throws Exception { } catch (Exception e) { } } + + @Test + public void testReadNotExistLedger() throws Exception { + LedgerOffloader offloader = getOffloader(new HashMap() {{ + put(TieredStorageConfiguration.MAX_OFFLOAD_SEGMENT_SIZE_IN_BYTES, "1000"); + put(config.getKeys(TieredStorageConfiguration.METADATA_FIELD_MAX_BLOCK_SIZE).get(0), "5242880"); + put(TieredStorageConfiguration.MAX_OFFLOAD_SEGMENT_ROLLOVER_TIME_SEC, "600"); + }}); + ManagedLedger ml = createMockManagedLedger(); + UUID uuid = UUID.randomUUID(); + long beginLedger = 0; + long beginEntry = 0; + + Map driverMeta = new HashMap() {{ + put(TieredStorageConfiguration.METADATA_FIELD_BUCKET, BUCKET); + }}; + OffloadHandle offloadHandle = offloader + .streamingOffload(ml, uuid, beginLedger, beginEntry, driverMeta).get(); + + // Segment should closed because size in bytes full + final LinkedList entries = new LinkedList<>(); + for (int i = 0; i < 10; i++) { + final byte[] data = new byte[100]; + random.nextBytes(data); + final EntryImpl entry = EntryImpl.create(0, i, data); + offloadHandle.offerEntry(entry); + entries.add(entry); + } + + final LedgerOffloader.OffloadResult offloadResult = offloadHandle.getOffloadResultAsync().get(); + assertEquals(offloadResult.endLedger, 0); + assertEquals(offloadResult.endEntry, 9); + final OffloadContext.Builder contextBuilder = OffloadContext.newBuilder(); + contextBuilder.addOffloadSegment( + MLDataFormats.OffloadSegment.newBuilder() + .setUidLsb(uuid.getLeastSignificantBits()) + .setUidMsb(uuid.getMostSignificantBits()) + .setComplete(true).setEndEntryId(9).build()); + + final ReadHandle readHandle = offloader.readOffloaded(0, contextBuilder.build(), driverMeta).get(); + + // delete blob(ledger) + blobStore.removeBlob(BUCKET, uuid.toString()); + + try { + readHandle.read(0, 9); + fail("Should be read fail"); + } catch (BKException e) { + assertEquals(e.getCode(), NoSuchLedgerExistsException); + } + } } diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java index ac87a8e424038..6a66c19f68aeb 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java @@ -18,6 +18,7 @@ */ package org.apache.bookkeeper.mledger.offload.jcloud.impl; +import static org.apache.bookkeeper.client.api.BKException.Code.NoSuchLedgerExistsException; import static org.mockito.AdditionalAnswers.delegatesTo; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; @@ -26,6 +27,7 @@ import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -591,4 +593,25 @@ public void testReadWithAClosedLedgerHandler() throws Exception { throw e; } } + + @Test + public void testReadNotExistLedger() throws Exception { + ReadHandle toWrite = buildReadHandle(DEFAULT_BLOCK_SIZE, 3); + LedgerOffloader offloader = getOffloader(); + + UUID uuid = UUID.randomUUID(); + offloader.offload(toWrite, uuid, new HashMap<>()).get(); + ReadHandle offloadRead = offloader.readOffloaded(toWrite.getId(), uuid, Collections.emptyMap()).get(); + assertEquals(offloadRead.getLastAddConfirmed(), toWrite.getLastAddConfirmed()); + + // delete blob(ledger) + blobStore.removeBlob(BUCKET, DataBlockUtils.dataBlockOffloadKey(toWrite.getId(), uuid)); + + try { + offloadRead.read(0, offloadRead.getLastAddConfirmed()); + fail("Should be read fail"); + } catch (BKException e) { + assertEquals(e.getCode(), NoSuchLedgerExistsException); + } + } } From 67422d9036052b09e2eb81d3a36d72983aad8880 Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Fri, 19 Apr 2024 20:35:36 +0530 Subject: [PATCH 434/699] Removed unused import --- .../apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java b/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java index 6897711730f9f..6b4719709a178 100644 --- a/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java +++ b/pulsar-io/kafka/src/test/java/org/apache/pulsar/io/kafka/source/KafkaAbstractSourceTest.java @@ -20,7 +20,6 @@ import com.google.common.collect.ImmutableMap; import java.time.Duration; -import java.util.Collection; import java.util.Collections; import java.util.Arrays; import java.lang.reflect.Field; From b7e0a19a6bc393b6581d4cfbac65d990a4ff7acd Mon Sep 17 00:00:00 2001 From: Mukesh Kumar <65598381+mukesh154@users.noreply.github.com> Date: Fri, 12 Apr 2024 12:28:56 +0530 Subject: [PATCH 435/699] [improve][broker] backlog quota exceed limit log replaced with `debug` (#22475) (cherry picked from commit 767f0b282a27581dfc16df9295ed01dd0df0a350) --- .../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 63136257a9bb3..d7df6c1e5f76e 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 @@ -2942,14 +2942,14 @@ public CompletableFuture checkBacklogQuotaExceeded(String producerName, Ba if ((retentionPolicy == BacklogQuota.RetentionPolicy.producer_request_hold || retentionPolicy == BacklogQuota.RetentionPolicy.producer_exception)) { if (backlogQuotaType == BacklogQuotaType.destination_storage && isSizeBacklogExceeded()) { - log.info("[{}] Size backlog quota exceeded. Cannot create producer [{}]", this.getName(), + log.debug("[{}] Size backlog quota exceeded. Cannot create producer [{}]", this.getName(), producerName); return FutureUtil.failedFuture(new TopicBacklogQuotaExceededException(retentionPolicy)); } if (backlogQuotaType == BacklogQuotaType.message_age) { return checkTimeBacklogExceeded().thenCompose(isExceeded -> { if (isExceeded) { - log.info("[{}] Time backlog quota exceeded. Cannot create producer [{}]", this.getName(), + log.debug("[{}] Time backlog quota exceeded. Cannot create producer [{}]", this.getName(), producerName); return FutureUtil.failedFuture(new TopicBacklogQuotaExceededException(retentionPolicy)); } else { From b336996a926fd9d6de997fc97886a0808f42bd55 Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Mon, 22 Apr 2024 09:46:29 +0530 Subject: [PATCH 436/699] Fix [improve] [proxy] Not close the socket if lookup failed caused by too many requests (apache#21216) --- .../apache/pulsar/proxy/server/DefaultLookupProxyHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DefaultLookupProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DefaultLookupProxyHandler.java index b62d988904f63..15163dd7f6063 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DefaultLookupProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DefaultLookupProxyHandler.java @@ -105,7 +105,7 @@ public void handleLookup(CommandLookupTopic lookup) { log.debug("Lookup Request ID {} from {} rejected - {}.", clientRequestId, clientAddress, throttlingErrorMessage); } - writeAndFlush(Commands.newLookupErrorResponse(ServerError.ServiceNotReady, + writeAndFlush(Commands.newLookupErrorResponse(ServerError.TooManyRequests, throttlingErrorMessage, clientRequestId)); } } From b195ef00a7bfe048353d1258b95259aeed0bbfea Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 4 Apr 2024 06:39:53 -0700 Subject: [PATCH 437/699] [improve][build] Upgrade Lombok to 1.18.32 for Java 22 support (#22425) (cherry picked from commit 5b6f91bc0f839c467bdc1af35c8eac7b14aa8822) (cherry picked from commit a9e815d464ca8d4620ba6d676d74831a730d533c) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 368a973c0dec9..568dc6490ef22 100644 --- a/pom.xml +++ b/pom.xml @@ -228,7 +228,7 @@ flexible messaging model and an intuitive client API. 0.9.1 2.1.0 3.24.2 - 1.18.30 + 1.18.32 1.3.2 2.3.1 1.2.0 From fcf433d7d01f1c9104d8fb83c169e7e5dbc184c0 Mon Sep 17 00:00:00 2001 From: hanmz Date: Wed, 10 Apr 2024 04:27:22 +0800 Subject: [PATCH 438/699] [fix][broker] Update topic partition failed when config maxNumPartitionsPerPartitionedTopic<0 (#22397) (cherry picked from commit fb5caeb2cd3353db0499e32e9ec79390741b809c) (cherry picked from commit 386f6f0bf73d01f7b23160e9e4642f4f7c01cac5) --- .../pulsar/broker/ServiceConfiguration.java | 3 +- .../admin/impl/PersistentTopicsBase.java | 2 +- .../broker/admin/PersistentTopicsTest.java | 45 +++++++++++++++++++ 3 files changed, 48 insertions(+), 2 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 b833223cb1e3e..da251eb0a537e 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 @@ -1351,7 +1351,8 @@ The delayed message index time step(in seconds) in per bucket snapshot segment, category = CATEGORY_SERVER, dynamic = true, doc = "The number of partitions per partitioned topic.\n" - + "If try to create or update partitioned topics by exceeded number of partitions, then fail." + + "If try to create or update partitioned topics by exceeded number of partitions, then fail.\n" + + "Use 0 or negative number to disable the check." ) private int maxNumPartitionsPerPartitionedTopic = 0; 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 a4a12e6a9138a..d931795789181 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 @@ -362,7 +362,7 @@ protected CompletableFuture internalCreateNonPartitionedTopicAsync(boolean } int brokerMaximumPartitionsPerTopic = pulsarService.getConfiguration() .getMaxNumPartitionsPerPartitionedTopic(); - if (brokerMaximumPartitionsPerTopic != 0 && expectPartitions > brokerMaximumPartitionsPerTopic) { + if (brokerMaximumPartitionsPerTopic > 0 && expectPartitions > brokerMaximumPartitionsPerTopic) { throw new RestException(422 /* Unprocessable entity*/, String.format("Expect partitions %s grater than maximum partitions per topic %s", expectPartitions, brokerMaximumPartitionsPerTopic)); 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 98a33274c94e0..cbffeca8e81f5 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 @@ -1736,6 +1736,51 @@ public void testUpdatePartitionedTopic() verify(response, timeout(5000).times(1)).resume(metaCaptor.capture()); partitionedTopicMetadata = metaCaptor.getValue(); Assert.assertEquals(partitionedTopicMetadata.partitions, 4); + + // test for configuration maxNumPartitionsPerPartitionedTopic + conf.setMaxNumPartitionsPerPartitionedTopic(4); + response = mock(AsyncResponse.class); + throwableCaptor = ArgumentCaptor.forClass(Throwable.class); + persistentTopics.updatePartitionedTopic(response, testTenant, testNamespaceLocal, topicName, false, true, + true, 5); + verify(response, timeout(5000).times(1)).resume(throwableCaptor.capture()); + Assert.assertEquals(throwableCaptor.getValue().getMessage(), + "Desired partitions 5 can't be greater than the maximum partitions per topic 4."); + + response = mock(AsyncResponse.class); + metaCaptor = ArgumentCaptor.forClass(PartitionedTopicMetadata.class); + persistentTopics.getPartitionedMetadata(response, testTenant, testNamespaceLocal, topicName, true, false); + verify(response, timeout(5000).times(1)).resume(metaCaptor.capture()); + partitionedTopicMetadata = metaCaptor.getValue(); + Assert.assertEquals(partitionedTopicMetadata.partitions, 4); + + conf.setMaxNumPartitionsPerPartitionedTopic(-1); + response = mock(AsyncResponse.class); + responseCaptor = ArgumentCaptor.forClass(Response.class); + persistentTopics.updatePartitionedTopic(response, testTenant, testNamespaceLocal, topicName, false, true, + true, 5); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + + response = mock(AsyncResponse.class); + metaCaptor = ArgumentCaptor.forClass(PartitionedTopicMetadata.class); + persistentTopics.getPartitionedMetadata(response, testTenant, testNamespaceLocal, topicName, true, false); + verify(response, timeout(5000).times(1)).resume(metaCaptor.capture()); + partitionedTopicMetadata = metaCaptor.getValue(); + Assert.assertEquals(partitionedTopicMetadata.partitions, 5); + + conf.setMaxNumPartitionsPerPartitionedTopic(0); + response = mock(AsyncResponse.class); + responseCaptor = ArgumentCaptor.forClass(Response.class); + persistentTopics.updatePartitionedTopic(response, testTenant, testNamespaceLocal, topicName, false, true, + true, 6); + verify(response, timeout(5000).times(1)).resume(responseCaptor.capture()); + + response = mock(AsyncResponse.class); + metaCaptor = ArgumentCaptor.forClass(PartitionedTopicMetadata.class); + persistentTopics.getPartitionedMetadata(response, testTenant, testNamespaceLocal, topicName, true, false); + verify(response, timeout(5000).times(1)).resume(metaCaptor.capture()); + partitionedTopicMetadata = metaCaptor.getValue(); + Assert.assertEquals(partitionedTopicMetadata.partitions, 6); } @Test From ef4deb13abf8863b45d60ad3a3512afdfc6f27d0 Mon Sep 17 00:00:00 2001 From: zhangqian <503837557@qq.com> Date: Wed, 10 Apr 2024 16:51:26 +0800 Subject: [PATCH 439/699] [fix][broker] Fix message drop record in producer stat (#22458) Co-authored-by: ceceezhang (cherry picked from commit cea1a9ba9b576bf43f0a45ff8d65369b0f2bbb36) (cherry picked from commit b41e7527158da8e12dcd491dca4698c4a74d07ba) --- .../main/java/org/apache/pulsar/broker/service/Producer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 fec2d52a239cc..3d8a01c42b519 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 @@ -752,7 +752,7 @@ public void updateRates() { } if (this.isNonPersistentTopic) { msgDrop.calculateRate(); - ((NonPersistentPublisherStatsImpl) stats).msgDropRate = msgDrop.getRate(); + ((NonPersistentPublisherStatsImpl) stats).msgDropRate = msgDrop.getValueRate(); } } From 19c7f8e86e07bbd44820efb831c360308f5dbcc4 Mon Sep 17 00:00:00 2001 From: sinan liu Date: Tue, 16 Apr 2024 21:19:44 +0800 Subject: [PATCH 440/699] [fix][test] SchemaMap in AutoConsumeSchema has been reused (#22500) (cherry picked from commit ffdfc0c4e0881c682132e79c3cbf9768b1ab4f89) (cherry picked from commit 42ae91ae31bc1ca27a13deb4b48a90ac06c89388) --- .../api/SimpleProducerConsumerTest.java | 66 +++++++++++-------- 1 file changed, 38 insertions(+), 28 deletions(-) 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 7552b84a1c553..691f501777eda 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 @@ -4329,6 +4329,10 @@ public static Object[] avroSchemaProvider() { public void testAccessAvroSchemaMetadata(Schema schema) throws Exception { log.info("-- Starting {} test --", methodName); + if (pulsarClient == null) { + pulsarClient = newPulsarClient(lookupUrl.toString(), 0); + } + final String topic = "persistent://my-property/my-ns/accessSchema"; Consumer consumer = pulsarClient.newConsumer(Schema.AUTO_CONSUME()) .topic(topic) @@ -4344,37 +4348,43 @@ public void testAccessAvroSchemaMetadata(Schema schema) throws Exception producer.send(payload); producer.close(); - GenericRecord res = consumer.receive(RECEIVE_TIMEOUT_SECONDS, TimeUnit.SECONDS).getValue(); - consumer.close(); - assertEquals(schema.getSchemaInfo().getType(), res.getSchemaType()); - org.apache.avro.generic.GenericRecord nativeAvroRecord = null; - JsonNode nativeJsonRecord = null; - if (schema.getSchemaInfo().getType() == SchemaType.AVRO) { - nativeAvroRecord = (org.apache.avro.generic.GenericRecord) res.getNativeObject(); - assertNotNull(nativeAvroRecord); - } else { - nativeJsonRecord = (JsonNode) res.getNativeObject(); - assertNotNull(nativeJsonRecord); - } - for (org.apache.pulsar.client.api.schema.Field f : res.getFields()) { - log.info("field {} {}", f.getName(), res.getField(f)); - assertEquals("field", f.getName()); - assertEquals("aaaaaaaaaaaaaaaaaaaaaaaaa", res.getField(f)); - - if (nativeAvroRecord != null) { - // test that the native schema is accessible - org.apache.avro.Schema.Field fieldDetails = nativeAvroRecord.getSchema().getField(f.getName()); - // a nullable string is an UNION - assertEquals(org.apache.avro.Schema.Type.UNION, fieldDetails.schema().getType()); - assertTrue(fieldDetails.schema().getTypes().stream().anyMatch(s -> s.getType() == org.apache.avro.Schema.Type.STRING)); - assertTrue(fieldDetails.schema().getTypes().stream().anyMatch(s -> s.getType() == org.apache.avro.Schema.Type.NULL)); + try { + GenericRecord res = consumer.receive(RECEIVE_TIMEOUT_SECONDS, TimeUnit.SECONDS).getValue(); + consumer.close(); + assertEquals(schema.getSchemaInfo().getType(), res.getSchemaType()); + org.apache.avro.generic.GenericRecord nativeAvroRecord = null; + JsonNode nativeJsonRecord = null; + if (schema.getSchemaInfo().getType() == SchemaType.AVRO) { + nativeAvroRecord = (org.apache.avro.generic.GenericRecord) res.getNativeObject(); + assertNotNull(nativeAvroRecord); } else { - assertEquals(JsonNodeType.STRING, nativeJsonRecord.get("field").getNodeType()); + nativeJsonRecord = (JsonNode) res.getNativeObject(); + assertNotNull(nativeJsonRecord); + } + for (org.apache.pulsar.client.api.schema.Field f : res.getFields()) { + log.info("field {} {}", f.getName(), res.getField(f)); + assertEquals("field", f.getName()); + assertEquals("aaaaaaaaaaaaaaaaaaaaaaaaa", res.getField(f)); + + if (nativeAvroRecord != null) { + // test that the native schema is accessible + org.apache.avro.Schema.Field fieldDetails = nativeAvroRecord.getSchema().getField(f.getName()); + // a nullable string is an UNION + assertEquals(org.apache.avro.Schema.Type.UNION, fieldDetails.schema().getType()); + assertTrue(fieldDetails.schema().getTypes().stream().anyMatch(s -> s.getType() == org.apache.avro.Schema.Type.STRING)); + assertTrue(fieldDetails.schema().getTypes().stream().anyMatch(s -> s.getType() == org.apache.avro.Schema.Type.NULL)); + } else { + assertEquals(JsonNodeType.STRING, nativeJsonRecord.get("field").getNodeType()); + } } + assertEquals(1, res.getFields().size()); + } catch (Exception e) { + fail(); + } finally { + pulsarClient.shutdown(); + pulsarClient = null; + admin.schemas().deleteSchema(topic); } - assertEquals(1, res.getFields().size()); - - admin.schemas().deleteSchema(topic); } @Test(timeOut = 100000) From d2b3eb7351ff9daf3b2764591486e6593957395e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Wed, 17 Apr 2024 03:07:30 +0800 Subject: [PATCH 441/699] [improve] Make the config `metricsBufferResponse` description more effective (#22490) (cherry picked from commit 4ca4e2855267e3b36ee1a27f7144b89ba9194821) (cherry picked from commit 94f12543a9ffe7e96f9af7ef9108d1b849ca3280) --- .../java/org/apache/pulsar/broker/ServiceConfiguration.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 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 da251eb0a537e..e11e2580aa585 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 @@ -2895,8 +2895,10 @@ The delayed message index time step(in seconds) in per bucket snapshot segment, private boolean exposeTopicLevelMetricsInPrometheus = true; @FieldContext( category = CATEGORY_METRICS, - doc = "If true, export buffered metrics" - ) + doc = "Set to true to enable the broker to cache the metrics response; the default is false. " + + "The caching period is defined by `managedLedgerStatsPeriodSeconds`. " + + "The broker returns the same response for subsequent requests within the same period. " + + "Ensure that the scrape interval of your monitoring system matches the caching period.") private boolean metricsBufferResponse = false; @FieldContext( category = CATEGORY_METRICS, From 00c2fbb4597538e2f713abfee6f20fff1406fa76 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Wed, 17 Apr 2024 03:12:34 +0800 Subject: [PATCH 442/699] [improve][broker] Add topic name to emitted error messages. (#22506) (cherry picked from commit d5b36da9a2e0d4f17bea8e033180e494e93dc442) (cherry picked from commit 1d202f93c8108eb8e293c70df9a932292329e822) --- .../pulsar/broker/service/AbstractTopic.java | 15 ++++++++------- .../apache/pulsar/broker/admin/AdminApi2Test.java | 6 ++++-- 2 files changed, 12 insertions(+), 9 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 d5d7e7ade5078..2f55587baedb1 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 @@ -863,7 +863,7 @@ protected CompletableFuture> incrementTopicEpochIfNeeded(Producer } } catch (Exception e) { - log.error("Encountered unexpected error during exclusive producer creation", e); + log.error("[{}] Encountered unexpected error during exclusive producer creation", topic, e); return FutureUtil.failedFuture(new BrokerServiceException(e)); } finally { lock.writeLock().unlock(); @@ -959,14 +959,14 @@ protected void checkTopicFenced() throws BrokerServiceException { protected CompletableFuture internalAddProducer(Producer producer) { if (isProducersExceeded(producer)) { log.warn("[{}] Attempting to add producer to topic which reached max producers limit", topic); - return CompletableFuture.failedFuture( - new BrokerServiceException.ProducerBusyException("Topic reached max producers limit")); + return CompletableFuture.failedFuture(new BrokerServiceException.ProducerBusyException( + "Topic '" + topic + "' reached max producers limit")); } if (isSameAddressProducersExceeded(producer)) { log.warn("[{}] Attempting to add producer to topic which reached max same address producers limit", topic); - return CompletableFuture.failedFuture( - new BrokerServiceException.ProducerBusyException("Topic reached max same address producers limit")); + return CompletableFuture.failedFuture(new BrokerServiceException.ProducerBusyException( + "Topic '" + topic + "' reached max same address producers limit")); } if (log.isDebugEnabled()) { @@ -1001,7 +1001,7 @@ private CompletableFuture tryOverwriteOldProducer(Producer oldProducer, Pr if (previousIsActive) { return CompletableFuture.failedFuture(new BrokerServiceException.NamingException( "Producer with name '" + newProducer.getProducerName() - + "' is already connected to topic")); + + "' is already connected to topic '" + topic + "'")); } else { // If the connection of the previous producer is not active, the method // "cnx().checkConnectionLiveness()" will trigger the close for it and kick off the previous @@ -1014,7 +1014,8 @@ private CompletableFuture tryOverwriteOldProducer(Producer oldProducer, Pr }); } return CompletableFuture.failedFuture(new BrokerServiceException.NamingException( - "Producer with name '" + newProducer.getProducerName() + "' is already connected to topic")); + "Producer with name '" + newProducer.getProducerName() + "' is already connected to topic '" + + topic + "'")); } } 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 7cf84673be86a..30e4875c170be 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 @@ -2886,7 +2886,8 @@ public void testMaxProducersPerTopicUnlimited() throws Exception { Producer producer = pulsarClient.newProducer().topic(topic).create(); fail("should fail"); } catch (PulsarClientException e) { - assertTrue(e.getMessage().contains("Topic reached max producers limit")); + String expectMsg = "Topic '" + topic + "' reached max producers limit"; + assertTrue(e.getMessage().contains(expectMsg)); } //set the limit to 3 admin.namespaces().setMaxProducersPerTopic(myNamespace, 3); @@ -2900,7 +2901,8 @@ public void testMaxProducersPerTopicUnlimited() throws Exception { Producer producer1 = pulsarClient.newProducer().topic(topic).create(); fail("should fail"); } catch (PulsarClientException e) { - assertTrue(e.getMessage().contains("Topic reached max producers limit")); + String expectMsg = "Topic '" + topic + "' reached max producers limit"; + assertTrue(e.getMessage().contains(expectMsg)); } //clean up From 58029496b2badf3e976adbd8e58adfbe10fc6908 Mon Sep 17 00:00:00 2001 From: hanmz Date: Wed, 17 Apr 2024 18:14:38 +0800 Subject: [PATCH 443/699] [improve][broker] Repeat the handleMetadataChanges callback when configurationMetadataStore equals localMetadataStore (#22519) (cherry picked from commit 1dd82a0affd6ec3686fa85d444c35bbbb4e9ce12) (cherry picked from commit 8f5b825b7a979acf80ad88ad2a939357d3f1a970) --- .../java/org/apache/pulsar/broker/service/BrokerService.java | 4 +++- 1 file changed, 3 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 442f14e36576d..97742e25af255 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 @@ -336,7 +336,9 @@ public BrokerService(PulsarService pulsar, EventLoopGroup eventLoopGroup) throws this.entryFilterProvider = new EntryFilterProvider(pulsar.getConfiguration()); pulsar.getLocalMetadataStore().registerListener(this::handleMetadataChanges); - pulsar.getConfigurationMetadataStore().registerListener(this::handleMetadataChanges); + if (pulsar.getConfigurationMetadataStore() != pulsar.getLocalMetadataStore()) { + pulsar.getConfigurationMetadataStore().registerListener(this::handleMetadataChanges); + } this.inactivityMonitor = OrderedScheduler.newSchedulerBuilder() From a207029aedba6569b0357314d35ed8f600c1d0de Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 17 Apr 2024 12:46:43 -0700 Subject: [PATCH 444/699] [improve][test] Move ShadowManagedLedgerImplTest to flaky tests (#22526) (cherry picked from commit 56970b714f5adb606b02d12a99db1ceec3fa7832) (cherry picked from commit 155f3cbf9f63993a4b6e84b107d17b556927b7c7) --- .../bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java index cc4b3f2481152..2aa04197ab91e 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ShadowManagedLedgerImplTest.java @@ -51,7 +51,7 @@ private ShadowManagedLedgerImpl openShadowManagedLedger(String name, String sour return (ShadowManagedLedgerImpl) shadowML; } - @Test + @Test(groups = "flaky") public void testShadowWrites() throws Exception { ManagedLedgerImpl sourceML = (ManagedLedgerImpl) factory.open("source_ML", new ManagedLedgerConfig() .setMaxEntriesPerLedger(2) From 973fdceb34b09712be9c1e16a7669815f8b41755 Mon Sep 17 00:00:00 2001 From: hanmz Date: Fri, 19 Apr 2024 06:49:18 +0800 Subject: [PATCH 445/699] [fix][broker] Fix typos in Consumer class (#22532) (cherry picked from commit 7aedb6b20c120ec0a7cc096e33e6305caca26786) (cherry picked from commit beb147c701728f9a2c0fcc9b56d4efb53f94a9bc) --- .../java/org/apache/pulsar/broker/service/Consumer.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 9485931304b0c..9406a30dd2f01 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 @@ -148,7 +148,7 @@ public class Consumer { @Setter private volatile long consumerEpoch; - private long negtiveUnackedMsgsTimestamp; + private long negativeUnackedMsgsTimestamp; @Getter private final SchemaType schemaType; @@ -1098,8 +1098,8 @@ private int addAndGetUnAckedMsgs(Consumer consumer, int ackedMessages) { subscription.addUnAckedMessages(ackedMessages); unackedMsgs = UNACKED_MESSAGES_UPDATER.addAndGet(consumer, ackedMessages); } - if (unackedMsgs < 0 && System.currentTimeMillis() - negtiveUnackedMsgsTimestamp >= 10_000) { - negtiveUnackedMsgsTimestamp = System.currentTimeMillis(); + if (unackedMsgs < 0 && System.currentTimeMillis() - negativeUnackedMsgsTimestamp >= 10_000) { + negativeUnackedMsgsTimestamp = System.currentTimeMillis(); log.warn("unackedMsgs is : {}, ackedMessages : {}, consumer : {}", unackedMsgs, ackedMessages, consumer); } return unackedMsgs; From 0904a475d47f34a1a75e14ce4f609b03f6b26c1a Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Fri, 19 Apr 2024 10:30:55 -0700 Subject: [PATCH 446/699] [fix][broker] Fix broken topic policy implementation compatibility with old pulsar version (#22535) (cherry picked from commit 59daac64c210f539e733f883edad09d08333aa62) (cherry picked from commit 8439082f79c5480b58be93fb360ed07b68016631) --- .../pulsar/broker/service/AbstractTopic.java | 52 +++++++++++-------- ...ternalClientConfigurationOverrideTest.java | 42 ++++++++++++++- 2 files changed, 72 insertions(+), 22 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 2f55587baedb1..ff6fa1100607a 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 @@ -216,13 +216,16 @@ protected void updateTopicPolicy(TopicPolicies data) { .updateTopicValue(formatSchemaCompatibilityStrategy(data.getSchemaCompatibilityStrategy())); } topicPolicies.getRetentionPolicies().updateTopicValue(data.getRetentionPolicies()); - topicPolicies.getMaxSubscriptionsPerTopic().updateTopicValue(data.getMaxSubscriptionsPerTopic()); - topicPolicies.getMaxUnackedMessagesOnConsumer().updateTopicValue(data.getMaxUnackedMessagesOnConsumer()); + topicPolicies.getMaxSubscriptionsPerTopic() + .updateTopicValue(normalizeValue(data.getMaxSubscriptionsPerTopic())); + topicPolicies.getMaxUnackedMessagesOnConsumer() + .updateTopicValue(normalizeValue(data.getMaxUnackedMessagesOnConsumer())); topicPolicies.getMaxUnackedMessagesOnSubscription() - .updateTopicValue(data.getMaxUnackedMessagesOnSubscription()); - topicPolicies.getMaxProducersPerTopic().updateTopicValue(data.getMaxProducerPerTopic()); - topicPolicies.getMaxConsumerPerTopic().updateTopicValue(data.getMaxConsumerPerTopic()); - topicPolicies.getMaxConsumersPerSubscription().updateTopicValue(data.getMaxConsumersPerSubscription()); + .updateTopicValue(normalizeValue(data.getMaxUnackedMessagesOnSubscription())); + topicPolicies.getMaxProducersPerTopic().updateTopicValue(normalizeValue(data.getMaxProducerPerTopic())); + topicPolicies.getMaxConsumerPerTopic().updateTopicValue(normalizeValue(data.getMaxConsumerPerTopic())); + topicPolicies.getMaxConsumersPerSubscription() + .updateTopicValue(normalizeValue(data.getMaxConsumersPerSubscription())); topicPolicies.getInactiveTopicPolicies().updateTopicValue(data.getInactiveTopicPolicies()); topicPolicies.getDeduplicationEnabled().updateTopicValue(data.getDeduplicationEnabled()); topicPolicies.getDeduplicationSnapshotIntervalSeconds().updateTopicValue( @@ -233,8 +236,8 @@ protected void updateTopicPolicy(TopicPolicies data) { Arrays.stream(BacklogQuota.BacklogQuotaType.values()).forEach(type -> this.topicPolicies.getBackLogQuotaMap().get(type).updateTopicValue( data.getBackLogQuotaMap() == null ? null : data.getBackLogQuotaMap().get(type.toString()))); - topicPolicies.getTopicMaxMessageSize().updateTopicValue(data.getMaxMessageSize()); - topicPolicies.getMessageTTLInSeconds().updateTopicValue(data.getMessageTTLInSeconds()); + topicPolicies.getTopicMaxMessageSize().updateTopicValue(normalizeValue(data.getMaxMessageSize())); + topicPolicies.getMessageTTLInSeconds().updateTopicValue(normalizeValue(data.getMessageTTLInSeconds())); topicPolicies.getPublishRate().updateTopicValue(PublishRate.normalize(data.getPublishRate())); topicPolicies.getDelayedDeliveryEnabled().updateTopicValue(data.getDelayedDeliveryEnabled()); topicPolicies.getReplicatorDispatchRate().updateTopicValue( @@ -261,15 +264,19 @@ protected void updateTopicPolicyByNamespacePolicy(Policies namespacePolicies) { topicPolicies.getReplicationClusters().updateNamespaceValue( new ArrayList<>(CollectionUtils.emptyIfNull(namespacePolicies.replication_clusters))); topicPolicies.getMaxUnackedMessagesOnConsumer() - .updateNamespaceValue(namespacePolicies.max_unacked_messages_per_consumer); + .updateNamespaceValue(normalizeValue(namespacePolicies.max_unacked_messages_per_consumer)); topicPolicies.getMaxUnackedMessagesOnSubscription() - .updateNamespaceValue(namespacePolicies.max_unacked_messages_per_subscription); - topicPolicies.getMessageTTLInSeconds().updateNamespaceValue(namespacePolicies.message_ttl_in_seconds); - topicPolicies.getMaxSubscriptionsPerTopic().updateNamespaceValue(namespacePolicies.max_subscriptions_per_topic); - topicPolicies.getMaxProducersPerTopic().updateNamespaceValue(namespacePolicies.max_producers_per_topic); - topicPolicies.getMaxConsumerPerTopic().updateNamespaceValue(namespacePolicies.max_consumers_per_topic); + .updateNamespaceValue(normalizeValue(namespacePolicies.max_unacked_messages_per_subscription)); + topicPolicies.getMessageTTLInSeconds() + .updateNamespaceValue(normalizeValue(namespacePolicies.message_ttl_in_seconds)); + topicPolicies.getMaxSubscriptionsPerTopic() + .updateNamespaceValue(normalizeValue(namespacePolicies.max_subscriptions_per_topic)); + topicPolicies.getMaxProducersPerTopic() + .updateNamespaceValue(normalizeValue(namespacePolicies.max_producers_per_topic)); + topicPolicies.getMaxConsumerPerTopic() + .updateNamespaceValue(normalizeValue(namespacePolicies.max_consumers_per_topic)); topicPolicies.getMaxConsumersPerSubscription() - .updateNamespaceValue(namespacePolicies.max_consumers_per_subscription); + .updateNamespaceValue(normalizeValue(namespacePolicies.max_consumers_per_subscription)); topicPolicies.getInactiveTopicPolicies().updateNamespaceValue(namespacePolicies.inactive_topic_policies); topicPolicies.getDeduplicationEnabled().updateNamespaceValue(namespacePolicies.deduplicationEnabled); topicPolicies.getDeduplicationSnapshotIntervalSeconds().updateNamespaceValue( @@ -299,6 +306,10 @@ protected void updateTopicPolicyByNamespacePolicy(Policies namespacePolicies) { updateEntryFilters(); } + private Integer normalizeValue(Integer policyValue) { + return policyValue != null && policyValue < 0 ? null : policyValue; + } + private void updateNamespaceDispatchRate(Policies namespacePolicies, String cluster) { DispatchRateImpl dispatchRate = namespacePolicies.topicDispatchRate.get(cluster); if (dispatchRate == null) { @@ -357,12 +368,11 @@ private void updateTopicPolicyByBrokerConfig() { topicPolicies.getMaxConsumerPerTopic().updateBrokerValue(config.getMaxConsumersPerTopic()); topicPolicies.getMaxConsumersPerSubscription().updateBrokerValue(config.getMaxConsumersPerSubscription()); topicPolicies.getDeduplicationEnabled().updateBrokerValue(config.isBrokerDeduplicationEnabled()); - topicPolicies.getRetentionPolicies().updateBrokerValue(new RetentionPolicies( - config.getDefaultRetentionTimeInMinutes(), config.getDefaultRetentionSizeInMB())); - topicPolicies.getDeduplicationSnapshotIntervalSeconds().updateBrokerValue( - config.getBrokerDeduplicationSnapshotIntervalSeconds()); - topicPolicies.getMaxUnackedMessagesOnConsumer() - .updateBrokerValue(config.getMaxUnackedMessagesPerConsumer()); + topicPolicies.getRetentionPolicies().updateBrokerValue( + new RetentionPolicies(config.getDefaultRetentionTimeInMinutes(), config.getDefaultRetentionSizeInMB())); + topicPolicies.getDeduplicationSnapshotIntervalSeconds() + .updateBrokerValue(config.getBrokerDeduplicationSnapshotIntervalSeconds()); + topicPolicies.getMaxUnackedMessagesOnConsumer().updateBrokerValue(config.getMaxUnackedMessagesPerConsumer()); topicPolicies.getMaxUnackedMessagesOnSubscription() .updateBrokerValue(config.getMaxUnackedMessagesPerSubscription()); //init backlogQuota diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerInternalClientConfigurationOverrideTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerInternalClientConfigurationOverrideTest.java index 1b1b383e930e3..f33202c3c4033 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerInternalClientConfigurationOverrideTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerInternalClientConfigurationOverrideTest.java @@ -18,17 +18,21 @@ */ package org.apache.pulsar.broker.service; +import static org.testng.Assert.assertEquals; import org.apache.pulsar.broker.PulsarServerException; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.internal.PulsarAdminImpl; +import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.ClusterDataImpl; +import org.apache.pulsar.common.policies.data.Policies; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; - +import lombok.Cleanup; import java.util.Optional; import java.util.Properties; @@ -112,4 +116,40 @@ public void testNamespaceServicePulsarClientConfiguration() { Assert.assertEquals(clientConf.getMemoryLimitBytes(), 100000); } + @Test + public void testOldNamespacePolicy() throws Exception { + + String ns = "prop/oldNsWithDefaultNonNullValues"; + String topic = "persistent://" + ns + "/t1"; + Policies policies = new Policies(); + policies.max_consumers_per_subscription = -1; + policies.max_consumers_per_topic = -1; + policies.max_producers_per_topic = -1; + policies.max_subscriptions_per_topic = -1; + policies.max_topics_per_namespace = -1; + policies.max_unacked_messages_per_consumer = -1; + policies.max_unacked_messages_per_subscription = -1; + admin.namespaces().createNamespace(ns, policies); + + @Cleanup + Producer producer = pulsarClient.newProducer() + .topic(topic).create(); + PersistentTopic topicRef = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topic).get(); + assertEquals(topicRef.topicPolicies.getMaxUnackedMessagesOnSubscription().get(), + conf.getMaxUnackedMessagesPerSubscription()); + assertEquals(topicRef.topicPolicies.getMaxConsumersPerSubscription().get(), + conf.getMaxConsumersPerSubscription()); + assertEquals(topicRef.topicPolicies.getMaxConsumerPerTopic().get(), + conf.getMaxConsumersPerTopic()); + assertEquals(topicRef.topicPolicies.getMaxProducersPerTopic().get(), + conf.getMaxProducersPerTopic()); + assertEquals(topicRef.topicPolicies.getMaxSubscriptionsPerTopic().get(), + conf.getMaxSubscriptionsPerTopic()); + assertEquals(topicRef.topicPolicies.getTopicMaxMessageSize().get(), + conf.getMaxMessageSize()); + assertEquals(topicRef.topicPolicies.getMaxUnackedMessagesOnConsumer().get(), + conf.getMaxUnackedMessagesPerConsumer()); + + + } } From 287e456b0c1fa4d3b53040613bb01d5db8268a0c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 22 Apr 2024 07:49:34 +0300 Subject: [PATCH 447/699] [improve][broker] Support X-Forwarded-For and HA Proxy Protocol for resolving original client IP of http/https requests (#22524) (cherry picked from commit 4a887217d835629cafb393ddf331441b484d4e2c) (cherry picked from commit 7d52dd7133106a7c39459af15e5ae395218cf6eb) --- conf/broker.conf | 10 + conf/functions_worker.yml | 10 + conf/proxy.conf | 10 + conf/standalone.conf | 10 + conf/websocket.conf | 10 + pom.xml | 1 + .../pulsar/broker/ServiceConfiguration.java | 16 ++ .../broker/web/JettyRequestLogFactory.java | 195 +++++++++++++++++- pulsar-broker/pom.xml | 7 + .../apache/pulsar/broker/web/WebService.java | 37 +++- .../web/WebServiceOriginalClientIPTest.java | 155 ++++++++++++++ pulsar-broker/src/test/resources/log4j2.xml | 40 ++++ .../pulsar/functions/worker/WorkerConfig.java | 16 ++ .../functions/worker/rest/WorkerServer.java | 38 +++- pulsar-proxy/pom.xml | 6 + .../proxy/server/ProxyConfiguration.java | 16 ++ .../proxy/server/ProxyServiceStarter.java | 31 ++- .../apache/pulsar/proxy/server/WebServer.java | 34 ++- .../server/ProxyOriginalClientIPTest.java | 157 ++++++++++++++ ...roxyServiceStarterDisableZeroCopyTest.java | 2 +- .../proxy/server/ProxyServiceStarterTest.java | 2 +- .../server/ProxyServiceTlsStarterTest.java | 2 +- pulsar-proxy/src/test/resources/log4j2.xml | 36 ++++ .../pulsar/websocket/service/ProxyServer.java | 39 +++- .../service/WebSocketProxyConfiguration.java | 14 ++ 25 files changed, 873 insertions(+), 21 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceOriginalClientIPTest.java create mode 100644 pulsar-broker/src/test/resources/log4j2.xml create mode 100644 pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyOriginalClientIPTest.java create mode 100644 pulsar-proxy/src/test/resources/log4j2.xml diff --git a/conf/broker.conf b/conf/broker.conf index 4c57fe4c73d8a..ac6da587b016d 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -88,6 +88,16 @@ advertisedAddress= # If true, the real IP addresses of consumers and producers can be obtained when getting topic statistics data. haProxyProtocolEnabled=false +# Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. +webServiceHaProxyProtocolEnabled=false + +# Trust X-Forwarded-For header for resolving the client IP for http/https requests. Default is false. +webServiceTrustXForwardedFor=false + +# Add detailed client/remote and server/local addresses and ports to http/https request logging. +# Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor is enabled. +webServiceLogDetailedAddresses= + # Number of threads to config Netty Acceptor. Default is 1 numAcceptorThreads= diff --git a/conf/functions_worker.yml b/conf/functions_worker.yml index 3871c74a88778..6f995576ebd64 100644 --- a/conf/functions_worker.yml +++ b/conf/functions_worker.yml @@ -27,6 +27,16 @@ workerHostname: localhost workerPort: 6750 workerPortTls: 6751 +# Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. +webServiceHaProxyProtocolEnabled: false + +# Trust X-Forwarded-For header for resolving the client IP for http/https requests. Default is false. +webServiceTrustXForwardedFor: false + +# Add detailed client/remote and server/local addresses and ports to http/https request logging. +# Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor is enabled. +webServiceLogDetailedAddresses: null + # The Configuration metadata store url # Examples: # * zk:my-zk-1:2181,my-zk-2:2181,my-zk-3:2181 diff --git a/conf/proxy.conf b/conf/proxy.conf index 5a9d433f39ceb..6e6c960e8009e 100644 --- a/conf/proxy.conf +++ b/conf/proxy.conf @@ -63,6 +63,16 @@ advertisedAddress= # If true, the real IP addresses of consumers and producers can be obtained when getting topic statistics data. haProxyProtocolEnabled=false +# Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. +webServiceHaProxyProtocolEnabled=false + +# Trust X-Forwarded-For header for resolving the client IP for http/https requests. Default is false. +webServiceTrustXForwardedFor=false + +# Add detailed client/remote and server/local addresses and ports to http/https request logging. +# Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor is enabled. +webServiceLogDetailedAddresses= + # 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 diff --git a/conf/standalone.conf b/conf/standalone.conf index 707ad06a7b01a..c43c3419fef92 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -51,6 +51,16 @@ advertisedAddress= # If true, the real IP addresses of consumers and producers can be obtained when getting topic statistics data. haProxyProtocolEnabled=false +# Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. +webServiceHaProxyProtocolEnabled=false + +# Trust X-Forwarded-For header for resolving the client IP for http/https requests. Default is false. +webServiceTrustXForwardedFor=false + +# Add detailed client/remote and server/local addresses and ports to http/https request logging. +# Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor is enabled. +webServiceLogDetailedAddresses= + # Number of threads to use for Netty IO. Default is set to 2 * Runtime.getRuntime().availableProcessors() numIOThreads= diff --git a/conf/websocket.conf b/conf/websocket.conf index 85023d8be217a..33e4cf455e022 100644 --- a/conf/websocket.conf +++ b/conf/websocket.conf @@ -46,6 +46,16 @@ statusFilePath= # Hostname or IP address the service binds on, default is 0.0.0.0. bindAddress=0.0.0.0 +# Enable or disable the use of HA proxy protocol for resolving the client IP for http/https requests. +webServiceHaProxyProtocolEnabled=false + +# Trust X-Forwarded-For header for resolving the client IP for http/https requests. Default is false. +webServiceTrustXForwardedFor=false + +# Add detailed client/remote and server/local addresses and ports to http/https request logging. +# Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor is enabled. +webServiceLogDetailedAddresses= + # Name of the pulsar cluster to connect to clusterName= diff --git a/pom.xml b/pom.xml index 568dc6490ef22..666ffa0fb73ad 100644 --- a/pom.xml +++ b/pom.xml @@ -272,6 +272,7 @@ flexible messaging model and an intuitive client API. 1.5.4 5.4.0 2.33.2 + 1.0.3 0.6.1 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 e11e2580aa585..3322adb3253a3 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 @@ -250,6 +250,22 @@ public class ServiceConfiguration implements PulsarConfiguration { + " when getting topic statistics data.") private boolean haProxyProtocolEnabled; + @FieldContext(category = CATEGORY_SERVER, + doc = "Enable or disable the use of HA proxy protocol for resolving the client IP for http/https " + + "requests. Default is false.") + private boolean webServiceHaProxyProtocolEnabled = false; + + @FieldContext(category = CATEGORY_SERVER, doc = + "Trust X-Forwarded-For header for resolving the client IP for http/https requests.\n" + + "Default is false.") + private boolean webServiceTrustXForwardedFor = false; + + @FieldContext(category = CATEGORY_SERVER, doc = + "Add detailed client/remote and server/local addresses and ports to http/https request logging.\n" + + "Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor " + + "is enabled.") + private Boolean webServiceLogDetailedAddresses; + @FieldContext( category = CATEGORY_SERVER, doc = "Number of threads to use for Netty Acceptor." diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/JettyRequestLogFactory.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/JettyRequestLogFactory.java index e5daa5852b51f..fc88647eb49ea 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/JettyRequestLogFactory.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/JettyRequestLogFactory.java @@ -18,9 +18,23 @@ */ package org.apache.pulsar.broker.web; +import java.net.InetSocketAddress; import java.util.TimeZone; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.extern.slf4j.Slf4j; +import org.eclipse.jetty.io.Connection; +import org.eclipse.jetty.io.EndPoint; +import org.eclipse.jetty.server.Connector; import org.eclipse.jetty.server.CustomRequestLog; +import org.eclipse.jetty.server.ProxyConnectionFactory; +import org.eclipse.jetty.server.Request; +import org.eclipse.jetty.server.RequestLog; +import org.eclipse.jetty.server.Response; +import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.Slf4jRequestLogWriter; +import org.eclipse.jetty.util.HostPort; +import org.eclipse.jetty.util.component.ContainerLifeCycle; /** * Class to standardize initialization of a Jetty request logger for all pulsar components. @@ -58,7 +72,184 @@ public class JettyRequestLogFactory { * Build a new Jetty request logger using the format defined in this class. * @return a request logger */ - public static CustomRequestLog createRequestLogger() { - return new CustomRequestLog(new Slf4jRequestLogWriter(), LOG_FORMAT); + public static RequestLog createRequestLogger() { + return createRequestLogger(false, null); + } + + /** + * Build a new Jetty request logger using the format defined in this class. + * @param showDetailedAddresses whether to show detailed addresses and ports in logs + * @return a request logger + */ + public static RequestLog createRequestLogger(boolean showDetailedAddresses, Server server) { + if (!showDetailedAddresses) { + return new CustomRequestLog(new Slf4jRequestLogWriter(), LOG_FORMAT); + } else { + return new OriginalClientIPRequestLog(server); + } + } + + /** + * Logs the original and real remote (client) and local (server) IP addresses + * when detailed addresses are enabled. + * Tracks the real addresses of remote and local using a registered Connection.Listener + * when detailed addresses are enabled. + * This is necessary when Proxy Protocol is used to pass the original client IP. + */ + @Slf4j + private static class OriginalClientIPRequestLog extends ContainerLifeCycle implements RequestLog { + private final ThreadLocal requestLogStringBuilder = ThreadLocal.withInitial(StringBuilder::new); + private final CustomRequestLog delegate; + private final Slf4jRequestLogWriter delegateLogWriter; + + OriginalClientIPRequestLog(Server server) { + delegate = new CustomRequestLog(this::write, LOG_FORMAT); + addBean(delegate); + delegateLogWriter = new Slf4jRequestLogWriter(); + addBean(delegateLogWriter); + if (server != null) { + for (Connector connector : server.getConnectors()) { + // adding the listener is only necessary for connectors that use ProxyConnectionFactory + if (connector.getDefaultConnectionFactory() instanceof ProxyConnectionFactory) { + connector.addBean(proxyProtocolOriginalEndpointListener); + } + } + } + } + + void write(String requestEntry) { + StringBuilder sb = requestLogStringBuilder.get(); + sb.setLength(0); + sb.append(requestEntry); + } + + @Override + public void log(Request request, Response response) { + delegate.log(request, response); + StringBuilder sb = requestLogStringBuilder.get(); + sb.append(" [R:"); + sb.append(request.getRemoteHost()); + sb.append(':'); + sb.append(request.getRemotePort()); + InetSocketAddress realRemoteAddress = lookupRealAddress(request.getHttpChannel().getRemoteAddress()); + if (realRemoteAddress != null) { + String realRemoteHost = HostPort.normalizeHost(realRemoteAddress.getHostString()); + int realRemotePort = realRemoteAddress.getPort(); + if (!realRemoteHost.equals(request.getRemoteHost()) || realRemotePort != request.getRemotePort()) { + sb.append(" via "); + sb.append(realRemoteHost); + sb.append(':'); + sb.append(realRemotePort); + } + } + sb.append("]->[L:"); + InetSocketAddress realLocalAddress = lookupRealAddress(request.getHttpChannel().getLocalAddress()); + if (realLocalAddress != null) { + String realLocalHost = HostPort.normalizeHost(realLocalAddress.getHostString()); + int realLocalPort = realLocalAddress.getPort(); + sb.append(realLocalHost); + sb.append(':'); + sb.append(realLocalPort); + if (!realLocalHost.equals(request.getLocalAddr()) || realLocalPort != request.getLocalPort()) { + sb.append(" dst "); + sb.append(request.getLocalAddr()); + sb.append(':'); + sb.append(request.getLocalPort()); + } + } else { + sb.append(request.getLocalAddr()); + sb.append(':'); + sb.append(request.getLocalPort()); + } + sb.append(']'); + try { + delegateLogWriter.write(sb.toString()); + } catch (Exception e) { + log.warn("Failed to write request log", e); + } + } + + private InetSocketAddress lookupRealAddress(InetSocketAddress socketAddress) { + if (socketAddress == null) { + return null; + } + if (proxyProtocolRealAddressMapping.isEmpty()) { + return socketAddress; + } + AddressEntry entry = proxyProtocolRealAddressMapping.get(new AddressKey(socketAddress.getHostString(), + socketAddress.getPort())); + if (entry != null) { + return entry.realAddress; + } else { + return socketAddress; + } + } + + private final Connection.Listener proxyProtocolOriginalEndpointListener = + new ProxyProtocolOriginalEndpointListener(); + + private final ConcurrentHashMap proxyProtocolRealAddressMapping = + new ConcurrentHashMap<>(); + + // Use a record as key since InetSocketAddress hash code changes if the address gets resolved + record AddressKey(String hostString, int port) { + + } + + record AddressEntry(InetSocketAddress realAddress, AtomicInteger referenceCount) { + + } + + // Tracks the real addresses of remote and local when detailed addresses are enabled. + // This is necessary when Proxy Protocol is used to pass the original client IP. + // The Proxy Protocol implementation in Jetty wraps the original endpoint with a ProxyEndPoint + // and the real endpoint information isn't available in the request object. + // This listener is added to all connectors to track the real addresses of the client and server. + class ProxyProtocolOriginalEndpointListener implements Connection.Listener { + @Override + public void onOpened(Connection connection) { + handleConnection(connection, true); + } + + @Override + public void onClosed(Connection connection) { + handleConnection(connection, false); + } + + private void handleConnection(Connection connection, boolean increment) { + if (connection.getEndPoint() instanceof ProxyConnectionFactory.ProxyEndPoint) { + ProxyConnectionFactory.ProxyEndPoint proxyEndPoint = + (ProxyConnectionFactory.ProxyEndPoint) connection.getEndPoint(); + EndPoint originalEndpoint = proxyEndPoint.unwrap(); + mapAddress(proxyEndPoint.getLocalAddress(), originalEndpoint.getLocalAddress(), increment); + mapAddress(proxyEndPoint.getRemoteAddress(), originalEndpoint.getRemoteAddress(), increment); + } + } + + private void mapAddress(InetSocketAddress current, InetSocketAddress real, boolean increment) { + // don't add the mapping if the current address is the same as the real address + if (real != null && current != null && current.equals(real)) { + return; + } + AddressKey key = new AddressKey(current.getHostString(), current.getPort()); + proxyProtocolRealAddressMapping.compute(key, (__, entry) -> { + if (entry == null) { + if (increment) { + entry = new AddressEntry(real, new AtomicInteger(1)); + } + } else { + if (increment) { + entry.referenceCount.incrementAndGet(); + } else { + if (entry.referenceCount.decrementAndGet() == 0) { + // remove the entry if the reference count drops to 0 + entry = null; + } + } + } + return entry; + }); + } + } } } diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index 3df9655e93260..b2fb4101aa68e 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -158,6 +158,13 @@ test + + io.github.hakky54 + consolecaptor + ${consolecaptor.version} + test + + io.dropwizard.metrics 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 7395144167af9..4009401971c33 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 @@ -31,12 +31,18 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.jetty.tls.JettySslContextFactory; +import org.eclipse.jetty.server.ConnectionFactory; import org.eclipse.jetty.server.ConnectionLimit; +import org.eclipse.jetty.server.ForwardedRequestCustomizer; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.HttpConfiguration; import org.eclipse.jetty.server.HttpConnectionFactory; +import org.eclipse.jetty.server.ProxyConnectionFactory; +import org.eclipse.jetty.server.RequestLog; +import org.eclipse.jetty.server.SecureRequestCustomizer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.server.SslConnectionFactory; import org.eclipse.jetty.server.handler.ContextHandler; import org.eclipse.jetty.server.handler.ContextHandlerCollection; import org.eclipse.jetty.server.handler.DefaultHandler; @@ -103,9 +109,18 @@ public WebService(PulsarService pulsar) throws PulsarServerException { Optional port = config.getWebServicePort(); HttpConfiguration httpConfig = new HttpConfiguration(); + if (config.isWebServiceTrustXForwardedFor()) { + httpConfig.addCustomizer(new ForwardedRequestCustomizer()); + } httpConfig.setRequestHeaderSize(pulsar.getConfig().getHttpMaxRequestHeaderSize()); + HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory(httpConfig); if (port.isPresent()) { - httpConnector = new ServerConnector(server, new HttpConnectionFactory(httpConfig)); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(httpConnectionFactory); + httpConnector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); httpConnector.setPort(port.get()); httpConnector.setHost(pulsar.getBindAddress()); connectors.add(httpConnector); @@ -144,7 +159,18 @@ public WebService(PulsarService pulsar) throws PulsarServerException { config.getWebServiceTlsProtocols(), config.getTlsCertRefreshCheckDurationSec()); } - httpsConnector = new ServerConnector(server, sslCtxFactory, new HttpConnectionFactory(httpConfig)); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(new SslConnectionFactory(sslCtxFactory, httpConnectionFactory.getProtocol())); + connectionFactories.add(httpConnectionFactory); + // org.eclipse.jetty.server.AbstractConnectionFactory.getFactories contains similar logic + // this is needed for TLS authentication + if (httpConfig.getCustomizer(SecureRequestCustomizer.class) == null) { + httpConfig.addCustomizer(new SecureRequestCustomizer()); + } + httpsConnector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); httpsConnector.setPort(tlsPort.get()); httpsConnector.setHost(pulsar.getBindAddress()); connectors.add(httpsConnector); @@ -285,7 +311,12 @@ public void addStaticResources(String basePath, String resourcePath) { public void start() throws PulsarServerException { try { RequestLogHandler requestLogHandler = new RequestLogHandler(); - requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger()); + boolean showDetailedAddresses = pulsar.getConfiguration().getWebServiceLogDetailedAddresses() != null + ? pulsar.getConfiguration().getWebServiceLogDetailedAddresses() : + (pulsar.getConfiguration().isWebServiceHaProxyProtocolEnabled() + || pulsar.getConfiguration().isWebServiceTrustXForwardedFor()); + RequestLog requestLogger = JettyRequestLogFactory.createRequestLogger(showDetailedAddresses, server); + requestLogHandler.setRequestLog(requestLogger); handlers.add(0, new ContextHandlerCollection()); handlers.add(requestLogHandler); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceOriginalClientIPTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceOriginalClientIPTest.java new file mode 100644 index 0000000000000..7f7fa85bd3bb4 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/web/WebServiceOriginalClientIPTest.java @@ -0,0 +1,155 @@ +/* + * 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.web; + +import static org.testng.Assert.assertTrue; +import java.time.Duration; +import java.time.temporal.ChronoUnit; +import java.util.Optional; +import lombok.extern.slf4j.Slf4j; +import nl.altindag.console.ConsoleCaptor; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.assertj.core.api.ThrowingConsumer; +import org.awaitility.Awaitility; +import org.eclipse.jetty.client.HttpClient; +import org.eclipse.jetty.client.ProxyProtocolClientConnectionFactory.V2; +import org.eclipse.jetty.client.api.ContentResponse; +import org.eclipse.jetty.util.ssl.SslContextFactory; +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 WebServiceOriginalClientIPTest extends MockedPulsarServiceBaseTest { + HttpClient httpClient; + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + httpClient = new HttpClient(new SslContextFactory(true)); + httpClient.start(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + if (httpClient != null) { + httpClient.stop(); + } + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + conf.setWebServiceTrustXForwardedFor(true); + conf.setWebServiceHaProxyProtocolEnabled(true); + conf.setWebServicePortTls(Optional.of(0)); + conf.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + conf.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); + conf.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); + } + + @DataProvider(name = "tlsEnabled") + public Object[][] tlsEnabled() { + return new Object[][] { { true }, { false } }; + } + + @Test(dataProvider = "tlsEnabled") + public void testClientIPIsPickedFromXForwardedForHeaderAndLogged(boolean tlsEnabled) throws Exception { + String metricsUrl = + (tlsEnabled ? pulsar.getWebServiceAddressTls() : pulsar.getWebServiceAddress()) + "/metrics/"; + performLoggingTest(consoleCaptor -> { + // Send a GET request to the metrics URL + ContentResponse response = httpClient.newRequest(metricsUrl) + .header("X-Forwarded-For", "11.22.33.44:12345") + .send(); + + // Validate the response + assertTrue(response.getContentAsString().contains("process_cpu_seconds_total")); + + // Validate that the client IP passed in X-Forwarded-For is logged + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("RequestLog") && line.contains("[R:11.22.33.44:12345 via "))); + }); + } + + @Test(dataProvider = "tlsEnabled") + public void testClientIPIsPickedFromForwardedHeaderAndLogged(boolean tlsEnabled) throws Exception { + String metricsUrl = + (tlsEnabled ? pulsar.getWebServiceAddressTls() : pulsar.getWebServiceAddress()) + "/metrics/"; + performLoggingTest(consoleCaptor -> { + // Send a GET request to the metrics URL + ContentResponse response = httpClient.newRequest(metricsUrl) + .header("Forwarded", "for=11.22.33.44:12345") + .send(); + + // Validate the response + assertTrue(response.getContentAsString().contains("process_cpu_seconds_total")); + + // Validate that the client IP passed in Forwarded is logged + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("RequestLog") && line.contains("[R:11.22.33.44:12345 via "))); + }); + } + + @Test(dataProvider = "tlsEnabled") + public void testClientIPIsPickedFromHAProxyProtocolAndLogged(boolean tlsEnabled) throws Exception { + String metricsUrl = (tlsEnabled ? pulsar.getWebServiceAddressTls() : pulsar.getWebServiceAddress()) + "/metrics/"; + performLoggingTest(consoleCaptor -> { + // Send a GET request to the metrics URL + ContentResponse response = httpClient.newRequest(metricsUrl) + // Jetty client will add HA Proxy protocol header with the given IP to the request + .tag(new V2.Tag(V2.Tag.Command.PROXY, null, V2.Tag.Protocol.STREAM, + // source IP and port + "99.22.33.44", 1234, + // destination IP and port + "5.4.3.1", 4321, + null)) + .send(); + + // Validate the response + assertTrue(response.getContentAsString().contains("process_cpu_seconds_total")); + + // Validate that the client IP and destination IP passed in HA Proxy protocol is logged + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("RequestLog") && line.contains("[R:99.22.33.44:1234 via ") + && line.contains(" dst 5.4.3.1:4321]"))); + }); + } + + void performLoggingTest(ThrowingConsumer testFunction) { + ConsoleCaptor consoleCaptor = new ConsoleCaptor(); + try { + Awaitility.await().atMost(Duration.of(2, ChronoUnit.SECONDS)).untilAsserted(() -> { + consoleCaptor.clearOutput(); + testFunction.accept(consoleCaptor); + }); + } finally { + consoleCaptor.close(); + System.out.println("--- Captured console output:"); + consoleCaptor.getStandardOutput().forEach(System.out::println); + consoleCaptor.getErrorOutput().forEach(System.err::println); + System.out.println("--- End of captured console output"); + } + } +} diff --git a/pulsar-broker/src/test/resources/log4j2.xml b/pulsar-broker/src/test/resources/log4j2.xml new file mode 100644 index 0000000000000..09a89702ee2ac --- /dev/null +++ b/pulsar-broker/src/test/resources/log4j2.xml @@ -0,0 +1,40 @@ + + + + + + + + + + + + + + + + + + + diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java index ec0e620d0ae8b..036311ea13230 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/worker/WorkerConfig.java @@ -163,6 +163,22 @@ public class WorkerConfig implements Serializable, PulsarConfiguration { + "(0 to disable limiting)") private int maxHttpServerConnections = 2048; + @FieldContext(category = CATEGORY_WORKER, + doc = "Enable or disable the use of HA proxy protocol for resolving the client IP for http/https " + + "requests. Default is false.") + private boolean webServiceHaProxyProtocolEnabled = false; + + @FieldContext(category = CATEGORY_WORKER, doc = + "Trust X-Forwarded-For header for resolving the client IP for http/https requests.\n" + + "Default is false.") + private boolean webServiceTrustXForwardedFor = false; + + @FieldContext(category = CATEGORY_WORKER, doc = + "Add detailed client/remote and server/local addresses and ports to http/https request logging.\n" + + "Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor " + + "is enabled.") + private Boolean webServiceLogDetailedAddresses; + @FieldContext( category = CATEGORY_WORKER, required = false, diff --git a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java index 2b3ea30121015..583d8ce558b08 100644 --- a/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java +++ b/pulsar-functions/worker/src/main/java/org/apache/pulsar/functions/worker/rest/WorkerServer.java @@ -35,10 +35,17 @@ import org.apache.pulsar.functions.worker.rest.api.v2.WorkerApiV2Resource; import org.apache.pulsar.functions.worker.rest.api.v2.WorkerStatsApiV2Resource; import org.apache.pulsar.jetty.tls.JettySslContextFactory; +import org.eclipse.jetty.server.ConnectionFactory; import org.eclipse.jetty.server.ConnectionLimit; +import org.eclipse.jetty.server.ForwardedRequestCustomizer; import org.eclipse.jetty.server.Handler; +import org.eclipse.jetty.server.HttpConfiguration; +import org.eclipse.jetty.server.HttpConnectionFactory; +import org.eclipse.jetty.server.ProxyConnectionFactory; +import org.eclipse.jetty.server.SecureRequestCustomizer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.server.SslConnectionFactory; import org.eclipse.jetty.server.handler.ContextHandlerCollection; import org.eclipse.jetty.server.handler.DefaultHandler; import org.eclipse.jetty.server.handler.HandlerCollection; @@ -88,10 +95,21 @@ private void init() { server.addBean(new ConnectionLimit(workerConfig.getMaxHttpServerConnections(), server)); } + HttpConfiguration httpConfig = new HttpConfiguration(); + if (workerConfig.isWebServiceTrustXForwardedFor()) { + httpConfig.addCustomizer(new ForwardedRequestCustomizer()); + } + HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory(httpConfig); + List connectors = new ArrayList<>(); if (this.workerConfig.getWorkerPort() != null) { log.info("Configuring http server on port={}", this.workerConfig.getWorkerPort()); - httpConnector = new ServerConnector(server); + List connectionFactories = new ArrayList<>(); + if (workerConfig.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(httpConnectionFactory); + httpConnector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); httpConnector.setPort(this.workerConfig.getWorkerPort()); connectors.add(httpConnector); } @@ -109,7 +127,10 @@ private void init() { workerConfig.isAuthenticateMetricsEndpoint(), filterInitializer)); RequestLogHandler requestLogHandler = new RequestLogHandler(); - requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger()); + boolean showDetailedAddresses = workerConfig.getWebServiceLogDetailedAddresses() != null + ? workerConfig.getWebServiceLogDetailedAddresses() : + (workerConfig.isWebServiceHaProxyProtocolEnabled() || workerConfig.isWebServiceTrustXForwardedFor()); + requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger(showDetailedAddresses, server)); handlers.add(0, new ContextHandlerCollection()); handlers.add(requestLogHandler); @@ -161,7 +182,18 @@ private void init() { workerConfig.getTlsCertRefreshCheckDurationSec() ); } - httpsConnector = new ServerConnector(server, sslCtxFactory); + List connectionFactories = new ArrayList<>(); + if (workerConfig.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(new SslConnectionFactory(sslCtxFactory, httpConnectionFactory.getProtocol())); + connectionFactories.add(httpConnectionFactory); + // org.eclipse.jetty.server.AbstractConnectionFactory.getFactories contains similar logic + // this is needed for TLS authentication + if (httpConfig.getCustomizer(SecureRequestCustomizer.class) == null) { + httpConfig.addCustomizer(new SecureRequestCustomizer()); + } + httpsConnector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); httpsConnector.setPort(this.workerConfig.getWorkerPortTls()); connectors.add(httpsConnector); } catch (Exception e) { diff --git a/pulsar-proxy/pom.xml b/pulsar-proxy/pom.xml index 039e32d71309f..4cac95f76dbc3 100644 --- a/pulsar-proxy/pom.xml +++ b/pulsar-proxy/pom.xml @@ -203,6 +203,12 @@ ${wiremock.version} test + + io.github.hakky54 + consolecaptor + ${consolecaptor.version} + test + 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 1ff141ec13a08..3d4c9e19aa4b0 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 @@ -274,6 +274,22 @@ public class ProxyConfiguration implements PulsarConfiguration { doc = "Enable or disable the proxy protocol.") private boolean haProxyProtocolEnabled; + @FieldContext(category = CATEGORY_SERVER, + doc = "Enable or disable the use of HA proxy protocol for resolving the client IP for http/https " + + "requests. Default is false.") + private boolean webServiceHaProxyProtocolEnabled = false; + + @FieldContext(category = CATEGORY_SERVER, doc = + "Trust X-Forwarded-For header for resolving the client IP for http/https requests.\n" + + "Default is false.") + private boolean webServiceTrustXForwardedFor = false; + + @FieldContext(category = CATEGORY_SERVER, doc = + "Add detailed client/remote and server/local addresses and ports to http/https request logging.\n" + + "Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor " + + "is enabled.") + private Boolean webServiceLogDetailedAddresses; + @FieldContext(category = CATEGORY_SERVER, doc = "Enables zero-copy transport of data across network interfaces using the spice. " + "Zero copy mode cannot be used when TLS is enabled or when proxyLogLevel is > 0.") diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java index aa612f12b1ba9..e0c67c151bc55 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java @@ -36,6 +36,7 @@ import java.util.Collection; import java.util.Collections; import java.util.Date; +import java.util.function.Consumer; import lombok.Getter; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.core.util.datetime.FixedDateFormat; @@ -103,8 +104,15 @@ public class ProxyServiceStarter { private WebServer server; private static boolean metricsInitialized; + private boolean embeddedMode; public ProxyServiceStarter(String[] args) throws Exception { + this(args, null, false); + } + + public ProxyServiceStarter(String[] args, Consumer proxyConfigurationCustomizer, + boolean embeddedMode) throws Exception { + this.embeddedMode = embeddedMode; try { DateFormat dateFormat = new SimpleDateFormat( FixedDateFormat.FixedFormat.ISO8601_OFFSET_DATE_TIME_HHMM.getPattern()); @@ -127,15 +135,26 @@ public ProxyServiceStarter(String[] args) throws Exception { CmdGenerateDocs cmd = new CmdGenerateDocs("pulsar"); cmd.addCommand("proxy", this); cmd.run(null); - System.exit(0); + if (embeddedMode) { + return; + } else { + System.exit(0); + } } } catch (Exception e) { jcommander.usage(); - System.exit(1); + if (embeddedMode) { + return; + } else { + System.exit(1); + } } // load config file config = PulsarConfigurationLoader.create(configFile, ProxyConfiguration.class); + if (proxyConfigurationCustomizer != null) { + proxyConfigurationCustomizer.accept(config); + } if (!isBlank(zookeeperServers)) { // Use zookeeperServers from command line @@ -225,7 +244,9 @@ public void start() throws Exception { // create a web-service server = new WebServer(config, authenticationService); - Runtime.getRuntime().addShutdownHook(new Thread(this::close)); + if (!embeddedMode) { + Runtime.getRuntime().addShutdownHook(new Thread(this::close)); + } proxyService.start(); @@ -283,7 +304,9 @@ public void close() { } catch (Exception e) { log.warn("server couldn't stop gracefully {}", e.getMessage(), e); } finally { - LogManager.shutdown(); + if (!embeddedMode) { + LogManager.shutdown(); + } } } 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 b95bbcab08b11..478b911eb23cf 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 @@ -37,13 +37,18 @@ import org.apache.pulsar.broker.web.RateLimitingFilter; import org.apache.pulsar.broker.web.WebExecutorThreadPool; import org.apache.pulsar.jetty.tls.JettySslContextFactory; +import org.eclipse.jetty.server.ConnectionFactory; import org.eclipse.jetty.server.ConnectionLimit; import org.eclipse.jetty.server.Connector; +import org.eclipse.jetty.server.ForwardedRequestCustomizer; import org.eclipse.jetty.server.Handler; import org.eclipse.jetty.server.HttpConfiguration; import org.eclipse.jetty.server.HttpConnectionFactory; +import org.eclipse.jetty.server.ProxyConnectionFactory; +import org.eclipse.jetty.server.SecureRequestCustomizer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.server.SslConnectionFactory; import org.eclipse.jetty.server.handler.ContextHandlerCollection; import org.eclipse.jetty.server.handler.DefaultHandler; import org.eclipse.jetty.server.handler.HandlerCollection; @@ -93,12 +98,21 @@ public WebServer(ProxyConfiguration config, AuthenticationService authentication List connectors = new ArrayList<>(); HttpConfiguration httpConfig = new HttpConfiguration(); + if (config.isWebServiceTrustXForwardedFor()) { + httpConfig.addCustomizer(new ForwardedRequestCustomizer()); + } httpConfig.setOutputBufferSize(config.getHttpOutputBufferSize()); httpConfig.setRequestHeaderSize(config.getHttpMaxRequestHeaderSize()); + HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory(httpConfig); if (config.getWebServicePort().isPresent()) { this.externalServicePort = config.getWebServicePort().get(); - connector = new ServerConnector(server, new HttpConnectionFactory(httpConfig)); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(httpConnectionFactory); + connector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); connector.setHost(config.getBindAddress()); connector.setPort(externalServicePort); connectors.add(connector); @@ -133,7 +147,18 @@ public WebServer(ProxyConfiguration config, AuthenticationService authentication config.getWebServiceTlsProtocols(), config.getTlsCertRefreshCheckDurationSec()); } - connectorTls = new ServerConnector(server, sslCtxFactory, new HttpConnectionFactory(httpConfig)); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(new SslConnectionFactory(sslCtxFactory, httpConnectionFactory.getProtocol())); + connectionFactories.add(httpConnectionFactory); + // org.eclipse.jetty.server.AbstractConnectionFactory.getFactories contains similar logic + // this is needed for TLS authentication + if (httpConfig.getCustomizer(SecureRequestCustomizer.class) == null) { + httpConfig.addCustomizer(new SecureRequestCustomizer()); + } + connectorTls = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); connectorTls.setPort(config.getWebServicePortTls().get()); connectorTls.setHost(config.getBindAddress()); connectors.add(connectorTls); @@ -281,7 +306,10 @@ public int getExternalServicePort() { public void start() throws Exception { RequestLogHandler requestLogHandler = new RequestLogHandler(); - requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger()); + boolean showDetailedAddresses = config.getWebServiceLogDetailedAddresses() != null + ? config.getWebServiceLogDetailedAddresses() : + (config.isWebServiceHaProxyProtocolEnabled() || config.isWebServiceTrustXForwardedFor()); + requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger(showDetailedAddresses, server)); handlers.add(0, new ContextHandlerCollection()); handlers.add(requestLogHandler); 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 new file mode 100644 index 0000000000000..b267439d47113 --- /dev/null +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyOriginalClientIPTest.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.proxy.server; + +import static org.testng.Assert.assertTrue; +import java.time.Duration; +import java.time.temporal.ChronoUnit; +import java.util.Optional; +import lombok.extern.slf4j.Slf4j; +import nl.altindag.console.ConsoleCaptor; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.assertj.core.api.ThrowingConsumer; +import org.awaitility.Awaitility; +import org.eclipse.jetty.client.HttpClient; +import org.eclipse.jetty.client.ProxyProtocolClientConnectionFactory.V2; +import org.eclipse.jetty.client.api.ContentResponse; +import org.eclipse.jetty.util.ssl.SslContextFactory; +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 ProxyOriginalClientIPTest extends MockedPulsarServiceBaseTest { + static final String[] ARGS = new String[]{"-c", "./src/test/resources/proxy.conf"}; + HttpClient httpClient; + ProxyServiceStarter serviceStarter; + String webServiceUrl; + String webServiceUrlTls; + + @Override + @BeforeClass + protected void setup() throws Exception { + internalSetup(); + serviceStarter = new ProxyServiceStarter(ARGS, proxyConfig -> { + proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl()); + proxyConfig.setBrokerWebServiceURL(pulsar.getWebServiceAddress()); + proxyConfig.setWebServicePort(Optional.of(0)); + proxyConfig.setWebServicePortTls(Optional.of(0)); + proxyConfig.setTlsEnabledWithBroker(false); + proxyConfig.setTlsCertificateFilePath(PROXY_CERT_FILE_PATH); + proxyConfig.setTlsKeyFilePath(PROXY_KEY_FILE_PATH); + proxyConfig.setServicePort(Optional.of(0)); + proxyConfig.setWebSocketServiceEnabled(true); + proxyConfig.setBrokerProxyAllowedTargetPorts("*"); + proxyConfig.setClusterName(configClusterName); + proxyConfig.setWebServiceTrustXForwardedFor(true); + proxyConfig.setWebServiceHaProxyProtocolEnabled(true); + }, true); + serviceStarter.start(); + webServiceUrl = "http://localhost:" + serviceStarter.getServer().getListenPortHTTP().get(); + webServiceUrlTls = "https://localhost:" + serviceStarter.getServer().getListenPortHTTPS().get(); + httpClient = new HttpClient(new SslContextFactory(true)); + httpClient.start(); + } + + @Override + @AfterClass(alwaysRun = true) + protected void cleanup() throws Exception { + internalCleanup(); + if (serviceStarter != null) { + serviceStarter.close(); + } + if (httpClient != null) { + httpClient.stop(); + } + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + conf.setWebServiceTrustXForwardedFor(true); + } + + @DataProvider(name = "tlsEnabled") + public Object[][] tlsEnabled() { + return new Object[][] { { true }, { false } }; + } + + @Test(dataProvider = "tlsEnabled") + public void testClientIPIsPickedFromXForwardedForHeaderAndLogged(boolean tlsEnabled) throws Exception { + String url = (tlsEnabled ? webServiceUrlTls : webServiceUrl) + "/admin/v2/brokers/leaderBroker"; + performLoggingTest(consoleCaptor -> { + // Send a GET request to the metrics URL + ContentResponse response = httpClient.newRequest(url) + .header("X-Forwarded-For", "11.22.33.44") + .send(); + + // Validate the response + assertTrue(response.getContentAsString().contains("\"brokerId\":\"" + pulsar.getBrokerId() + "\"")); + + // Validate that the client IP passed in X-Forwarded-For is logged + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("pulsar-external-web-") && line.contains("RequestLog") + && line.contains("R:11.22.33.44")), "Expected to find client IP in proxy logs"); + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("pulsar-web-") && line.contains("RequestLog") + && line.contains("R:11.22.33.44")), "Expected to find client IP in broker logs"); + }); + } + + @Test(dataProvider = "tlsEnabled") + public void testClientIPIsPickedFromHAProxyProtocolAndLogged(boolean tlsEnabled) throws Exception { + String url = (tlsEnabled ? webServiceUrlTls : webServiceUrl) + "/admin/v2/brokers/leaderBroker"; + performLoggingTest(consoleCaptor -> { + // Send a GET request to the metrics URL + ContentResponse response = httpClient.newRequest(url) + // Jetty client will add HA Proxy protocol header with the given IP to the request + .tag(new V2.Tag("99.22.33.44", 1234)) + .send(); + + // Validate the response + assertTrue(response.getContentAsString().contains("\"brokerId\":\"" + pulsar.getBrokerId() + "\"")); + + // Validate that the client IP passed in HA proxy protocol is logged + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("pulsar-external-web-") && line.contains("RequestLog") + && line.contains("R:99.22.33.44")), "Expected to find client IP in proxy logs"); + assertTrue(consoleCaptor.getStandardOutput().stream() + .anyMatch(line -> line.contains("pulsar-web-") && line.contains("RequestLog") + && line.contains("R:99.22.33.44")), "Expected to find client IP in broker logs"); + }); + } + + void performLoggingTest(ThrowingConsumer testFunction) { + ConsoleCaptor consoleCaptor = new ConsoleCaptor(); + try { + Awaitility.await().atMost(Duration.of(2, ChronoUnit.SECONDS)).untilAsserted(() -> { + consoleCaptor.clearOutput(); + testFunction.accept(consoleCaptor); + }); + } finally { + consoleCaptor.close(); + System.out.println("--- Captured console output:"); + consoleCaptor.getStandardOutput().forEach(System.out::println); + consoleCaptor.getErrorOutput().forEach(System.err::println); + System.out.println("--- End of captured console output"); + } + } +} 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 0c9fa5c7ac322..9eaa992741635 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); + serviceStarter = new ProxyServiceStarter(ARGS, 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 a9bead706a373..759eabdb5a20e 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 @@ -54,7 +54,7 @@ public class ProxyServiceStarterTest extends MockedPulsarServiceBaseTest { @BeforeClass protected void setup() throws Exception { internalSetup(); - serviceStarter = new ProxyServiceStarter(ARGS); + serviceStarter = new ProxyServiceStarter(ARGS, 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/ProxyServiceTlsStarterTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyServiceTlsStarterTest.java index 6247c2a66e874..a4ebe25b428a3 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 @@ -55,7 +55,7 @@ public class ProxyServiceTlsStarterTest extends MockedPulsarServiceBaseTest { @BeforeClass protected void setup() throws Exception { internalSetup(); - serviceStarter = new ProxyServiceStarter(ARGS); + serviceStarter = new ProxyServiceStarter(ARGS, 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/log4j2.xml b/pulsar-proxy/src/test/resources/log4j2.xml new file mode 100644 index 0000000000000..261bd2edf6980 --- /dev/null +++ b/pulsar-proxy/src/test/resources/log4j2.xml @@ -0,0 +1,36 @@ + + + + + + + + + + + + + + + diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java index 7aed43d056c67..bbb34a3e3f73d 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/ProxyServer.java @@ -35,10 +35,17 @@ import org.apache.pulsar.broker.web.WebExecutorThreadPool; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.jetty.tls.JettySslContextFactory; +import org.eclipse.jetty.server.ConnectionFactory; import org.eclipse.jetty.server.ConnectionLimit; +import org.eclipse.jetty.server.ForwardedRequestCustomizer; import org.eclipse.jetty.server.Handler; +import org.eclipse.jetty.server.HttpConfiguration; +import org.eclipse.jetty.server.HttpConnectionFactory; +import org.eclipse.jetty.server.ProxyConnectionFactory; +import org.eclipse.jetty.server.SecureRequestCustomizer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.server.SslConnectionFactory; import org.eclipse.jetty.server.handler.ContextHandlerCollection; import org.eclipse.jetty.server.handler.DefaultHandler; import org.eclipse.jetty.server.handler.HandlerCollection; @@ -73,10 +80,22 @@ public ProxyServer(WebSocketProxyConfiguration config) if (config.getMaxHttpServerConnections() > 0) { server.addBean(new ConnectionLimit(config.getMaxHttpServerConnections(), server)); } + + HttpConfiguration httpConfig = new HttpConfiguration(); + if (config.isWebServiceTrustXForwardedFor()) { + httpConfig.addCustomizer(new ForwardedRequestCustomizer()); + } + HttpConnectionFactory httpConnectionFactory = new HttpConnectionFactory(httpConfig); + List connectors = new ArrayList<>(); if (config.getWebServicePort().isPresent()) { - connector = new ServerConnector(server); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(httpConnectionFactory); + connector = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); connector.setPort(config.getWebServicePort().get()); connectors.add(connector); } @@ -111,7 +130,18 @@ public ProxyServer(WebSocketProxyConfiguration config) config.getWebServiceTlsProtocols(), config.getTlsCertRefreshCheckDurationSec()); } - connectorTls = new ServerConnector(server, sslCtxFactory); + List connectionFactories = new ArrayList<>(); + if (config.isWebServiceHaProxyProtocolEnabled()) { + connectionFactories.add(new ProxyConnectionFactory()); + } + connectionFactories.add(new SslConnectionFactory(sslCtxFactory, httpConnectionFactory.getProtocol())); + connectionFactories.add(httpConnectionFactory); + // org.eclipse.jetty.server.AbstractConnectionFactory.getFactories contains similar logic + // this is needed for TLS authentication + if (httpConfig.getCustomizer(SecureRequestCustomizer.class) == null) { + httpConfig.addCustomizer(new SecureRequestCustomizer()); + } + connectorTls = new ServerConnector(server, connectionFactories.toArray(new ConnectionFactory[0])); connectorTls.setPort(config.getWebServicePortTls().get()); connectors.add(connectorTls); } catch (Exception e) { @@ -169,7 +199,10 @@ public void start() throws PulsarServerException { .map(ServerConnector.class::cast).map(ServerConnector::getPort).map(Object::toString) .collect(Collectors.joining(","))); RequestLogHandler requestLogHandler = new RequestLogHandler(); - requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger()); + boolean showDetailedAddresses = conf.getWebServiceLogDetailedAddresses() != null + ? conf.getWebServiceLogDetailedAddresses() : + (conf.isWebServiceHaProxyProtocolEnabled() || conf.isWebServiceTrustXForwardedFor()); + requestLogHandler.setRequestLog(JettyRequestLogFactory.createRequestLogger(showDetailedAddresses, server)); handlers.add(0, new ContextHandlerCollection()); handlers.add(requestLogHandler); diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java index b7efc4c08f3c1..637b74c05de84 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java @@ -96,6 +96,20 @@ public class WebSocketProxyConfiguration implements PulsarConfiguration { @FieldContext(doc = "Hostname or IP address the service binds on, default is 0.0.0.0.") private String bindAddress = "0.0.0.0"; + @FieldContext(doc = "Enable or disable the use of HA proxy protocol for resolving the client IP for http/https " + + "requests. Default is false.") + private boolean webServiceHaProxyProtocolEnabled = false; + + @FieldContext(doc = "Trust X-Forwarded-For header for resolving the client IP for http/https requests.\n" + + "Default is false.") + private boolean webServiceTrustXForwardedFor = false; + + @FieldContext(doc = + "Add detailed client/remote and server/local addresses and ports to http/https request logging.\n" + + "Defaults to true when either webServiceHaProxyProtocolEnabled or webServiceTrustXForwardedFor " + + "is enabled.") + private Boolean webServiceLogDetailedAddresses; + @FieldContext(doc = "Maximum size of a text message during parsing in WebSocket proxy") private int webSocketMaxTextFrameSize = 1024 * 1024; // --- Authentication --- From fd8329d93f17511247dbb66f18b0321a7bf1befb Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Tue, 23 Apr 2024 00:05:41 +0800 Subject: [PATCH 448/699] [fix][ml] Fix NPE of getValidPositionAfterSkippedEntries when recovering a terminated managed ledger (#22552) (cherry picked from commit 35599b7325347838203a92ca63b78d134b7864c2) (cherry picked from commit def695b18b270faf4e159b060ee75dbc4e699744) --- .../mledger/impl/ManagedLedgerImpl.java | 2 +- .../mledger/impl/ManagedCursorTest.java | 61 +++++++++++++++++++ 2 files changed, 62 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 32ac345629d32..3c56d6c66a5fd 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 @@ -3747,7 +3747,7 @@ public PositionImpl getValidPositionAfterSkippedEntries(final PositionImpl posit Long nextLedgerId = ledgers.ceilingKey(skippedPosition.getLedgerId() + 1); // This means it has jumped to the last position if (nextLedgerId == null) { - if (currentLedgerEntries == 0) { + if (currentLedgerEntries == 0 && currentLedger != null) { return PositionImpl.get(currentLedger.getId(), 0); } return lastConfirmedEntry.getNext(); 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 c9bd64171c15a..4e3f8b7908438 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 @@ -4695,5 +4695,66 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { && cursorReadPosition.getEntryId() == expectReadPosition.getEntryId()); } + @Test + public void testRecoverCursorWithTerminateManagedLedger() throws Exception { + String mlName = "my_test_ledger"; + String cursorName = "c1"; + + ManagedLedgerConfig config = new ManagedLedgerConfig(); + ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open(mlName, config); + ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor(cursorName); + + // Write some data. + Position p0 = ledger.addEntry("entry-0".getBytes()); + Position p1 = ledger.addEntry("entry-1".getBytes()); + + // Read message. + List entries = c1.readEntries(2); + assertEquals(entries.size(), 2); + assertEquals(entries.get(0).getPosition(), p0); + assertEquals(entries.get(1).getPosition(), p1); + entries.forEach(Entry::release); + + // Mark delete the last message. + c1.markDelete(p1); + Position markDeletedPosition = c1.getMarkDeletedPosition(); + Assert.assertEquals(markDeletedPosition, p1); + + // Terminate the managed ledger. + Position lastPosition = ledger.terminate(); + assertEquals(lastPosition, p1); + + // Close the ledger. + ledger.close(); + + // Reopen the ledger. + ledger = (ManagedLedgerImpl) factory.open(mlName, config); + BookKeeper mockBookKeeper = mock(BookKeeper.class); + final ManagedCursorImpl cursor = new ManagedCursorImpl(mockBookKeeper, new ManagedLedgerConfig(), ledger, + cursorName); + + CompletableFuture recoverFuture = new CompletableFuture<>(); + // Recover the cursor. + cursor.recover(new VoidCallback() { + @Override + public void operationComplete() { + recoverFuture.complete(null); + } + + @Override + public void operationFailed(ManagedLedgerException exception) { + recoverFuture.completeExceptionally(exception); + } + }); + + recoverFuture.join(); + assertTrue(recoverFuture.isDone()); + assertFalse(recoverFuture.isCompletedExceptionally()); + + // Verify the cursor state. + assertEquals(cursor.getMarkDeletedPosition(), markDeletedPosition); + assertEquals(cursor.getReadPosition(), markDeletedPosition.getNext()); + } + private static final Logger log = LoggerFactory.getLogger(ManagedCursorTest.class); } From b7c5eaf94fc760b81d11e8d34d69797bb0795173 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 22 Apr 2024 18:13:45 +0300 Subject: [PATCH 449/699] [fix][offload] Increase file upload limit from 2048MiB to 4096MiB for GCP/GCS offloading (#22554) (cherry picked from commit e81f37000ec212676c5daffa17faad8fc604ff77) (cherry picked from commit 5c09c20b33c8e9563df82b4f510fe1d7b519f33c) --- conf/broker.conf | 11 ++++++----- .../common/policies/data/OffloadPoliciesImpl.java | 7 ++++--- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index ac6da587b016d..2fa2ce1eb2e60 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1671,10 +1671,10 @@ s3ManagedLedgerOffloadBucket= # For Amazon S3 ledger offload, Alternative endpoint to connect to (useful for testing) s3ManagedLedgerOffloadServiceEndpoint= -# For Amazon S3 ledger offload, Max block size in bytes. (64MB by default, 5MB minimum) +# For Amazon S3 ledger offload, Max block size in bytes. (64MiB by default, 5MiB minimum) s3ManagedLedgerOffloadMaxBlockSizeInBytes=67108864 -# For Amazon S3 ledger offload, Read buffer size in bytes (1MB by default) +# For Amazon S3 ledger offload, Read buffer size in bytes (1MiB by default) s3ManagedLedgerOffloadReadBufferSizeInBytes=1048576 # For Google Cloud Storage ledger offload, region where offload bucket is located. @@ -1684,10 +1684,11 @@ gcsManagedLedgerOffloadRegion= # For Google Cloud Storage ledger offload, Bucket to place offloaded ledger into gcsManagedLedgerOffloadBucket= -# For Google Cloud Storage ledger offload, Max block size in bytes. (64MB by default, 5MB minimum) -gcsManagedLedgerOffloadMaxBlockSizeInBytes=67108864 +# For Google Cloud Storage ledger offload, Max block size in bytes. (128MiB by default, 5MiB minimum) +# Since JClouds limits the maximum number of blocks to 32, the maximum size of a ledger is 32 times the block size. +gcsManagedLedgerOffloadMaxBlockSizeInBytes=134217728 -# For Google Cloud Storage ledger offload, Read buffer size in bytes (1MB by default) +# For Google Cloud Storage ledger offload, Read buffer size in bytes (1MiB by default) gcsManagedLedgerOffloadReadBufferSizeInBytes=1048576 # For Google Cloud Storage, path to json file containing service account credentials. diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java index 51e181811c228..6c40aa3f2edd0 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/OffloadPoliciesImpl.java @@ -79,8 +79,9 @@ public class OffloadPoliciesImpl implements Serializable, OffloadPolicies { } } - public static final int DEFAULT_MAX_BLOCK_SIZE_IN_BYTES = 64 * 1024 * 1024; // 64MB - public static final int DEFAULT_READ_BUFFER_SIZE_IN_BYTES = 1024 * 1024; // 1MB + public static final int DEFAULT_MAX_BLOCK_SIZE_IN_BYTES = 64 * 1024 * 1024; // 64MiB + public static final int DEFAULT_GCS_MAX_BLOCK_SIZE_IN_BYTES = 128 * 1024 * 1024; // 128MiB + public static final int DEFAULT_READ_BUFFER_SIZE_IN_BYTES = 1024 * 1024; // 1MiB public static final int DEFAULT_OFFLOAD_MAX_THREADS = 2; public static final int DEFAULT_OFFLOAD_MAX_PREFETCH_ROUNDS = 1; public static final String DEFAULT_OFFLOADER_DIRECTORY = "./offloaders"; @@ -163,7 +164,7 @@ public class OffloadPoliciesImpl implements Serializable, OffloadPolicies { private String gcsManagedLedgerOffloadBucket = null; @Configuration @JsonProperty(access = JsonProperty.Access.READ_WRITE) - private Integer gcsManagedLedgerOffloadMaxBlockSizeInBytes = DEFAULT_MAX_BLOCK_SIZE_IN_BYTES; + private Integer gcsManagedLedgerOffloadMaxBlockSizeInBytes = DEFAULT_GCS_MAX_BLOCK_SIZE_IN_BYTES; @Configuration @JsonProperty(access = JsonProperty.Access.READ_WRITE) private Integer gcsManagedLedgerOffloadReadBufferSizeInBytes = DEFAULT_READ_BUFFER_SIZE_IN_BYTES; From 2f451301e26404b140b7eb872b89624f2fa26ee1 Mon Sep 17 00:00:00 2001 From: coderzc Date: Tue, 23 Apr 2024 15:04:57 +0800 Subject: [PATCH 450/699] Revert "[fix][test][branch-3.0] Fix broken ManagedLedgerTest.testGetNumberOfEntriesInStorage" This reverts commit e3531e808c172ff588e36499c41746835d06904a. (cherry picked from commit d6791a8de269498cf2f5d82a0ac54baab60f8845) --- .../org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 c28a32d9ec2e1..c16c31a7e963e 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 @@ -2643,7 +2643,7 @@ public void testGetNumberOfEntriesInStorage() throws Exception { // trigger ledger rollover and wait for the new ledger created Awaitility.await().untilAsserted(() -> { - assertEquals(managedLedger.getState(), ManagedLedgerImpl.State.ClosedLedger); + assertEquals("LedgerOpened", WhiteboxImpl.getInternalState(managedLedger, "state").toString()); }); managedLedger.createLedgerAfterClosed(); Awaitility.await().untilAsserted(() -> { From 6b7aa50bdda48b70a789decb857e9ae148510960 Mon Sep 17 00:00:00 2001 From: coderzc Date: Tue, 23 Apr 2024 15:27:19 +0800 Subject: [PATCH 451/699] [fix][test][branch-3.0] Fix test PersistentTopicsTest.testUpdatePartitionedTopic (cherry picked from commit 6e849fcb06447d767c4f3fa521b5a1c4d14c9aae) --- .../org/apache/pulsar/broker/admin/PersistentTopicsTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 cbffeca8e81f5..42912d17c1c2b 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 @@ -1745,7 +1745,7 @@ public void testUpdatePartitionedTopic() true, 5); verify(response, timeout(5000).times(1)).resume(throwableCaptor.capture()); Assert.assertEquals(throwableCaptor.getValue().getMessage(), - "Desired partitions 5 can't be greater than the maximum partitions per topic 4."); + "Expect partitions 5 grater than maximum partitions per topic 4"); response = mock(AsyncResponse.class); metaCaptor = ArgumentCaptor.forClass(PartitionedTopicMetadata.class); From 85b8b322b8f576190ba135ad18267a78b5b12f98 Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Tue, 23 Apr 2024 16:15:01 +0530 Subject: [PATCH 452/699] Updated license version for Datastax --- .../server/src/assemble/LICENSE.bin.txt | 44 +++++++++---------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index f3881ce74cc45..e0af5bb16daf6 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -585,28 +585,28 @@ Bouncy Castle License - org.bouncycastle-bcutil-jdk18on-1.78.jar Datastax - - com.datastax.oss-managed-ledger-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-auth-oidc-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-auth-sasl-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-common-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-config-validation-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-docs-tools-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-api-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-instance-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-local-runner-original-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-proto-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-runtime-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-secrets-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-utils-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-worker-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-io-core-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-metadata-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-proxy-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-testclient-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-transaction-common-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-transaction-coordinator-3.1.3.2-SNAPSHOT.jar - - com.datastax.oss-testmocks-3.1.3.2-SNAPSHOT.jar + - com.datastax.oss-managed-ledger-3.1.4.0.jar + - com.datastax.oss-pulsar-broker-3.1.4.0.jar + - com.datastax.oss-pulsar-broker-auth-oidc-3.1.4.0.jar + - com.datastax.oss-pulsar-broker-auth-sasl-3.1.4.0.jar + - com.datastax.oss-pulsar-broker-common-3.1.4.0.jar + - com.datastax.oss-pulsar-config-validation-3.1.4.0.jar + - com.datastax.oss-pulsar-docs-tools-3.1.4.0.jar + - com.datastax.oss-pulsar-functions-api-3.1.4.0.jar + - com.datastax.oss-pulsar-functions-instance-3.1.4.0.jar + - com.datastax.oss-pulsar-functions-local-runner-original-3.1.4.0.jar + - com.datastax.oss-pulsar-functions-proto-3.1.4.0.jar + - com.datastax.oss-pulsar-functions-runtime-3.1.4.0.jar + - com.datastax.oss-pulsar-functions-secrets-3.1.4.0.jar + - com.datastax.oss-pulsar-functions-utils-3.1.4.0.jar + - com.datastax.oss-pulsar-functions-worker-3.1.4.0.jar + - com.datastax.oss-pulsar-io-core-3.1.4.0.jar + - com.datastax.oss-pulsar-metadata-3.1.4.0.jar + - com.datastax.oss-pulsar-proxy-3.1.4.0.jar + - com.datastax.oss-pulsar-testclient-3.1.4.0.jar + - com.datastax.oss-pulsar-transaction-common-3.1.4.0.jar + - com.datastax.oss-pulsar-transaction-coordinator-3.1.4.0.jar + - com.datastax.oss-testmocks-3.1.4.0.jar ------------------------ From 1d54ff8f776efc24a94471fd2f6bfcdff6bdbe31 Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Tue, 23 Apr 2024 16:34:47 +0530 Subject: [PATCH 453/699] Release 3.1.4.0 --- 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-experimental/pom.xml | 2 +- docker/pulsar/pom.xml | 2 +- jclouds-shaded/pom.xml | 2 +- managed-ledger/pom.xml | 2 +- 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-client-1x-base/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-1x/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-2x-shaded/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/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/hdfs2/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-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-sql/pom.xml | 2 +- pulsar-sql/presto-distribution/pom.xml | 2 +- pulsar-sql/presto-pulsar-plugin/pom.xml | 2 +- pulsar-sql/presto-pulsar/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 +- 134 files changed, 136 insertions(+), 136 deletions(-) diff --git a/bouncy-castle/bc/pom.xml b/bouncy-castle/bc/pom.xml index 92ca225693e09..797fddcbb1125 100644 --- a/bouncy-castle/bc/pom.xml +++ b/bouncy-castle/bc/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.3.2-SNAPSHOT + 3.1.4.0 .. diff --git a/bouncy-castle/bcfips-include-test/pom.xml b/bouncy-castle/bcfips-include-test/pom.xml index c7fac538ba314..2e2c33a2386fe 100644 --- a/bouncy-castle/bcfips-include-test/pom.xml +++ b/bouncy-castle/bcfips-include-test/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.3.2-SNAPSHOT + 3.1.4.0 .. diff --git a/bouncy-castle/bcfips/pom.xml b/bouncy-castle/bcfips/pom.xml index b750dd11218ca..a2241fc99fe23 100644 --- a/bouncy-castle/bcfips/pom.xml +++ b/bouncy-castle/bcfips/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.3.2-SNAPSHOT + 3.1.4.0 .. diff --git a/bouncy-castle/pom.xml b/bouncy-castle/pom.xml index 5a6f5b667a46a..4b110d7d1fe5c 100644 --- a/bouncy-castle/pom.xml +++ b/bouncy-castle/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.3.2-SNAPSHOT + 3.1.4.0 .. diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 50e20c9689e22..a1002e7bc4f9c 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -31,12 +31,12 @@ com.datastax.oss buildtools - 3.1.3.2-SNAPSHOT + 3.1.4.0 jar Pulsar Build Tools - 2024-04-15T04:49:52Z + 2024-04-23T11:04:44Z 1.8 1.8 3.1.0 diff --git a/distribution/io/pom.xml b/distribution/io/pom.xml index b9a004cf65b14..e559f00222158 100644 --- a/distribution/io/pom.xml +++ b/distribution/io/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.3.2-SNAPSHOT + 3.1.4.0 .. diff --git a/distribution/offloaders/pom.xml b/distribution/offloaders/pom.xml index 1a541249bf3ac..03ccedd9e738a 100644 --- a/distribution/offloaders/pom.xml +++ b/distribution/offloaders/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.3.2-SNAPSHOT + 3.1.4.0 .. diff --git a/distribution/pom.xml b/distribution/pom.xml index aea65b0f7eb16..8f81b25e86852 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.3.2-SNAPSHOT + 3.1.4.0 .. diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index 21e4dc996f79e..db7e6d7493ea0 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.3.2-SNAPSHOT + 3.1.4.0 .. diff --git a/distribution/shell/pom.xml b/distribution/shell/pom.xml index f6ef5ecc22449..7c2a955eb60d1 100644 --- a/distribution/shell/pom.xml +++ b/distribution/shell/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.3.2-SNAPSHOT + 3.1.4.0 .. diff --git a/docker/pom.xml b/docker/pom.xml index 7c76d4ab12465..a1d92b9380b04 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.3.2-SNAPSHOT + 3.1.4.0 docker-images Apache Pulsar :: Docker Images diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index 08cbc2abbdf78..b731423c905c4 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.3.2-SNAPSHOT + 3.1.4.0 4.0.0 pulsar-all-docker-image diff --git a/docker/pulsar-experimental/pom.xml b/docker/pulsar-experimental/pom.xml index e681b02784f6c..6af5bb2c8e541 100644 --- a/docker/pulsar-experimental/pom.xml +++ b/docker/pulsar-experimental/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.3.2-SNAPSHOT + 3.1.4.0 4.0.0 pulsar-experimental-docker-image diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 238fc1e141f2f..ce3ed7e5e7d33 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.3.2-SNAPSHOT + 3.1.4.0 4.0.0 pulsar-docker-image diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml index 3672ee3444398..b9a49ffbb57d0 100644 --- a/jclouds-shaded/pom.xml +++ b/jclouds-shaded/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.3.2-SNAPSHOT + 3.1.4.0 .. diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index 7ddcb13f88502..495d18d82ee56 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.3.2-SNAPSHOT + 3.1.4.0 .. diff --git a/pom.xml b/pom.xml index 666ffa0fb73ad..4b398232711e2 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ com.datastax.oss pulsar - 3.1.3.2-SNAPSHOT + 3.1.4.0 Pulsar Pulsar is a distributed pub-sub messaging platform with a very @@ -95,7 +95,7 @@ flexible messaging model and an intuitive client API. UTF-8 UTF-8 - 2024-04-15T04:49:52Z + 2024-04-23T11:04:44Z true system topic --rely on--> deduplication recover + --may rely on--> (tiered storage) + */ + return false; + } + @Override public boolean isEncryptionRequired() { // System topics are only written by the broker that can't know the encryption context. 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 1dcd477ca53f3..b90e2f11b6014 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 @@ -37,6 +37,8 @@ import io.netty.channel.EventLoopGroup; import java.lang.reflect.Field; import java.util.Map; +import java.util.Optional; +import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedCursor; @@ -50,9 +52,11 @@ 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.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.protocol.Commands; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.compaction.CompactionServiceFactory; @@ -487,4 +491,32 @@ public void testMessageDeduplication() throws Exception { messageDeduplication.purgeInactiveProducers(); assertTrue(messageDeduplication.getInactiveProducers().isEmpty()); } + + + @Test + public void testMessageDeduplicationShouldNotWorkForSystemTopic() throws PulsarAdminException { + final String localName = UUID.randomUUID().toString(); + final String namespace = "prop/ns-abc"; + final String prefix = "persistent://%s/".formatted(namespace); + final String topic = prefix + localName; + admin.topics().createNonPartitionedTopic(topic); + + // broker level policies + final String eventSystemTopic = prefix + SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME; + final Optional optionalTopic = pulsar.getBrokerService().getTopic(eventSystemTopic, true).join(); + assertTrue(optionalTopic.isPresent()); + final Topic ptRef = optionalTopic.get(); + assertTrue(ptRef.isSystemTopic()); + assertFalse(ptRef.isDeduplicationEnabled()); + + // namespace level policies + admin.namespaces().setDeduplicationStatus(namespace, true); + assertTrue(ptRef.isSystemTopic()); + assertFalse(ptRef.isDeduplicationEnabled()); + + // topic level policies + admin.topicPolicies().setDeduplicationStatus(eventSystemTopic, true); + assertTrue(ptRef.isSystemTopic()); + assertFalse(ptRef.isDeduplicationEnabled()); + } } From df29a6c1751c196847074b6572aff449291f1448 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 8 May 2024 06:56:35 +0300 Subject: [PATCH 496/699] [improve][ws] Add memory limit configuration for Pulsar client used in Websocket proxy (#22666) (cherry picked from commit 80d46758e89b088688d521aa8ae401bfb00c98b2) (cherry picked from commit e5515c5d17366087025e2a243a988a3290728515) --- conf/broker.conf | 3 +++ conf/standalone.conf | 3 +++ conf/websocket.conf | 3 +++ .../org/apache/pulsar/broker/ServiceConfiguration.java | 7 +++++++ .../java/org/apache/pulsar/websocket/WebSocketService.java | 3 ++- .../websocket/service/WebSocketProxyConfiguration.java | 3 +++ 6 files changed, 21 insertions(+), 1 deletion(-) diff --git a/conf/broker.conf b/conf/broker.conf index 2fa2ce1eb2e60..906a84ce89e15 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1549,6 +1549,9 @@ webSocketNumServiceThreads= # Number of connections per Broker in Pulsar Client used in WebSocket proxy webSocketConnectionsPerBroker= +# Memory limit in MBs for direct memory in Pulsar Client used in WebSocket proxy +webSocketPulsarClientMemoryLimitInMB=0 + # Time in milliseconds that idle WebSocket session times out webSocketSessionIdleTimeoutMillis=300000 diff --git a/conf/standalone.conf b/conf/standalone.conf index c43c3419fef92..d9b368d2952ca 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -973,6 +973,9 @@ webSocketNumIoThreads=8 # Number of connections per Broker in Pulsar Client used in WebSocket proxy webSocketConnectionsPerBroker=8 +# Memory limit in MBs for direct memory in Pulsar Client used in WebSocket proxy +webSocketPulsarClientMemoryLimitInMB=0 + # Time in milliseconds that idle WebSocket session times out webSocketSessionIdleTimeoutMillis=300000 diff --git a/conf/websocket.conf b/conf/websocket.conf index 33e4cf455e022..bead36c6ddea6 100644 --- a/conf/websocket.conf +++ b/conf/websocket.conf @@ -71,6 +71,9 @@ numHttpServerThreads= # Number of connections per Broker in Pulsar Client used in WebSocket proxy webSocketConnectionsPerBroker= +# Memory limit in MBs for direct memory in Pulsar Client used in WebSocket proxy +webSocketPulsarClientMemoryLimitInMB=0 + # Time in milliseconds that idle WebSocket session times out webSocketSessionIdleTimeoutMillis=300000 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 3322adb3253a3..29d5d4b2ea30b 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 @@ -2879,6 +2879,13 @@ The delayed message index time step(in seconds) in per bucket snapshot segment, doc = "Number of connections per Broker in Pulsar Client used in WebSocket proxy" ) private int webSocketConnectionsPerBroker = Runtime.getRuntime().availableProcessors(); + + @FieldContext( + category = CATEGORY_WEBSOCKET, + doc = "Memory limit in MBs for direct memory in Pulsar Client used in WebSocket proxy" + ) + private int webSocketPulsarClientMemoryLimitInMB = 0; + @FieldContext( category = CATEGORY_WEBSOCKET, doc = "Time in milliseconds that idle WebSocket session times out" 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 66b2a0075ec2d..889f4431cc35b 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 @@ -195,7 +195,8 @@ public synchronized void setLocalCluster(ClusterData clusterData) { private PulsarClient createClientInstance(ClusterData clusterData) throws IOException { ClientBuilder clientBuilder = PulsarClient.builder() // - .memoryLimit(0, SizeUnit.BYTES) + .memoryLimit(SizeUnit.MEGA_BYTES.toBytes(config.getWebSocketPulsarClientMemoryLimitInMB()), + SizeUnit.BYTES) .statsInterval(0, TimeUnit.SECONDS) // .enableTls(config.isTlsEnabled()) // .allowTlsInsecureConnection(config.isTlsAllowInsecureConnection()) // diff --git a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java index 637b74c05de84..98ee2cb1d7df0 100644 --- a/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java +++ b/pulsar-websocket/src/main/java/org/apache/pulsar/websocket/service/WebSocketProxyConfiguration.java @@ -185,6 +185,9 @@ public class WebSocketProxyConfiguration implements PulsarConfiguration { @FieldContext(doc = "Number of connections per broker in Pulsar client used in WebSocket proxy") private int webSocketConnectionsPerBroker = Runtime.getRuntime().availableProcessors(); + @FieldContext(doc = "Memory limit in MBs for direct memory in Pulsar Client used in WebSocket proxy") + private int webSocketPulsarClientMemoryLimitInMB = 0; + @FieldContext(doc = "Timeout of idling WebSocket session (in milliseconds)") private int webSocketSessionIdleTimeoutMillis = 300000; From c07b665d240bfb306bd3be5964be35c77098a065 Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Wed, 8 May 2024 13:10:49 +0800 Subject: [PATCH 497/699] [fix][broker] avoid offload system topic (#22497) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: 道君 (cherry picked from commit 3114199c185cb03a7fdb1b8af2bbc356162cf42d) (cherry picked from commit 20483f54efff842e84e6fe32f33fe7ebfc352e38) --- .../pulsar/broker/service/BrokerService.java | 8 +- .../broker/service/BrokerServiceTest.java | 93 +++++++++++++++++++ 2 files changed, 100 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 27234044368dc..aca9bd1a22806 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 @@ -1975,7 +1975,13 @@ private CompletableFuture getManagedLedgerConfig(@Nonnull T topicLevelOffloadPolicies, OffloadPoliciesImpl.oldPoliciesCompatible(nsLevelOffloadPolicies, policies.orElse(null)), getPulsar().getConfig().getProperties()); - if (NamespaceService.isSystemServiceNamespace(namespace.toString())) { + if (NamespaceService.isSystemServiceNamespace(namespace.toString()) + || SystemTopicNames.isSystemTopic(topicName)) { + /* + Avoid setting broker internal system topics using off-loader because some of them are the + preconditions of other topics. The slow replying log speed will cause a delay in all the topic + loading.(timeout) + */ managedLedgerConfig.setLedgerOffloader(NullLedgerOffloader.INSTANCE); } else { if (topicLevelOffloadPolicies != null) { 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 25d9b328967b0..9f561889aa825 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 @@ -68,11 +68,14 @@ import java.util.concurrent.atomic.AtomicReference; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.api.ReadHandle; +import org.apache.bookkeeper.mledger.LedgerOffloader; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; 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.commons.lang3.StringUtils; import org.apache.http.HttpResponse; import org.apache.http.client.HttpClient; @@ -112,6 +115,8 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.BundlesData; import org.apache.pulsar.common.policies.data.LocalPolicies; +import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; +import org.apache.pulsar.common.policies.data.OffloadedReadPriority; import org.apache.pulsar.common.policies.data.SubscriptionStats; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.protocol.Commands; @@ -1834,4 +1839,92 @@ public void testUnsubscribeNonDurableSub() throws Exception { fail("Unsubscribe failed"); } } + + + @Test + public void testOffloadConfShouldNotAppliedForSystemTopic() throws PulsarAdminException { + final String driver = "aws-s3"; + final String region = "test-region"; + final String bucket = "test-bucket"; + final String role = "test-role"; + final String roleSessionName = "test-role-session-name"; + final String credentialId = "test-credential-id"; + final String credentialSecret = "test-credential-secret"; + final String endPoint = "test-endpoint"; + final Integer maxBlockSizeInBytes = 5; + final Integer readBufferSizeInBytes = 2; + final Long offloadThresholdInBytes = 10L; + final Long offloadThresholdInSeconds = 1000L; + final Long offloadDeletionLagInMillis = 5L; + + final OffloadPoliciesImpl offloadPolicies = OffloadPoliciesImpl.create( + driver, + region, + bucket, + endPoint, + role, + roleSessionName, + credentialId, + credentialSecret, + maxBlockSizeInBytes, + readBufferSizeInBytes, + offloadThresholdInBytes, + offloadThresholdInSeconds, + offloadDeletionLagInMillis, + OffloadedReadPriority.TIERED_STORAGE_FIRST + ); + + var fakeOffloader = new LedgerOffloader() { + @Override + public String getOffloadDriverName() { + return driver; + } + + @Override + public CompletableFuture offload(ReadHandle ledger, UUID uid, Map extraMetadata) { + return CompletableFuture.completedFuture(null); + } + + @Override + public CompletableFuture readOffloaded(long ledgerId, UUID uid, Map offloadDriverMetadata) { + return CompletableFuture.completedFuture(null); + } + + @Override + public CompletableFuture deleteOffloaded(long ledgerId, UUID uid, Map offloadDriverMetadata) { + return CompletableFuture.completedFuture(null); + } + + @Override + public OffloadPoliciesImpl getOffloadPolicies() { + return offloadPolicies; + } + + @Override + public void close() { + } + }; + + final BrokerService brokerService = pulsar.getBrokerService(); + final String namespace = "prop/" + UUID.randomUUID(); + admin.namespaces().createNamespace(namespace); + admin.namespaces().setOffloadPolicies(namespace, offloadPolicies); + + // Inject the cache to avoid real load off-loader jar + final Map ledgerOffloaderMap = pulsar.getLedgerOffloaderMap(); + ledgerOffloaderMap.put(NamespaceName.get(namespace), fakeOffloader); + + // (1) test normal topic + final String normalTopic = "persistent://" + namespace + "/" + UUID.randomUUID(); + var managedLedgerConfig = brokerService.getManagedLedgerConfig(TopicName.get(normalTopic)).join(); + + Assert.assertEquals(managedLedgerConfig.getLedgerOffloader(), fakeOffloader); + + // (2) test system topic + for (String eventTopicName : SystemTopicNames.EVENTS_TOPIC_NAMES) { + managedLedgerConfig = brokerService.getManagedLedgerConfig(TopicName.get(eventTopicName)).join(); + Assert.assertEquals(managedLedgerConfig.getLedgerOffloader(), NullLedgerOffloader.INSTANCE); + } + } } + From 79f8e29c8a8f41fb33c062d742771686b3bfe11a Mon Sep 17 00:00:00 2001 From: hrzzzz <64506104+hrzzzz@users.noreply.github.com> Date: Thu, 9 May 2024 21:49:27 +0800 Subject: [PATCH 498/699] [fix][broker] Fix ProducerBusy issue due to incorrect userCreatedProducerCount on non-persistent topic (#22685) Co-authored-by: ruihongzhou (cherry picked from commit 253e6506ea2c5ccc6afe1117e311cf24685ce4e9) (cherry picked from commit 89b545eaa8c25a5be5d5e92981474ff752f015db) --- .../nonpersistent/NonPersistentTopic.java | 10 -------- .../nonpersistent/NonPersistentTopicTest.java | 23 ++++++++++++++++++- 2 files changed, 22 insertions(+), 11 deletions(-) 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 152e5631dd90d..aadb232ede062 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 @@ -18,7 +18,6 @@ */ package org.apache.pulsar.broker.service.nonpersistent; -import static com.google.common.base.Preconditions.checkArgument; import static org.apache.bookkeeper.mledger.impl.cache.RangeEntryCacheManagerImpl.create; import static org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; @@ -55,7 +54,6 @@ import org.apache.pulsar.broker.service.BrokerServiceException.UnsupportedVersionException; import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.service.Dispatcher; -import org.apache.pulsar.broker.service.Producer; import org.apache.pulsar.broker.service.Replicator; import org.apache.pulsar.broker.service.StreamingStats; import org.apache.pulsar.broker.service.Subscription; @@ -243,14 +241,6 @@ public boolean isReplicationBacklogExist() { return false; } - @Override - public void removeProducer(Producer producer) { - checkArgument(producer.getTopic() == this); - if (producers.remove(producer.getProducerName(), producer)) { - handleProducerRemoved(producer); - } - } - @Override public CompletableFuture checkIfTransactionBufferRecoverCompletely(boolean isTxnEnabled) { return CompletableFuture.completedFuture(null); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopicTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopicTest.java index b33381126e5c2..69929de17c83a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopicTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/nonpersistent/NonPersistentTopicTest.java @@ -18,11 +18,13 @@ */ package org.apache.pulsar.broker.service.nonpersistent; -import java.util.Optional; +import java.lang.reflect.Field; import java.util.UUID; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import lombok.Cleanup; +import org.apache.pulsar.broker.service.AbstractTopic; import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.broker.service.SubscriptionOption; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -250,4 +252,23 @@ public void testSubscriptionsOnNonPersistentTopic() throws Exception { Awaitility.waitAtMost(10, TimeUnit.SECONDS).pollInterval(1, TimeUnit.SECONDS) .until(() -> subscriptionMap.get(keySharedSubName) == null); } + + @Test + public void testRemoveProducerOnNonPersistentTopic() throws Exception { + final String topicName = "non-persistent://prop/ns-abc/topic_" + UUID.randomUUID(); + + Producer producer = pulsarClient.newProducer() + .topic(topicName) + .create(); + + NonPersistentTopic topic = (NonPersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get(); + Field field = AbstractTopic.class.getDeclaredField("userCreatedProducerCount"); + field.setAccessible(true); + int userCreatedProducerCount = (int) field.get(topic); + assertEquals(userCreatedProducerCount, 1); + + producer.close(); + userCreatedProducerCount = (int) field.get(topic); + assertEquals(userCreatedProducerCount, 0); + } } From 0a02cd94c20378eafb5af03f29b14c9528003c63 Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Fri, 10 May 2024 04:10:31 -0700 Subject: [PATCH 499/699] [fix][client] Fix ReaderBuilder doest not give illegalArgument on connection failure retry (#22639) (cherry picked from commit b56f238f6aaffdc0b37b9f6e2185b219f8708570) (cherry picked from commit 8b2d5e9ce9d5dc2c529ac7d885441c966dafa100) --- .../apache/pulsar/client/impl/ReaderTest.java | 27 +++++++++++++++++++ .../pulsar/client/impl/ReaderBuilderImpl.java | 5 ++-- .../pulsar/client/impl/BuildersTest.java | 2 +- 3 files changed, 31 insertions(+), 3 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java index 0e46445af69a1..ee353363c6a12 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java @@ -36,6 +36,8 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; @@ -48,6 +50,7 @@ import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; +import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Range; import org.apache.pulsar.client.api.Reader; @@ -851,4 +854,28 @@ public void testHasMessageAvailableAfterSeekTimestamp(boolean initializeLastMess assertTrue(reader.hasMessageAvailable()); } } + + @Test + public void testReaderBuilderStateOnRetryFailure() throws Exception { + String ns = "my-property/my-ns"; + String topic = "persistent://" + ns + "/testRetryReader"; + RetentionPolicies retention = new RetentionPolicies(-1, -1); + admin.namespaces().setRetention(ns, retention); + String badUrl = "pulsar://bad-host:8080"; + + PulsarClient client = PulsarClient.builder().serviceUrl(badUrl).build(); + + ReaderBuilder readerBuilder = client.newReader().topic(topic).startMessageFromRollbackDuration(100, + TimeUnit.SECONDS); + + for (int i = 0; i < 3; i++) { + try { + readerBuilder.createAsync().get(1, TimeUnit.SECONDS); + } catch (TimeoutException e) { + log.info("It should time out due to invalid url"); + } catch (IllegalArgumentException e) { + fail("It should not fail with corrupt reader state"); + } + } + } } 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 2860cda0ceef1..ef230475be53b 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 @@ -86,8 +86,9 @@ public CompletableFuture> createAsync() { .failedFuture(new IllegalArgumentException("Topic name must be set on the reader builder")); } - if (conf.getStartMessageId() != null && conf.getStartMessageFromRollbackDurationInSec() > 0 - || conf.getStartMessageId() == null && conf.getStartMessageFromRollbackDurationInSec() <= 0) { + boolean isStartMsgIdExist = conf.getStartMessageId() != null && conf.getStartMessageId() != MessageId.earliest; + if ((isStartMsgIdExist && conf.getStartMessageFromRollbackDurationInSec() > 0) + || (conf.getStartMessageId() == null && conf.getStartMessageFromRollbackDurationInSec() <= 0)) { return FutureUtil .failedFuture(new IllegalArgumentException( "Start message id or start message from roll back must be specified but they cannot be" diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BuildersTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BuildersTest.java index 607689e0e2b3b..5f52f86d8b014 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BuildersTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/BuildersTest.java @@ -106,7 +106,7 @@ public void readerBuilderLoadConfTest() throws Exception { @Test(expectedExceptions = {PulsarClientException.class}, expectedExceptionsMessageRegExp = ".* must be specified but they cannot be specified at the same time.*") public void shouldNotSetTwoOptAtTheSameTime() throws Exception { PulsarClient client = PulsarClient.builder().serviceUrl("pulsar://localhost:6650").build(); - try (Reader reader = client.newReader().topic("abc").startMessageId(MessageId.earliest) + try (Reader reader = client.newReader().topic("abc").startMessageId(MessageId.latest) .startMessageFromRollbackDuration(10, TimeUnit.HOURS).create()) { // no-op } finally { From 1a02229a046e962f8ef4c94c73f9a25855859f24 Mon Sep 17 00:00:00 2001 From: Nikhil Erigila <60037808+nikhilerigila09@users.noreply.github.com> Date: Fri, 10 May 2024 16:41:20 +0530 Subject: [PATCH 500/699] [fix][sec] Upgrade postgresql version to avoid CVE-2024-1597 (#22635) (cherry picked from commit 2cfd9597676828bae68c9dac74e41d65a1a29864) (cherry picked from commit cea5409cde1c33f085fe28fd335574852913e45b) --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 40d7974d003f1..df8e523539800 100644 --- a/pom.xml +++ b/pom.xml @@ -188,7 +188,7 @@ flexible messaging model and an intuitive client API. 5.1.0 3.42.0.0 8.0.11 - 42.5.1 + 42.5.5 0.4.6 2.7.5 0.4.4-hotfix1 @@ -198,7 +198,7 @@ flexible messaging model and an intuitive client API. 8.12.1 368 1.9.7.Final - 42.5.0 + 42.5.5 8.0.30 1.15.16.Final From 7042b52ba273b72763f749142c687cf1fe2c1e08 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Wed, 8 May 2024 19:34:00 +0800 Subject: [PATCH 501/699] [fix] [broker] rename to changeMaxReadPositionCount (#22656) (cherry picked from commit 5ab05129514c1e71a09ec3f28b2b2dda9ce3e47f) (cherry picked from commit 1cf6a72f970c1a6b937a8d441b13184b2d73e471) --- .../buffer/impl/TopicTransactionBuffer.java | 16 ++++++++-------- .../broker/transaction/TransactionTest.java | 12 ++++++------ 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java index a36216bd6258b..81c9ecfc728e9 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 @@ -76,8 +76,8 @@ public class TopicTransactionBuffer extends TopicTransactionBufferState implemen */ private final LinkedMap ongoingTxns = new LinkedMap<>(); - // when add abort or change max read position, the count will +1. Take snapshot will set 0 into it. - private final AtomicLong changeMaxReadPositionAndAddAbortTimes = new AtomicLong(); + // when change max read position, the count will +1. Take snapshot will reset the count. + private final AtomicLong changeMaxReadPositionCount = new AtomicLong(); private final LongAdder txnCommittedCounter = new LongAdder(); @@ -429,15 +429,15 @@ private void handleLowWaterMark(TxnID txnID, long lowWaterMark) { } private void takeSnapshotByChangeTimes() { - if (changeMaxReadPositionAndAddAbortTimes.get() >= takeSnapshotIntervalNumber) { - this.changeMaxReadPositionAndAddAbortTimes.set(0); + if (changeMaxReadPositionCount.get() >= takeSnapshotIntervalNumber) { + this.changeMaxReadPositionCount.set(0); this.snapshotAbortedTxnProcessor.takeAbortedTxnsSnapshot(this.maxReadPosition); } } private void takeSnapshotByTimeout() { - if (changeMaxReadPositionAndAddAbortTimes.get() > 0) { - this.changeMaxReadPositionAndAddAbortTimes.set(0); + if (changeMaxReadPositionCount.get() > 0) { + this.changeMaxReadPositionCount.set(0); this.snapshotAbortedTxnProcessor.takeAbortedTxnsSnapshot(this.maxReadPosition); } this.timer.newTimeout(TopicTransactionBuffer.this, @@ -454,7 +454,7 @@ void updateMaxReadPosition(TxnID txnID) { maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry(); } if (preMaxReadPosition.compareTo(this.maxReadPosition) != 0) { - this.changeMaxReadPositionAndAddAbortTimes.getAndIncrement(); + this.changeMaxReadPositionCount.getAndIncrement(); } } @@ -489,7 +489,7 @@ public void syncMaxReadPositionForNormalPublish(PositionImpl position) { } else if (checkIfReady()) { if (ongoingTxns.isEmpty()) { maxReadPosition = position; - changeMaxReadPositionAndAddAbortTimes.incrementAndGet(); + changeMaxReadPositionCount.incrementAndGet(); } } } 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 f57ce7df2c5f0..1571844e89395 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 @@ -1052,10 +1052,10 @@ public void testCancelTxnTimeout() throws Exception{ } @Test - public void testNotChangeMaxReadPositionAndAddAbortTimesWhenCheckIfNoSnapshot() throws Exception { + public void testNotChangeMaxReadPositionCountWhenCheckIfNoSnapshot() throws Exception { PersistentTopic persistentTopic = (PersistentTopic) getPulsarServiceList().get(0) .getBrokerService() - .getTopic(NAMESPACE1 + "/changeMaxReadPositionAndAddAbortTimes" + UUID.randomUUID(), true) + .getTopic(NAMESPACE1 + "/changeMaxReadPositionCount" + UUID.randomUUID(), true) .get().get(); TransactionBuffer buffer = persistentTopic.getTransactionBuffer(); Field processorField = TopicTransactionBuffer.class.getDeclaredField("snapshotAbortedTxnProcessor"); @@ -1063,9 +1063,9 @@ public void testNotChangeMaxReadPositionAndAddAbortTimesWhenCheckIfNoSnapshot() AbortedTxnProcessor abortedTxnProcessor = (AbortedTxnProcessor) processorField.get(buffer); Field changeTimeField = TopicTransactionBuffer - .class.getDeclaredField("changeMaxReadPositionAndAddAbortTimes"); + .class.getDeclaredField("changeMaxReadPositionCount"); changeTimeField.setAccessible(true); - AtomicLong changeMaxReadPositionAndAddAbortTimes = (AtomicLong) changeTimeField.get(buffer); + AtomicLong changeMaxReadPositionCount = (AtomicLong) changeTimeField.get(buffer); Field field1 = TopicTransactionBufferState.class.getDeclaredField("state"); field1.setAccessible(true); @@ -1074,10 +1074,10 @@ public void testNotChangeMaxReadPositionAndAddAbortTimesWhenCheckIfNoSnapshot() TopicTransactionBufferState.State state = (TopicTransactionBufferState.State) field1.get(buffer); Assert.assertEquals(state, TopicTransactionBufferState.State.NoSnapshot); }); - Assert.assertEquals(changeMaxReadPositionAndAddAbortTimes.get(), 0L); + Assert.assertEquals(changeMaxReadPositionCount.get(), 0L); buffer.syncMaxReadPositionForNormalPublish(new PositionImpl(1, 1)); - Assert.assertEquals(changeMaxReadPositionAndAddAbortTimes.get(), 0L); + Assert.assertEquals(changeMaxReadPositionCount.get(), 0L); } From 99c2a9dccff1b392a6aabe63ff49c64560a18100 Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Mon, 13 May 2024 20:04:55 +0800 Subject: [PATCH 502/699] [fix][broker] fix replicated subscriptions for transactional messages (#22452) (cherry picked from commit 9fd1b61fc45d06348af0241f002966087f1822a0) (cherry picked from commit e300fbd72180e50ae049362499ed06bdcb0342ff) --- .../service/persistent/PersistentTopic.java | 26 +-- .../ReplicatedSubscriptionsController.java | 4 +- .../transaction/buffer/TransactionBuffer.java | 3 +- .../buffer/impl/InMemTransactionBuffer.java | 13 +- .../buffer/impl/TopicTransactionBuffer.java | 70 +++++-- .../buffer/impl/TransactionBufferDisable.java | 13 +- .../broker/service/PersistentTopicTest.java | 4 +- .../service/ReplicatorSubscriptionTest.java | 25 +++ ...ransactionalReplicateSubscriptionTest.java | 182 ++++++++++++++++++ .../transaction/TransactionProduceTest.java | 36 ++++ .../broker/transaction/TransactionTest.java | 2 +- 11 files changed, 343 insertions(+), 35 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionalReplicateSubscriptionTest.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 95935581315e3..55d6f6d70a1d9 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 @@ -124,6 +124,7 @@ import org.apache.pulsar.broker.stats.NamespaceStats; import org.apache.pulsar.broker.stats.ReplicationMetrics; import org.apache.pulsar.broker.transaction.buffer.TransactionBuffer; +import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBuffer; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferDisable; import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore; import org.apache.pulsar.client.admin.LongRunningProcessStatus; @@ -254,9 +255,13 @@ protected TopicStatsHelper initialValue() { @Getter protected final TransactionBuffer transactionBuffer; + @Getter + private final TopicTransactionBuffer.MaxReadPositionCallBack maxReadPositionCallBack = + (oldPosition, newPosition) -> updateMaxReadPositionMovedForwardTimestamp(); - // Record the last time a data message (ie: not an internal Pulsar marker) is published on the topic - private volatile long lastDataMessagePublishedTimestamp = 0; + // Record the last time max read position is moved forward, unless it's a marker message. + @Getter + private volatile long lastMaxReadPositionMovedForwardTimestamp = 0; @Getter private final ExecutorService orderedExecutor; @@ -352,7 +357,7 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS } else { this.transactionBuffer = new TransactionBufferDisable(this); } - transactionBuffer.syncMaxReadPositionForNormalPublish((PositionImpl) ledger.getLastConfirmedEntry()); + transactionBuffer.syncMaxReadPositionForNormalPublish((PositionImpl) ledger.getLastConfirmedEntry(), true); if (ledger instanceof ShadowManagedLedgerImpl) { shadowSourceTopic = TopicName.get(ledger.getConfig().getShadowSource()); } else { @@ -653,6 +658,10 @@ private void decrementPendingWriteOpsAndCheck() { } } + private void updateMaxReadPositionMovedForwardTimestamp() { + lastMaxReadPositionMovedForwardTimestamp = Clock.systemUTC().millis(); + } + @Override public void addComplete(Position pos, ByteBuf entryData, Object ctx) { PublishContext publishContext = (PublishContext) ctx; @@ -661,12 +670,9 @@ public void addComplete(Position pos, ByteBuf entryData, Object ctx) { // Message has been successfully persisted messageDeduplication.recordMessagePersisted(publishContext, position); - if (!publishContext.isMarkerMessage()) { - lastDataMessagePublishedTimestamp = Clock.systemUTC().millis(); - } - // in order to sync the max position when cursor read entries - transactionBuffer.syncMaxReadPositionForNormalPublish((PositionImpl) ledger.getLastConfirmedEntry()); + transactionBuffer.syncMaxReadPositionForNormalPublish((PositionImpl) ledger.getLastConfirmedEntry(), + publishContext.isMarkerMessage()); publishContext.setMetadataFromEntryData(entryData); publishContext.completed(null, position.getLedgerId(), position.getEntryId()); decrementPendingWriteOpsAndCheck(); @@ -3837,10 +3843,6 @@ private CompletableFuture transactionBufferCleanupAndClose() { return transactionBuffer.clearSnapshot().thenCompose(__ -> transactionBuffer.closeAsync()); } - public long getLastDataMessagePublishedTimestamp() { - return lastDataMessagePublishedTimestamp; - } - public Optional getShadowSourceTopic() { return Optional.ofNullable(shadowSourceTopic); } 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 e011ed8d660f6..3a796b3e96dd4 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 @@ -206,8 +206,8 @@ private void receiveSubscriptionUpdated(ReplicatedSubscriptionsUpdate update) { private void startNewSnapshot() { cleanupTimedOutSnapshots(); - if (topic.getLastDataMessagePublishedTimestamp() < lastCompletedSnapshotStartTime - || topic.getLastDataMessagePublishedTimestamp() == 0) { + if (topic.getLastMaxReadPositionMovedForwardTimestamp() < lastCompletedSnapshotStartTime + || topic.getLastMaxReadPositionMovedForwardTimestamp() == 0) { // There was no message written since the last snapshot, we can skip creating a new snapshot if (log.isDebugEnabled()) { log.debug("[{}] There is no new data in topic. Skipping snapshot creation.", topic.getName()); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBuffer.java index 7eb5d6f789c22..79eb263473f2f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/TransactionBuffer.java @@ -149,8 +149,9 @@ public interface TransactionBuffer { /** * Sync max read position for normal publish. * @param position {@link PositionImpl} the position to sync. + * @param isMarkerMessage whether the message is marker message. */ - void syncMaxReadPositionForNormalPublish(PositionImpl position); + void syncMaxReadPositionForNormalPublish(PositionImpl position, boolean isMarkerMessage); /** * Get the can read max position. diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java index 978536c5f4e36..bab7b64c608c4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java @@ -33,6 +33,7 @@ import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.service.Topic; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor; import org.apache.pulsar.broker.transaction.buffer.TransactionBuffer; import org.apache.pulsar.broker.transaction.buffer.TransactionBufferReader; @@ -213,11 +214,17 @@ public TransactionBufferReader newReader(long sequenceId) throws final ConcurrentMap buffers; final Map> txnIndex; private final Topic topic; + private final TopicTransactionBuffer.MaxReadPositionCallBack maxReadPositionCallBack; public InMemTransactionBuffer(Topic topic) { this.buffers = new ConcurrentHashMap<>(); this.txnIndex = new HashMap<>(); this.topic = topic; + if (topic instanceof PersistentTopic) { + this.maxReadPositionCallBack = ((PersistentTopic) topic).getMaxReadPositionCallBack(); + } else { + this.maxReadPositionCallBack = null; + } } @Override @@ -369,8 +376,10 @@ public boolean isTxnAborted(TxnID txnID, PositionImpl readPosition) { } @Override - public void syncMaxReadPositionForNormalPublish(PositionImpl position) { - //no-op + public void syncMaxReadPositionForNormalPublish(PositionImpl position, boolean isMarkerMessage) { + if (!isMarkerMessage && maxReadPositionCallBack != null) { + maxReadPositionCallBack.maxReadPositionMovedForward(null, position); + } } @Override 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 81c9ecfc728e9..cbf9630658f19 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TopicTransactionBuffer.java @@ -102,6 +102,7 @@ public class TopicTransactionBuffer extends TopicTransactionBufferState implemen private final AbortedTxnProcessor snapshotAbortedTxnProcessor; + private final MaxReadPositionCallBack maxReadPositionCallBack; private final AbortedTxnProcessor.SnapshotType snapshotType; public TopicTransactionBuffer(PersistentTopic topic) { @@ -120,6 +121,7 @@ public TopicTransactionBuffer(PersistentTopic topic) { snapshotAbortedTxnProcessor = new SingleSnapshotAbortedTxnProcessorImpl(topic); snapshotType = AbortedTxnProcessor.SnapshotType.Single; } + this.maxReadPositionCallBack = topic.getMaxReadPositionCallBack(); this.recover(); } @@ -175,7 +177,7 @@ public void handleTxnEntry(Entry entry) { if (Markers.isTxnAbortMarker(msgMetadata)) { snapshotAbortedTxnProcessor.putAbortedTxnAndPosition(txnID, position); } - updateMaxReadPosition(txnID); + removeTxnAndUpdateMaxReadPosition(txnID); } else { handleTransactionMessage(txnID, position); } @@ -290,7 +292,8 @@ private void handleTransactionMessage(TxnID txnId, Position position) { ongoingTxns.put(txnId, (PositionImpl) position); PositionImpl firstPosition = ongoingTxns.get(ongoingTxns.firstKey()); // max read position is less than first ongoing transaction message position - maxReadPosition = ((ManagedLedgerImpl) topic.getManagedLedger()).getPreviousPosition(firstPosition); + updateMaxReadPosition(((ManagedLedgerImpl) topic.getManagedLedger()).getPreviousPosition(firstPosition), + false); } } @@ -314,7 +317,7 @@ public CompletableFuture commitTxn(TxnID txnID, long lowWaterMark) { @Override public void addComplete(Position position, ByteBuf entryData, Object ctx) { synchronized (TopicTransactionBuffer.this) { - updateMaxReadPosition(txnID); + removeTxnAndUpdateMaxReadPosition(txnID); handleLowWaterMark(txnID, lowWaterMark); snapshotAbortedTxnProcessor.trimExpiredAbortedTxns(); takeSnapshotByChangeTimes(); @@ -361,7 +364,7 @@ public CompletableFuture abortTxn(TxnID txnID, long lowWaterMark) { public void addComplete(Position position, ByteBuf entryData, Object ctx) { synchronized (TopicTransactionBuffer.this) { snapshotAbortedTxnProcessor.putAbortedTxnAndPosition(txnID, (PositionImpl) position); - updateMaxReadPosition(txnID); + removeTxnAndUpdateMaxReadPosition(txnID); snapshotAbortedTxnProcessor.trimExpiredAbortedTxns(); takeSnapshotByChangeTimes(); txnAbortedCounter.increment(); @@ -444,17 +447,39 @@ private void takeSnapshotByTimeout() { takeSnapshotIntervalTime, TimeUnit.MILLISECONDS); } - void updateMaxReadPosition(TxnID txnID) { - PositionImpl preMaxReadPosition = this.maxReadPosition; + /** + * remove the specified transaction from ongoing transaction list and update the max read position. + * @param txnID + */ + void removeTxnAndUpdateMaxReadPosition(TxnID txnID) { ongoingTxns.remove(txnID); if (!ongoingTxns.isEmpty()) { PositionImpl position = ongoingTxns.get(ongoingTxns.firstKey()); - maxReadPosition = ((ManagedLedgerImpl) topic.getManagedLedger()).getPreviousPosition(position); + updateMaxReadPosition(((ManagedLedgerImpl) topic.getManagedLedger()).getPreviousPosition(position), false); } else { - maxReadPosition = (PositionImpl) topic.getManagedLedger().getLastConfirmedEntry(); + updateMaxReadPosition((PositionImpl) topic.getManagedLedger().getLastConfirmedEntry(), false); } - if (preMaxReadPosition.compareTo(this.maxReadPosition) != 0) { - this.changeMaxReadPositionCount.getAndIncrement(); + } + + /** + * update the max read position. if the new position is greater than the current max read position, + * we will trigger the callback, unless the disableCallback is true. + * Currently, we only use the callback to update the lastMaxReadPositionMovedForwardTimestamp. + * For non-transactional production, some marker messages will be sent to the topic, in which case we don't need + * to trigger the callback. + * @param newPosition new max read position to update. + * @param disableCallback whether disable the callback. + */ + void updateMaxReadPosition(PositionImpl newPosition, boolean disableCallback) { + PositionImpl preMaxReadPosition = this.maxReadPosition; + this.maxReadPosition = newPosition; + if (preMaxReadPosition.compareTo(this.maxReadPosition) < 0) { + if (!checkIfNoSnapshot()) { + this.changeMaxReadPositionCount.getAndIncrement(); + } + if (!disableCallback) { + maxReadPositionCallBack.maxReadPositionMovedForward(preMaxReadPosition, this.maxReadPosition); + } } } @@ -479,17 +504,22 @@ public synchronized boolean isTxnAborted(TxnID txnID, PositionImpl readPosition) return snapshotAbortedTxnProcessor.checkAbortedTransaction(txnID); } + /** + * Sync max read position for normal publish. + * @param position {@link PositionImpl} the position to sync. + * @param isMarkerMessage whether the message is marker message, in such case, we + * don't need to trigger the callback to update lastMaxReadPositionMovedForwardTimestamp. + */ @Override - public void syncMaxReadPositionForNormalPublish(PositionImpl position) { + public void syncMaxReadPositionForNormalPublish(PositionImpl position, boolean isMarkerMessage) { // when ongoing transaction is empty, proved that lastAddConfirm is can read max position, because callback // thread is the same tread, in this time the lastAddConfirm don't content transaction message. synchronized (TopicTransactionBuffer.this) { if (checkIfNoSnapshot()) { - this.maxReadPosition = position; + updateMaxReadPosition(position, isMarkerMessage); } else if (checkIfReady()) { if (ongoingTxns.isEmpty()) { - maxReadPosition = position; - changeMaxReadPositionCount.incrementAndGet(); + updateMaxReadPosition(position, isMarkerMessage); } } } @@ -674,6 +704,18 @@ private void closeReader(SystemTopicClient.Reader rea } } + /** + * A functional interface to handle the max read position move forward. + */ + public interface MaxReadPositionCallBack { + /** + * callback method when max read position move forward. + * @param oldPosition the old max read position. + * @param newPosition the new max read position. + */ + void maxReadPositionMovedForward(PositionImpl oldPosition, PositionImpl newPosition); + } + static class FillEntryQueueCallback implements AsyncCallbacks.ReadEntriesCallback { private final AtomicLong outstandingReadsRequests = new AtomicLong(0); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java index 9de0888ae5b0b..c1eb9d94a939a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java @@ -27,6 +27,7 @@ import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.transaction.buffer.TransactionBuffer; import org.apache.pulsar.broker.transaction.buffer.TransactionBufferReader; import org.apache.pulsar.broker.transaction.buffer.TransactionMeta; @@ -42,8 +43,14 @@ public class TransactionBufferDisable implements TransactionBuffer { private final Topic topic; + private final TopicTransactionBuffer.MaxReadPositionCallBack maxReadPositionCallBack; public TransactionBufferDisable(Topic topic) { this.topic = topic; + if (topic instanceof PersistentTopic) { + this.maxReadPositionCallBack = ((PersistentTopic) topic).getMaxReadPositionCallBack(); + } else { + this.maxReadPositionCallBack = null; + } } @Override @@ -91,8 +98,10 @@ public boolean isTxnAborted(TxnID txnID, PositionImpl readPosition) { } @Override - public void syncMaxReadPositionForNormalPublish(PositionImpl position) { - //no-op + public void syncMaxReadPositionForNormalPublish(PositionImpl position, boolean isMarkerMessage) { + if (!isMarkerMessage && maxReadPositionCallBack != null) { + maxReadPositionCallBack.maxReadPositionMovedForward(null, position); + } } @Override 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 fbcf787da91ae..bf6065a8b9e3a 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 @@ -291,6 +291,8 @@ public void testPublishMessage() throws Exception { }).when(ledgerMock).asyncAddEntry(any(ByteBuf.class), any(AddEntryCallback.class), any()); PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, brokerService); + long lastMaxReadPositionMovedForwardTimestamp = topic.getLastMaxReadPositionMovedForwardTimestamp(); + /* * MessageMetadata.Builder messageMetadata = MessageMetadata.newBuilder(); * messageMetadata.setPublishTime(System.currentTimeMillis()); messageMetadata.setProducerName("producer-name"); @@ -315,10 +317,10 @@ public void setMetadataFromEntryData(ByteBuf entryData) { assertEquals(entryData.array(), payload.array()); } }; - topic.publishMessage(payload, publishContext); assertTrue(latch.await(1, TimeUnit.SECONDS)); + assertTrue(topic.getLastMaxReadPositionMovedForwardTimestamp() > lastMaxReadPositionMovedForwardTimestamp); } @Test diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java index bfafdf89ed69b..870054ecaf641 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java @@ -26,6 +26,8 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; import com.google.common.collect.Sets; + +import java.lang.reflect.Field; import java.lang.reflect.Method; import java.nio.charset.StandardCharsets; import java.util.ArrayList; @@ -40,8 +42,10 @@ import lombok.Cleanup; import org.apache.bookkeeper.mledger.Position; import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.service.persistent.ReplicatedSubscriptionsController; +import org.apache.pulsar.broker.transaction.buffer.impl.TopicTransactionBufferState; import org.apache.pulsar.client.admin.LongRunningProcessStatus; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; @@ -728,6 +732,21 @@ public void testReplicatedSubscriptionRestApi3() throws Exception { consumer4.close(); } + /** + * before sending message, we should wait for transaction buffer recover complete, + * or the MaxReadPosition will not move forward when the message is sent, and the + * MaxReadPositionMovedForwardTimestamp will not be updated, then the replication will not be triggered. + * @param topicName + * @throws Exception + */ + private void waitTBRecoverComplete(PulsarService pulsarService, String topicName) throws Exception { + TopicTransactionBufferState buffer = (TopicTransactionBufferState) ((PersistentTopic) pulsarService.getBrokerService() + .getTopic(topicName, false).get().get()).getTransactionBuffer(); + Field stateField = TopicTransactionBufferState.class.getDeclaredField("state"); + stateField.setAccessible(true); + Awaitility.await().until(() -> !stateField.get(buffer).toString().equals("Initializing")); + } + /** * Tests replicated subscriptions when replicator producer is closed */ @@ -755,6 +774,9 @@ public void testReplicatedSubscriptionWhenReplicatorProducerIsClosed() throws Ex .subscribe(); // send one message to trigger replication + if (config1.isTransactionCoordinatorEnabled()) { + waitTBRecoverComplete(pulsar1, topicName); + } @Cleanup Producer producer = client1.newProducer().topic(topicName) .enableBatching(false) @@ -916,6 +938,9 @@ public void testReplicatedSubscriptionWithCompaction() throws Exception { .statsInterval(0, TimeUnit.SECONDS).build(); Producer producer = client.newProducer(Schema.STRING).topic(topicName).create(); + if (config1.isTransactionCoordinatorEnabled()) { + waitTBRecoverComplete(pulsar1, topicName); + } producer.newMessage().key("K1").value("V1").send(); producer.newMessage().key("K1").value("V2").send(); producer.close(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionalReplicateSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionalReplicateSubscriptionTest.java new file mode 100644 index 0000000000000..2d348f8259746 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TransactionalReplicateSubscriptionTest.java @@ -0,0 +1,182 @@ +/* + * 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.collect.Sets; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +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.transaction.Transaction; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.SystemTopicNames; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; +import org.apache.pulsar.metadata.api.MetadataStoreException; +import org.awaitility.Awaitility; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.util.Collections; +import java.util.LinkedHashSet; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; + +public class TransactionalReplicateSubscriptionTest extends ReplicatorTestBase { + @Override + @BeforeClass(timeOut = 300000) + public void setup() throws Exception { + super.setup(); + admin1.namespaces().createNamespace(NamespaceName.SYSTEM_NAMESPACE.toString()); + createTransactionCoordinatorAssign(16, pulsar1); + } + + @Override + @AfterClass(alwaysRun = true, timeOut = 300000) + public void cleanup() throws Exception { + super.cleanup(); + } + + /** + * enable transaction coordinator for the cluster1 + */ + @Override + public void setConfig1DefaultValue(){ + super.setConfig1DefaultValue(); + config1.setTransactionCoordinatorEnabled(true); + } + + protected void createTransactionCoordinatorAssign(int numPartitionsOfTC, PulsarService pulsarService) throws MetadataStoreException { + pulsarService.getPulsarResources() + .getNamespaceResources() + .getPartitionedTopicResources() + .createPartitionedTopic(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, + new PartitionedTopicMetadata(numPartitionsOfTC)); + } + + /** + * Test replicated subscription with transaction. + * @throws Exception + */ + @Test + public void testReplicatedSubscribeAndSwitchToStandbyClusterWithTransaction() throws Exception { + final String namespace = BrokerTestUtil.newUniqueName("pulsar/ns_"); + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + namespace + "/tp_"); + final String subscriptionName = "s1"; + final boolean isReplicatedSubscription = true; + final int messagesCount = 20; + final LinkedHashSet sentMessages = new LinkedHashSet<>(); + final Set receivedMessages = Collections.synchronizedSet(new LinkedHashSet<>()); + admin1.namespaces().createNamespace(namespace); + admin1.namespaces().setNamespaceReplicationClusters(namespace, Sets.newHashSet("r1", "r2")); + admin1.topics().createNonPartitionedTopic(topicName); + admin1.topics().createSubscription(topicName, subscriptionName, MessageId.earliest, isReplicatedSubscription); + final PersistentTopic topic1 = + (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get(); + + // Send messages + // Wait for the topic created on the cluster2. + // Wait for the snapshot created. + final PulsarClient client1 = PulsarClient.builder().serviceUrl(url1.toString()).enableTransaction(true).build(); + Producer producer1 = client1.newProducer(Schema.STRING).topic(topicName).enableBatching(false).create(); + Consumer consumer1 = client1.newConsumer(Schema.STRING).topic(topicName) + .subscriptionName(subscriptionName).replicateSubscriptionState(isReplicatedSubscription).subscribe(); + Transaction txn1 = client1.newTransaction() + .withTransactionTimeout(5, TimeUnit.SECONDS) + .build().get(); + for (int i = 0; i < messagesCount / 2; i++) { + String msg = i + ""; + producer1.newMessage(txn1).value(msg).send(); + sentMessages.add(msg); + } + txn1.commit().get(); + Awaitility.await().untilAsserted(() -> { + ConcurrentOpenHashMap replicators = topic1.getReplicators(); + assertTrue(replicators != null && replicators.size() == 1, "Replicator should started"); + assertTrue(replicators.values().iterator().next().isConnected(), "Replicator should be connected"); + assertTrue(topic1.getReplicatedSubscriptionController().get().getLastCompletedSnapshotId().isPresent(), + "One snapshot should be finished"); + }); + final PersistentTopic topic2 = + (PersistentTopic) pulsar2.getBrokerService().getTopic(topicName, false).join().get(); + Awaitility.await().untilAsserted(() -> { + assertTrue(topic2.getReplicatedSubscriptionController().isPresent(), + "Replicated subscription controller should created"); + }); + Transaction txn2 = client1.newTransaction() + .withTransactionTimeout(5, TimeUnit.SECONDS) + .build().get(); + for (int i = messagesCount / 2; i < messagesCount; i++) { + String msg = i + ""; + producer1.newMessage(txn2).value(msg).send(); + sentMessages.add(msg); + } + txn2.commit().get(); + + // Consume half messages and wait the subscription created on the cluster2. + for (int i = 0; i < messagesCount / 2; i++){ + Message message = consumer1.receive(2, TimeUnit.SECONDS); + if (message == null) { + fail("Should not receive null."); + } + receivedMessages.add(message.getValue()); + consumer1.acknowledge(message); + } + Awaitility.await().untilAsserted(() -> { + assertNotNull(topic2.getSubscriptions().get(subscriptionName), "Subscription should created"); + }); + + // Switch client to cluster2. + // Since the cluster1 was not crash, all messages will be replicated to the cluster2. + consumer1.close(); + final PulsarClient client2 = PulsarClient.builder().serviceUrl(url2.toString()).build(); + final Consumer consumer2 = client2.newConsumer(Schema.AUTO_CONSUME()).topic(topicName) + .subscriptionName(subscriptionName).replicateSubscriptionState(isReplicatedSubscription).subscribe(); + + // Verify all messages will be consumed. + Awaitility.await().untilAsserted(() -> { + while (true) { + Message message = consumer2.receive(2, TimeUnit.SECONDS); + if (message != null) { + receivedMessages.add(message.getValue().toString()); + consumer2.acknowledge(message); + } else { + break; + } + } + assertEquals(receivedMessages.size(), sentMessages.size()); + }); + + consumer2.close(); + producer1.close(); + client1.close(); + client2.close(); + } +} 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 ddd8cf0790321..146cf0cc4deb3 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 @@ -19,11 +19,14 @@ package org.apache.pulsar.broker.transaction; import static java.nio.charset.StandardCharsets.UTF_8; +import static org.testng.Assert.assertTrue; + import java.lang.reflect.Field; import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; @@ -38,7 +41,9 @@ import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.tuple.MutablePair; import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.broker.transaction.pendingack.impl.PendingAckHandleImpl; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -180,6 +185,37 @@ private void produceTest(boolean endAction) throws Exception { log.info("produce and {} test finished.", endAction ? "commit" : "abort"); } + @Test + public void testUpdateLastMaxReadPositionMovedForwardTimestampForTransactionalPublish() throws Exception { + final String topic = NAMESPACE1 + "/testUpdateLastMaxReadPositionMovedForwardTimestampForTransactionalPublish"; + PulsarClient pulsarClient = this.pulsarClient; + Transaction txn = pulsarClient.newTransaction() + .withTransactionTimeout(5, TimeUnit.SECONDS) + .build().get(); + @Cleanup + Producer producer = pulsarClient + .newProducer() + .topic(topic) + .sendTimeout(0, TimeUnit.SECONDS) + .create(); + PersistentTopic persistentTopic = getTopic(topic); + long lastMaxReadPositionMovedForwardTimestamp = persistentTopic.getLastMaxReadPositionMovedForwardTimestamp(); + + // transactional publish will not update lastMaxReadPositionMovedForwardTimestamp + producer.newMessage(txn).value("hello world".getBytes()).send(); + assertTrue(persistentTopic.getLastMaxReadPositionMovedForwardTimestamp() == lastMaxReadPositionMovedForwardTimestamp); + + // commit transaction will update lastMaxReadPositionMovedForwardTimestamp + txn.commit().get(); + assertTrue(persistentTopic.getLastMaxReadPositionMovedForwardTimestamp() > lastMaxReadPositionMovedForwardTimestamp); + } + + private PersistentTopic getTopic(String topic) throws ExecutionException, InterruptedException { + Optional optionalTopic = getPulsarServiceList().get(0).getBrokerService() + .getTopic(topic, true).get(); + return (PersistentTopic) optionalTopic.get(); + } + private void checkMessageId(List> futureList, boolean isFinished) { futureList.forEach(messageIdFuture -> { try { 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 1571844e89395..80063f207002f 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 @@ -1076,7 +1076,7 @@ public void testNotChangeMaxReadPositionCountWhenCheckIfNoSnapshot() throws Exce }); Assert.assertEquals(changeMaxReadPositionCount.get(), 0L); - buffer.syncMaxReadPositionForNormalPublish(new PositionImpl(1, 1)); + buffer.syncMaxReadPositionForNormalPublish(new PositionImpl(1, 1), false); Assert.assertEquals(changeMaxReadPositionCount.get(), 0L); } From 53f0b91d89a7fcf5ace854f330890b6e9ab4bf8c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 12 Apr 2024 10:09:54 -0700 Subject: [PATCH 503/699] [fix][test] Fix NPE in BookKeeperClusterTestCase tearDown (#22493) (cherry picked from commit d1748573f1cb294838b69b5d80af672c3ee9e453) (cherry picked from commit 4bb0d8135c1194d469575764b1e96f9f9f56d8be) --- .../apache/bookkeeper/test/BookKeeperClusterTestCase.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java b/managed-ledger/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java index 80bb6256591bc..e316083e837fd 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java @@ -86,7 +86,7 @@ public abstract class BookKeeperClusterTestCase { protected String testName; - @BeforeMethod + @BeforeMethod(alwaysRun = true) public void handleTestMethodName(Method method) { testName = method.getName(); } @@ -148,7 +148,7 @@ public BookKeeperClusterTestCase(int numBookies, int numOfZKNodes, int testTimeo } } - @BeforeTest + @BeforeTest(alwaysRun = true) public void setUp() throws Exception { setUp(getLedgersRootPath()); } @@ -222,7 +222,9 @@ public void tearDown() throws Exception { tearDownException = e; } - executor.shutdownNow(); + if (executor != null) { + executor.shutdownNow(); + } LOG.info("Tearing down test {} in {} ms.", testName, sw.elapsed(TimeUnit.MILLISECONDS)); if (tearDownException != null) { From 4fa1c21954b50049b9fce41170ee3b49997d1b4b Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 14 May 2024 08:47:00 +0300 Subject: [PATCH 504/699] [improve][ci][branch-3.0] Upgrade actions in pulsar-ci and pulsar-ci-flaky, port owasp cache change (cherry picked from commit db40c8f31a8eeb8d8f5e7c4daca6234cfea116ac) --- .github/actions/upload-coverage/action.yml | 8 +- .github/changes-filter.yaml | 4 +- .github/workflows/ci-go-functions.yaml | 6 +- .github/workflows/pulsar-ci-flaky.yaml | 14 +-- .github/workflows/pulsar-ci.yaml | 126 ++++++++++++--------- 5 files changed, 90 insertions(+), 68 deletions(-) diff --git a/.github/actions/upload-coverage/action.yml b/.github/actions/upload-coverage/action.yml index a9706e77333cb..0ba73e94a8389 100644 --- a/.github/actions/upload-coverage/action.yml +++ b/.github/actions/upload-coverage/action.yml @@ -51,7 +51,7 @@ runs: - name: "Upload to Codecov (attempt #1)" id: codecov-upload-1 if: steps.repo-check.outputs.passed == 'true' - uses: codecov/codecov-action@v3 + uses: codecov/codecov-action@v4 continue-on-error: true with: flags: ${{ inputs.flags }} @@ -64,7 +64,7 @@ runs: - name: "Upload to Codecov (attempt #2)" id: codecov-upload-2 if: steps.codecov-upload-1.outcome == 'failure' - uses: codecov/codecov-action@v3 + uses: codecov/codecov-action@v4 continue-on-error: true with: flags: ${{ inputs.flags }} @@ -77,7 +77,7 @@ runs: - name: "Upload to Codecov (attempt #3)" id: codecov-upload-3 if: steps.codecov-upload-2.outcome == 'failure' - uses: codecov/codecov-action@v3 + uses: codecov/codecov-action@v4 # fail on last attempt continue-on-error: false with: @@ -97,4 +97,4 @@ runs: [Code coverage report](https://app.codecov.io/github/$GITHUB_REPOSITORY/commit/${head_sha}/tree) - EOF \ No newline at end of file + EOF diff --git a/.github/changes-filter.yaml b/.github/changes-filter.yaml index 250ebf692f6e6..66e5db32d4c47 100644 --- a/.github/changes-filter.yaml +++ b/.github/changes-filter.yaml @@ -12,10 +12,12 @@ docs: - 'deployment/**' - 'wiki/**' - 'pip/**' +java_non_tests: + - '**/src/main/java/**/*.java' tests: - added|modified: '**/src/test/java/**/*.java' need_owasp: - 'pom.xml' - '**/pom.xml' - 'src/owasp-dependency-check-false-positives.xml' - - 'src/owasp-dependency-check-suppressions.xml' \ No newline at end of file + - 'src/owasp-dependency-check-suppressions.xml' diff --git a/.github/workflows/ci-go-functions.yaml b/.github/workflows/ci-go-functions.yaml index 08c287b04b557..d7daf012f686f 100644 --- a/.github/workflows/ci-go-functions.yaml +++ b/.github/workflows/ci-go-functions.yaml @@ -43,7 +43,7 @@ jobs: docs_only: ${{ steps.check_changes.outputs.docs_only }} steps: - name: checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Detect changed files id: changes @@ -80,13 +80,13 @@ jobs: steps: - name: Check out code into the Go module directory - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm - name: Set up Go - uses: actions/setup-go@v2 + uses: actions/setup-go@v5 with: go-version: ${{ matrix.go-version }} id: go diff --git a/.github/workflows/pulsar-ci-flaky.yaml b/.github/workflows/pulsar-ci-flaky.yaml index 86fc41f0d8d10..27d2e9895ef7e 100644 --- a/.github/workflows/pulsar-ci-flaky.yaml +++ b/.github/workflows/pulsar-ci-flaky.yaml @@ -54,7 +54,7 @@ jobs: collect_coverage: ${{ steps.check_coverage.outputs.collect_coverage }} steps: - name: checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Detect changed files id: changes @@ -101,7 +101,7 @@ jobs: if: ${{ needs.preconditions.outputs.docs_only != 'true' }} steps: - name: checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm @@ -115,7 +115,7 @@ jobs: limit-access-to-actor: true - name: Cache local Maven repository - uses: actions/cache@v3 + uses: actions/cache@v4 timeout-minutes: 5 with: path: | @@ -126,7 +126,7 @@ jobs: ${{ runner.os }}-m2-dependencies-core-modules- - name: Set up JDK 17 - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: 17 @@ -156,7 +156,7 @@ jobs: - name: Upload Jacoco report files to build artifacts if: ${{ needs.preconditions.outputs.collect_coverage == 'true' }} - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: Jacoco-coverage-report-flaky path: target/jacoco_test_coverage_report_flaky.zip @@ -176,7 +176,7 @@ jobs: annotate_only: 'true' - name: Upload Surefire reports - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: Unit-BROKER_FLAKY-surefire-reports @@ -184,7 +184,7 @@ jobs: retention-days: 7 - name: Upload possible heap dump - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ always() }} with: name: Unit-BROKER_FLAKY-heapdump diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 5bba36c1d9caa..2ba58ba102bf9 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -56,7 +56,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Detect changed files id: changes @@ -102,7 +102,7 @@ jobs: if: ${{ needs.preconditions.outputs.docs_only != 'true' }} steps: - name: checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm @@ -116,7 +116,7 @@ jobs: limit-access-to-actor: true - name: Cache local Maven repository - uses: actions/cache@v3 + uses: actions/cache@v4 timeout-minutes: 5 with: path: | @@ -127,7 +127,7 @@ jobs: ${{ runner.os }}-m2-dependencies-core-modules- - name: Set up JDK 17 - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: 17 @@ -210,7 +210,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm @@ -224,7 +224,7 @@ jobs: limit-access-to-actor: true - name: Cache Maven dependencies - uses: actions/cache@v3 + uses: actions/cache@v4 timeout-minutes: 5 with: path: | @@ -235,7 +235,7 @@ jobs: ${{ runner.os }}-m2-dependencies-core-modules- - name: Set up JDK ${{ matrix.jdk || '17' }} - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: ${{ matrix.jdk || '17' }} @@ -277,7 +277,7 @@ jobs: annotate_only: 'true' - name: Upload Surefire reports - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: Unit-${{ matrix.group }}-surefire-reports @@ -285,7 +285,7 @@ jobs: retention-days: 7 - name: Upload possible heap dump, core dump or crash files - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ always() }} with: name: Unit-${{ matrix.group }}-dumps @@ -314,7 +314,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm @@ -328,7 +328,7 @@ jobs: limit-access-to-actor: true - name: Cache Maven dependencies - uses: actions/cache@v3 + uses: actions/cache@v4 timeout-minutes: 5 with: path: | @@ -339,7 +339,7 @@ jobs: ${{ runner.os }}-m2-dependencies-core-modules- - name: Set up JDK ${{ matrix.jdk || '17' }} - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: ${{ matrix.jdk || '17' }} @@ -361,7 +361,7 @@ jobs: zip -qr jacoco_test_coverage_report_unittests.zip jacoco_test_coverage_report || true - name: Upload Jacoco report files to build artifacts - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: Jacoco-coverage-report-unittests path: target/jacoco_test_coverage_report_unittests.zip @@ -394,7 +394,7 @@ jobs: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} steps: - name: checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm @@ -408,7 +408,7 @@ jobs: limit-access-to-actor: true - name: Cache Maven dependencies - uses: actions/cache@v3 + uses: actions/cache@v4 timeout-minutes: 5 with: path: | @@ -419,7 +419,7 @@ jobs: ${{ runner.os }}-m2-dependencies-core-modules- - name: Set up JDK 17 - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: 17 @@ -509,7 +509,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm @@ -523,7 +523,7 @@ jobs: limit-access-to-actor: true - name: Cache Maven dependencies - uses: actions/cache@v3 + uses: actions/cache@v4 timeout-minutes: 5 with: path: | @@ -534,7 +534,7 @@ jobs: ${{ runner.os }}-m2-dependencies-core-modules- - name: Set up JDK 17 - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: 17 @@ -558,7 +558,7 @@ jobs: ${{ matrix.setup }} - name: Set up runtime JDK ${{ matrix.runtime_jdk }} - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 if: ${{ matrix.runtime_jdk }} with: distribution: 'temurin' @@ -591,7 +591,7 @@ jobs: annotate_only: 'true' - name: Upload Surefire reports - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: Integration-${{ matrix.group }}-surefire-reports @@ -599,7 +599,7 @@ jobs: retention-days: 7 - name: Upload container logs - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} continue-on-error: true with: @@ -625,7 +625,7 @@ jobs: PULSAR_TEST_IMAGE_NAME: apachepulsar/java-test-image:latest steps: - name: checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm @@ -639,7 +639,7 @@ jobs: limit-access-to-actor: true - name: Cache Maven dependencies - uses: actions/cache@v3 + uses: actions/cache@v4 timeout-minutes: 5 with: path: | @@ -650,7 +650,7 @@ jobs: ${{ runner.os }}-m2-dependencies-core-modules- - name: Set up JDK 17 - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: 17 @@ -677,7 +677,7 @@ jobs: zip -qr jacoco_test_coverage_report_inttests.zip jacoco_test_coverage_report jacoco_inttest_coverage_report || true - name: Upload Jacoco report files to build artifacts - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: Jacoco-coverage-report-inttests path: target/jacoco_test_coverage_report_inttests.zip @@ -712,7 +712,7 @@ jobs: if: ${{ needs.preconditions.outputs.docs_only != 'true' }} steps: - name: checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm @@ -734,7 +734,7 @@ jobs: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} steps: - name: checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm @@ -753,7 +753,7 @@ jobs: mode: full - name: Cache local Maven repository - uses: actions/cache@v3 + uses: actions/cache@v4 timeout-minutes: 5 with: path: | @@ -765,7 +765,7 @@ jobs: ${{ runner.os }}-m2-dependencies-core-modules- - name: Set up JDK 17 - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: 17 @@ -873,7 +873,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm @@ -891,7 +891,7 @@ jobs: limit-access-to-actor: true - name: Cache local Maven repository - uses: actions/cache@v3 + uses: actions/cache@v4 timeout-minutes: 5 with: path: | @@ -903,7 +903,7 @@ jobs: ${{ runner.os }}-m2-dependencies-core-modules- - name: Set up JDK 17 - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: 17 @@ -953,7 +953,7 @@ jobs: annotate_only: 'true' - name: Upload container logs - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} continue-on-error: true with: @@ -962,7 +962,7 @@ jobs: retention-days: 7 - name: Upload Surefire reports - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: System-${{ matrix.name }}-surefire-reports @@ -988,7 +988,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm @@ -1002,7 +1002,7 @@ jobs: limit-access-to-actor: true - name: Cache local Maven repository - uses: actions/cache@v3 + uses: actions/cache@v4 timeout-minutes: 5 with: path: | @@ -1014,7 +1014,7 @@ jobs: ${{ runner.os }}-m2-dependencies-core-modules- - name: Set up JDK 17 - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: 17 @@ -1040,7 +1040,7 @@ jobs: zip -qr jacoco_test_coverage_report_systests.zip jacoco_test_coverage_report jacoco_inttest_coverage_report || true - name: Upload Jacoco report files to build artifacts - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 with: name: Jacoco-coverage-report-systests path: target/jacoco_test_coverage_report_systests.zip @@ -1086,7 +1086,7 @@ jobs: steps: - name: checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm @@ -1104,7 +1104,7 @@ jobs: limit-access-to-actor: true - name: Cache local Maven repository - uses: actions/cache@v3 + uses: actions/cache@v4 timeout-minutes: 5 with: path: | @@ -1116,7 +1116,7 @@ jobs: ${{ runner.os }}-m2-dependencies-core-modules- - name: Set up JDK 17 - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: 17 @@ -1158,7 +1158,7 @@ jobs: annotate_only: 'true' - name: Upload container logs - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} continue-on-error: true with: @@ -1167,7 +1167,7 @@ jobs: retention-days: 7 - name: Upload Surefire reports - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ !success() }} with: name: System-${{ matrix.name }}-surefire-reports @@ -1195,7 +1195,7 @@ jobs: if: ${{ needs.preconditions.outputs.docs_only != 'true' }} steps: - name: checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm @@ -1217,13 +1217,13 @@ jobs: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} steps: - name: checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm - name: Cache Maven dependencies - uses: actions/cache@v3 + uses: actions/cache@v4 timeout-minutes: 5 with: path: | @@ -1234,7 +1234,7 @@ jobs: ${{ runner.os }}-m2-dependencies-all- - name: Set up JDK 17 - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: 17 @@ -1252,7 +1252,7 @@ jobs: GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} steps: - name: checkout - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Tune Runner VM uses: ./.github/actions/tune-runner-vm @@ -1266,17 +1266,19 @@ jobs: limit-access-to-actor: true - name: Cache Maven dependencies - uses: actions/cache@v3 + uses: actions/cache@v4 timeout-minutes: 5 with: path: | ~/.m2/repository/*/*/* !~/.m2/repository/org/apache/pulsar + !~/.m2/repository/org/owasp/dependency-check-data key: ${{ runner.os }}-m2-dependencies-core-modules-${{ hashFiles('**/pom.xml') }} + lookup-only: true restore-keys: | ${{ runner.os }}-m2-dependencies-core-modules- - name: Set up JDK ${{ matrix.jdk || '17' }} - uses: actions/setup-java@v3 + uses: actions/setup-java@v4 with: distribution: 'temurin' java-version: ${{ matrix.jdk || '17' }} @@ -1291,6 +1293,24 @@ jobs: run: | cd $HOME $GITHUB_WORKSPACE/build/pulsar_ci_tool.sh restore_tar_from_github_actions_artifacts pulsar-maven-repository-binaries + + - name: OWASP cache key weeknum + id: get-weeknum + run: | + echo "weeknum=$(date -u +"%Y-%U")" >> $GITHUB_OUTPUT + shell: bash + + - name: Restore OWASP Dependency Check data + id: restore-owasp-dependency-check-data + uses: actions/cache/restore@v4 + timeout-minutes: 5 + with: + path: ~/.m2/repository/org/owasp/dependency-check-data + key: owasp-dependency-check-data-${{ steps.get-weeknum.outputs.weeknum }} + enableCrossOsArchive: true + restore-keys: | + owasp-dependency-check-data- + # Projects dependent on flume, hdfs, hbase, and presto currently excluded from the scan. - name: trigger dependency check run: | @@ -1298,7 +1318,7 @@ jobs: -pl '!pulsar-sql,!distribution/server,!distribution/io,!distribution/offloaders,!pulsar-sql/presto-distribution,!tiered-storage/file-system,!pulsar-io/flume,!pulsar-io/hbase,!pulsar-io/hdfs2,!pulsar-io/hdfs3,!pulsar-io/docs,!pulsar-io/jdbc/openmldb' - name: Upload report - uses: actions/upload-artifact@v3 + uses: actions/upload-artifact@v4 if: ${{ cancelled() || failure() }} continue-on-error: true with: @@ -1349,7 +1369,7 @@ jobs: - name: checkout if: ${{ needs.preconditions.outputs.docs_only != 'true' }} - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Tune Runner VM if: ${{ needs.preconditions.outputs.docs_only != 'true' }} From f7a36284f15afc5f2b2d728e815544e742b365f2 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Wed, 15 May 2024 12:02:38 +0530 Subject: [PATCH 505/699] fix checkstyle errors --- .../transaction/buffer/impl/TransactionBufferDisable.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java index c1eb9d94a939a..ebd61dbaa82ec 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java @@ -26,8 +26,8 @@ import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.Topic; -import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor; import org.apache.pulsar.broker.transaction.buffer.TransactionBuffer; import org.apache.pulsar.broker.transaction.buffer.TransactionBufferReader; import org.apache.pulsar.broker.transaction.buffer.TransactionMeta; From 08f270b0baee2feefd4e4a3b5a456d5957efd8f0 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Mon, 20 May 2024 11:03:51 +0530 Subject: [PATCH 506/699] Updated license version for Datastax --- .../server/src/assemble/LICENSE.bin.txt | 44 +++++++++---------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index cc79cbe9f5f17..7cd88d22d1231 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -585,28 +585,28 @@ Bouncy Castle License - org.bouncycastle-bcutil-jdk18on-1.78.jar Datastax - - com.datastax.oss-managed-ledger-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-auth-oidc-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-auth-sasl-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-common-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-pulsar-config-validation-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-pulsar-docs-tools-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-api-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-instance-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-local-runner-original-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-proto-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-runtime-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-secrets-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-utils-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-worker-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-pulsar-io-core-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-pulsar-metadata-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-pulsar-proxy-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-pulsar-testclient-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-pulsar-transaction-common-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-pulsar-transaction-coordinator-3.1.4.1-SNAPSHOT.jar - - com.datastax.oss-testmocks-3.1.4.1-SNAPSHOT.jar + - com.datastax.oss-managed-ledger-3.1.4.1.jar + - com.datastax.oss-pulsar-broker-3.1.4.1.jar + - com.datastax.oss-pulsar-broker-auth-oidc-3.1.4.1.jar + - com.datastax.oss-pulsar-broker-auth-sasl-3.1.4.1.jar + - com.datastax.oss-pulsar-broker-common-3.1.4.1.jar + - com.datastax.oss-pulsar-config-validation-3.1.4.1.jar + - com.datastax.oss-pulsar-docs-tools-3.1.4.1.jar + - com.datastax.oss-pulsar-functions-api-3.1.4.1.jar + - com.datastax.oss-pulsar-functions-instance-3.1.4.1.jar + - com.datastax.oss-pulsar-functions-local-runner-original-3.1.4.1.jar + - com.datastax.oss-pulsar-functions-proto-3.1.4.1.jar + - com.datastax.oss-pulsar-functions-runtime-3.1.4.1.jar + - com.datastax.oss-pulsar-functions-secrets-3.1.4.1.jar + - com.datastax.oss-pulsar-functions-utils-3.1.4.1.jar + - com.datastax.oss-pulsar-functions-worker-3.1.4.1.jar + - com.datastax.oss-pulsar-io-core-3.1.4.1.jar + - com.datastax.oss-pulsar-metadata-3.1.4.1.jar + - com.datastax.oss-pulsar-proxy-3.1.4.1.jar + - com.datastax.oss-pulsar-testclient-3.1.4.1.jar + - com.datastax.oss-pulsar-transaction-common-3.1.4.1.jar + - com.datastax.oss-pulsar-transaction-coordinator-3.1.4.1.jar + - com.datastax.oss-testmocks-3.1.4.1.jar ------------------------ From 9bcbd6fd28c0bb73421f28b4e093c9f62c8ce0bb Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Mon, 20 May 2024 11:28:13 +0530 Subject: [PATCH 507/699] Release 3.1.4.1 --- 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-experimental/pom.xml | 2 +- docker/pulsar/pom.xml | 2 +- jclouds-shaded/pom.xml | 2 +- managed-ledger/pom.xml | 2 +- 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-client-1x-base/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-1x/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-2x-shaded/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/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/hdfs2/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-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-sql/pom.xml | 2 +- pulsar-sql/presto-distribution/pom.xml | 2 +- pulsar-sql/presto-pulsar-plugin/pom.xml | 2 +- pulsar-sql/presto-pulsar/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 +- 134 files changed, 136 insertions(+), 136 deletions(-) diff --git a/bouncy-castle/bc/pom.xml b/bouncy-castle/bc/pom.xml index 9edc27e80a6e4..d7bacb975a255 100644 --- a/bouncy-castle/bc/pom.xml +++ b/bouncy-castle/bc/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.4.1-SNAPSHOT + 3.1.4.1 .. diff --git a/bouncy-castle/bcfips-include-test/pom.xml b/bouncy-castle/bcfips-include-test/pom.xml index c129f375db65e..16730c7eeb88f 100644 --- a/bouncy-castle/bcfips-include-test/pom.xml +++ b/bouncy-castle/bcfips-include-test/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.4.1-SNAPSHOT + 3.1.4.1 .. diff --git a/bouncy-castle/bcfips/pom.xml b/bouncy-castle/bcfips/pom.xml index a12b981da6ad4..b7b6bd443be60 100644 --- a/bouncy-castle/bcfips/pom.xml +++ b/bouncy-castle/bcfips/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.4.1-SNAPSHOT + 3.1.4.1 .. diff --git a/bouncy-castle/pom.xml b/bouncy-castle/pom.xml index 6b6a1f1bdea40..df8be83d25759 100644 --- a/bouncy-castle/pom.xml +++ b/bouncy-castle/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.4.1-SNAPSHOT + 3.1.4.1 .. diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 97a79a0211dd1..13b458619a867 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -31,12 +31,12 @@ com.datastax.oss buildtools - 3.1.4.1-SNAPSHOT + 3.1.4.1 jar Pulsar Build Tools - 2024-04-23T11:05:03Z + 2024-05-20T05:58:12Z 1.8 1.8 3.1.0 diff --git a/distribution/io/pom.xml b/distribution/io/pom.xml index e59a62d0352d0..b59fd8e5463a2 100644 --- a/distribution/io/pom.xml +++ b/distribution/io/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.4.1-SNAPSHOT + 3.1.4.1 .. diff --git a/distribution/offloaders/pom.xml b/distribution/offloaders/pom.xml index 105fb50df1033..5dfbcecda4af4 100644 --- a/distribution/offloaders/pom.xml +++ b/distribution/offloaders/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.4.1-SNAPSHOT + 3.1.4.1 .. diff --git a/distribution/pom.xml b/distribution/pom.xml index 5da77d2435a86..748d79cd9a5f0 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.4.1-SNAPSHOT + 3.1.4.1 .. diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index bef1a4a887b25..243df8fcf1eb8 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.4.1-SNAPSHOT + 3.1.4.1 .. diff --git a/distribution/shell/pom.xml b/distribution/shell/pom.xml index ea55b28be913e..58befd81a46d5 100644 --- a/distribution/shell/pom.xml +++ b/distribution/shell/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.4.1-SNAPSHOT + 3.1.4.1 .. diff --git a/docker/pom.xml b/docker/pom.xml index b2f42c3b8e755..d37434ccb94c2 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.4.1-SNAPSHOT + 3.1.4.1 docker-images Apache Pulsar :: Docker Images diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index ad21cbf4a6063..89f73e754da1c 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.4.1-SNAPSHOT + 3.1.4.1 4.0.0 pulsar-all-docker-image diff --git a/docker/pulsar-experimental/pom.xml b/docker/pulsar-experimental/pom.xml index d5ceef340cbee..d3153dfb4e411 100644 --- a/docker/pulsar-experimental/pom.xml +++ b/docker/pulsar-experimental/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.4.1-SNAPSHOT + 3.1.4.1 4.0.0 pulsar-experimental-docker-image diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index bbfef440e1ab7..c037740e8e9bb 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.4.1-SNAPSHOT + 3.1.4.1 4.0.0 pulsar-docker-image diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml index 785bb7c7256fb..a3f5fc56e8dea 100644 --- a/jclouds-shaded/pom.xml +++ b/jclouds-shaded/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.4.1-SNAPSHOT + 3.1.4.1 .. diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index c5d2761df8cb7..1fc1639f5db72 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.4.1-SNAPSHOT + 3.1.4.1 .. diff --git a/pom.xml b/pom.xml index df8e523539800..354f9f38d5b57 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ com.datastax.oss pulsar - 3.1.4.1-SNAPSHOT + 3.1.4.1 Pulsar Pulsar is a distributed pub-sub messaging platform with a very @@ -95,7 +95,7 @@ flexible messaging model and an intuitive client API. UTF-8 UTF-8 - 2024-04-23T11:05:03Z + 2024-05-20T05:58:12Z true 1.18.3 2.2 + 5.4.0 3.3.0 @@ -1452,6 +1453,12 @@ flexible messaging model and an intuitive client API. oshi-core-java11 ${oshi.version} + + io.rest-assured + rest-assured + ${restassured.version} + test + diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index 76d902a253aeb..b33ebab872cfe 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -431,6 +431,12 @@ javax.activation + + io.rest-assured + rest-assured + test + + diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java index 6ad1697adfc39..c889062088e00 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BacklogQuotaManager.java @@ -18,10 +18,12 @@ */ package org.apache.pulsar.broker.service; +import static java.util.concurrent.TimeUnit.SECONDS; import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; +import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedCursor.IndividualDeletedEntries; @@ -32,6 +34,7 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.resources.NamespaceResources; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.service.persistent.PersistentTopicMetrics.BacklogQuotaMetrics; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; @@ -41,6 +44,7 @@ @Slf4j public class BacklogQuotaManager { + @Getter private final BacklogQuotaImpl defaultQuota; private final NamespaceResources namespaceResources; @@ -55,10 +59,6 @@ public BacklogQuotaManager(PulsarService pulsar) { this.namespaceResources = pulsar.getPulsarResources().getNamespaceResources(); } - public BacklogQuotaImpl getDefaultQuota() { - return this.defaultQuota; - } - public BacklogQuotaImpl getBacklogQuota(NamespaceName namespace, BacklogQuotaType backlogQuotaType) { try { if (namespaceResources == null) { @@ -86,30 +86,34 @@ public BacklogQuotaImpl getBacklogQuota(NamespaceName namespace, BacklogQuotaTyp public void handleExceededBacklogQuota(PersistentTopic persistentTopic, BacklogQuotaType backlogQuotaType, boolean preciseTimeBasedBacklogQuotaCheck) { BacklogQuota quota = persistentTopic.getBacklogQuota(backlogQuotaType); + BacklogQuotaMetrics topicBacklogQuotaMetrics = + persistentTopic.getPersistentTopicMetrics().getBacklogQuotaMetrics(); log.info("Backlog quota type {} exceeded for topic [{}]. Applying [{}] policy", backlogQuotaType, persistentTopic.getName(), quota.getPolicy()); switch (quota.getPolicy()) { - case consumer_backlog_eviction: - switch (backlogQuotaType) { - case destination_storage: + case consumer_backlog_eviction: + switch (backlogQuotaType) { + case destination_storage: dropBacklogForSizeLimit(persistentTopic, quota); + topicBacklogQuotaMetrics.recordSizeBasedBacklogEviction(); break; - case message_age: + case message_age: dropBacklogForTimeLimit(persistentTopic, quota, preciseTimeBasedBacklogQuotaCheck); + topicBacklogQuotaMetrics.recordTimeBasedBacklogEviction(); break; - default: - break; - } - break; - case producer_exception: - case producer_request_hold: - if (!advanceSlowestSystemCursor(persistentTopic)) { - // The slowest is not a system cursor. Disconnecting producers to put backpressure. - disconnectProducers(persistentTopic); - } - break; - default: - break; + default: + break; + } + break; + case producer_exception: + case producer_request_hold: + if (!advanceSlowestSystemCursor(persistentTopic)) { + // The slowest is not a system cursor. Disconnecting producers to put backpressure. + disconnectProducers(persistentTopic); + } + break; + default: + break; } } @@ -210,7 +214,7 @@ private void dropBacklogForTimeLimit(PersistentTopic persistentTopic, BacklogQuo ); } else { // If disabled precise time based backlog quota check, will try to remove whole ledger from cursor's backlog - Long currentMillis = ((ManagedLedgerImpl) persistentTopic.getManagedLedger()).getClock().millis(); + long currentMillis = ((ManagedLedgerImpl) persistentTopic.getManagedLedger()).getClock().millis(); ManagedLedgerImpl mLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); try { for (; ; ) { @@ -229,7 +233,7 @@ private void dropBacklogForTimeLimit(PersistentTopic persistentTopic, BacklogQuo } // Timestamp only > 0 if ledger has been closed if (ledgerInfo.getTimestamp() > 0 - && currentMillis - ledgerInfo.getTimestamp() > quota.getLimitTime() * 1000) { + && currentMillis - ledgerInfo.getTimestamp() > SECONDS.toMillis(quota.getLimitTime())) { // skip whole ledger for the slowest cursor PositionImpl nextPosition = PositionImpl.get(mLedger.getNextValidLedger(ledgerInfo.getLedgerId()), -1); @@ -263,19 +267,20 @@ private void disconnectProducers(PersistentTopic persistentTopic) { futures.add(producer.disconnect()); }); - FutureUtil.waitForAll(futures).thenRun(() -> { - log.info("All producers on topic [{}] are disconnected", persistentTopic.getName()); - }).exceptionally(exception -> { - log.error("Error in disconnecting producers on topic [{}] [{}]", persistentTopic.getName(), exception); - return null; - + FutureUtil.waitForAll(futures) + .thenRun(() -> + log.info("All producers on topic [{}] are disconnected", persistentTopic.getName())) + .exceptionally(exception -> { + log.error("Error in disconnecting producers on topic [{}] [{}]", persistentTopic.getName(), + exception); + return null; }); } /** * Advances the slowest cursor if that is a system cursor. * - * @param persistentTopic + * @param persistentTopic Persistent topic * @return true if the slowest cursor is a system cursor */ private boolean advanceSlowestSystemCursor(PersistentTopic persistentTopic) { 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 8d7415f01671b..45f9bdd7b0f7e 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 @@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.MILLISECONDS; 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; @@ -38,6 +39,7 @@ import io.netty.channel.socket.SocketChannel; import io.netty.handler.ssl.SslContext; import io.netty.util.concurrent.DefaultThreadFactory; +import io.prometheus.client.Histogram; import java.io.Closeable; import java.io.IOException; import java.lang.reflect.Field; @@ -198,6 +200,12 @@ public class BrokerService implements Closeable { private static final double GRACEFUL_SHUTDOWN_QUIET_PERIOD_RATIO_OF_TOTAL_TIMEOUT = 0.25d; private static final double GRACEFUL_SHUTDOWN_TIMEOUT_RATIO_OF_TOTAL_TIMEOUT = 0.5d; + private static final Histogram backlogQuotaCheckDuration = Histogram.build() + .name("pulsar_storage_backlog_quota_check_duration_seconds") + .help("The duration of the backlog quota check process.") + .buckets(5, 10, 30, 60, 300) + .register(); + private final PulsarService pulsar; private final ManagedLedgerFactory managedLedgerFactory; @@ -930,7 +938,7 @@ CompletableFuture shutdownEventLoopGracefully(EventLoopGroup eventLoopGrou long timeout = (long) (GRACEFUL_SHUTDOWN_TIMEOUT_RATIO_OF_TOTAL_TIMEOUT * brokerShutdownTimeoutMs); return NettyFutureUtil.toCompletableFutureVoid( eventLoopGroup.shutdownGracefully(quietPeriod, - timeout, TimeUnit.MILLISECONDS)); + timeout, MILLISECONDS)); } private CompletableFuture closeChannel(Channel channel) { @@ -2194,6 +2202,7 @@ public BacklogQuotaManager getBacklogQuotaManager() { } public void monitorBacklogQuota() { + long startTimeMillis = System.currentTimeMillis(); forEachPersistentTopic(topic -> { if (topic.isSizeBacklogExceeded()) { getBacklogQuotaManager().handleExceededBacklogQuota(topic, @@ -2213,6 +2222,9 @@ public void monitorBacklogQuota() { log.error("Error when checkTimeBacklogExceeded({}) in monitorBacklogQuota", topic.getName(), throwable); return null; + }).whenComplete((unused, throwable) -> { + backlogQuotaCheckDuration.observe( + MILLISECONDS.toSeconds(System.currentTimeMillis() - startTimeMillis)); }); } }); @@ -2734,7 +2746,7 @@ private void updateConfigurationAndRegisterListeners() { // add listener to notify broker managedLedgerCacheEvictionTimeThresholdMillis dynamic config registerConfigurationListener( "managedLedgerCacheEvictionTimeThresholdMillis", (cacheEvictionTimeThresholdMills) -> { - managedLedgerFactory.updateCacheEvictionTimeThreshold(TimeUnit.MILLISECONDS + managedLedgerFactory.updateCacheEvictionTimeThreshold(MILLISECONDS .toNanos((long) cacheEvictionTimeThresholdMills)); }); @@ -3192,7 +3204,7 @@ private void createPendingLoadTopic() { pendingTopic.getTopicFuture() .completeExceptionally((e instanceof RuntimeException && e.getCause() != null) ? e.getCause() : e); // schedule to process next pending topic - inactivityMonitor.schedule(this::createPendingLoadTopic, 100, TimeUnit.MILLISECONDS); + inactivityMonitor.schedule(this::createPendingLoadTopic, 100, MILLISECONDS); return null; }); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java index e06d9b2f81e7c..2e961b793d8e4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/Topic.java @@ -68,7 +68,7 @@ default void setOriginalSequenceId(long originalSequenceId) { /** * Return the producer name for the original producer. - * + *

    * For messages published locally, this will return the same local producer name, though in case of replicated * messages, the original producer name will differ */ @@ -136,7 +136,7 @@ default void setEntryTimestamp(long entryTimestamp) { /** * Tries to add a producer to the topic. Several validations will be performed. * - * @param producer + * @param producer Producer to add * @param producerQueuedFuture * a future that will be triggered if the producer is being queued up prior of getting established * @return the "topic epoch" if there is one or empty @@ -148,7 +148,7 @@ default void setEntryTimestamp(long entryTimestamp) { /** * Wait TransactionBuffer Recovers completely. * Take snapshot after TB Recovers completely. - * @param isTxnEnabled + * @param isTxnEnabled isTxnEnabled * @return a future which has completely if isTxn = false. Or a future return by takeSnapshot. */ CompletableFuture checkIfTransactionBufferRecoverCompletely(boolean isTxnEnabled); @@ -268,6 +268,13 @@ default boolean isDeduplicationEnabled() { BacklogQuota getBacklogQuota(BacklogQuotaType backlogQuotaType); + /** + * Uses the best-effort (not necessarily up-to-date) information available to return the age. + * @return The oldest unacknowledged message age in seconds, or -1 if not available + */ + long getBestEffortOldestUnacknowledgedMessageAgeSeconds(); + + void updateRates(NamespaceStats nsStats, NamespaceBundleStats currentBundleStats, StatsOutputStream topicStatsStream, ClusterReplicationMetrics clusterReplicationMetrics, String namespaceName, boolean hydratePublishers); 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 aadb232ede062..23d938d22fd4b 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 @@ -166,7 +166,7 @@ public CompletableFuture initialize() { .getPoliciesAsync(TopicName.get(topic).getNamespaceObject()) .thenCompose(optPolicies -> { final Policies policies; - if (!optPolicies.isPresent()) { + if (optPolicies.isEmpty()) { log.warn("[{}] Policies not present and isEncryptionRequired will be set to false", topic); isEncryptionRequired = false; policies = new Policies(); @@ -1183,4 +1183,9 @@ protected boolean isMigrated() { public boolean isPersistent() { return false; } + + @Override + public long getBestEffortOldestUnacknowledgedMessageAgeSeconds() { + return -1; + } } 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 55d6f6d70a1d9..b83a8206b4ab4 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 @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.service.persistent; import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.pulsar.broker.service.persistent.SubscribeRateLimiter.isSubscribeRateEnabled; import static org.apache.pulsar.common.naming.SystemTopicNames.isEventSystemTopic; @@ -49,10 +50,12 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import java.util.function.BiFunction; import java.util.stream.Collectors; import javax.annotation.Nonnull; import lombok.Getter; +import lombok.Value; import org.apache.bookkeeper.client.api.LedgerMetadata; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.AsyncCallbacks.AddEntryCallback; @@ -74,6 +77,7 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException.NonRecoverableLedgerException; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer; +import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer.CursorInfo; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; @@ -294,6 +298,30 @@ public CloseFutures(CompletableFuture waitDisconnectClients, this.waitDisconnectClients = waitDisconnectClients; this.notWaitDisconnectClients = notWaitDisconnectClients; } + + } + + @Getter + private final PersistentTopicMetrics persistentTopicMetrics = new PersistentTopicMetrics(); + + private volatile TimeBasedBacklogQuotaCheckResult timeBasedBacklogQuotaCheckResult; + private static final AtomicReferenceFieldUpdater + TIME_BASED_BACKLOG_QUOTA_CHECK_RESULT_UPDATER = AtomicReferenceFieldUpdater.newUpdater( + PersistentTopic.class, + TimeBasedBacklogQuotaCheckResult.class, + "timeBasedBacklogQuotaCheckResult"); + @Value + private static class TimeBasedBacklogQuotaCheckResult { + PositionImpl oldestCursorMarkDeletePosition; + String cursorName; + long positionPublishTimestampInMillis; + long dataVersion; + } + + @Value + private static class EstimateTimeBasedBacklogQuotaCheckResult { + boolean truncateBacklogToMatchQuota; + Long estimatedOldestUnacknowledgedMessageTimestamp; } private static class TopicStatsHelper { @@ -509,7 +537,7 @@ public CompletableFuture unloadSubscription(@Nonnull String subName) { if (!lock.writeLock().tryLock()) { return CompletableFuture.failedFuture(new SubscriptionConflictUnloadException(String.format("Conflict" + " topic-close, topic-delete, another-subscribe-unload, cannot unload subscription %s now", - topic, subName))); + subName))); } try { if (isFenced) { @@ -1149,7 +1177,7 @@ private CompletableFuture getNonDurableSubscription(Stri private void resetSubscriptionCursor(Subscription subscription, CompletableFuture subscriptionFuture, long startMessageRollbackDurationSec) { long timestamp = System.currentTimeMillis() - - TimeUnit.SECONDS.toMillis(startMessageRollbackDurationSec); + - SECONDS.toMillis(startMessageRollbackDurationSec); final Subscription finalSubscription = subscription; subscription.resetCursor(timestamp).handle((s, ex) -> { if (ex != null) { @@ -1680,7 +1708,7 @@ CompletableFuture checkReplicationAndRetryOnFailure() { if (!(th.getCause() instanceof TopicFencedException)) { // retriable exception brokerService.executor().schedule(this::checkReplicationAndRetryOnFailure, - POLICY_UPDATE_FAILURE_RETRY_TIME_SECONDS, TimeUnit.SECONDS); + POLICY_UPDATE_FAILURE_RETRY_TIME_SECONDS, SECONDS); } result.completeExceptionally(th); return null; @@ -2453,6 +2481,19 @@ public CompletableFuture asyncGetStats(boolean getPreciseBacklog stats.lastOffloadFailureTimeStamp = ledger.getLastOffloadedFailureTimestamp(); Optional mxBean = getCompactorMXBean(); + stats.backlogQuotaLimitSize = getBacklogQuota(BacklogQuotaType.destination_storage).getLimitSize(); + stats.backlogQuotaLimitTime = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); + + TimeBasedBacklogQuotaCheckResult backlogQuotaCheckResult = timeBasedBacklogQuotaCheckResult; + stats.oldestBacklogMessageAgeSeconds = (backlogQuotaCheckResult == null) + ? (long) -1 + : TimeUnit.MILLISECONDS.toSeconds( + Clock.systemUTC().millis() - backlogQuotaCheckResult.getPositionPublishTimestampInMillis()); + + stats.oldestBacklogMessageSubscriptionName = (backlogQuotaCheckResult == null) + ? null + : backlogQuotaCheckResult.getCursorName(); + stats.compaction.reset(); mxBean.flatMap(bean -> bean.getCompactionRecordForTopic(topic)).map(compactionRecord -> { stats.compaction.lastCompactionRemovedEventCount = compactionRecord.getLastCompactionRemovedEventCount(); @@ -2756,7 +2797,7 @@ public void checkGC() { int maxInactiveDurationInSec = topicPolicies.getInactiveTopicPolicies().get().getMaxInactiveDurationSeconds(); if (isActive(deleteMode)) { lastActive = System.nanoTime(); - } else if (System.nanoTime() - lastActive < TimeUnit.SECONDS.toNanos(maxInactiveDurationInSec)) { + } else if (System.nanoTime() - lastActive < SECONDS.toNanos(maxInactiveDurationInSec)) { // Gc interval did not expire yet return; } else if (shouldTopicBeRetained()) { @@ -3097,36 +3138,128 @@ public boolean isSizeBacklogExceeded() { return (storageSize >= backlogQuotaLimitInBytes); } + @Override + public long getBestEffortOldestUnacknowledgedMessageAgeSeconds() { + TimeBasedBacklogQuotaCheckResult result = timeBasedBacklogQuotaCheckResult; + if (result == null) { + return -1; + } else { + return TimeUnit.MILLISECONDS.toSeconds( + Clock.systemUTC().millis() - result.getPositionPublishTimestampInMillis()); + } + } + + private void updateResultIfNewer(TimeBasedBacklogQuotaCheckResult updatedResult) { + TIME_BASED_BACKLOG_QUOTA_CHECK_RESULT_UPDATER.updateAndGet(this, + existingResult -> { + if (existingResult == null + || ManagedCursorContainer.DataVersion.compareVersions( + updatedResult.getDataVersion(), existingResult.getDataVersion()) > 0) { + return updatedResult; + } else { + return existingResult; + } + }); + + } + /** * @return determine if backlog quota enforcement needs to be done for topic based on time limit */ public CompletableFuture checkTimeBacklogExceeded() { 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 backlog quota by time is not set and we have no durable cursor. - if (backlogQuotaLimitInSecond <= 0 - || ((ManagedCursorContainer) ledger.getCursors()).getSlowestReaderPosition() == null) { + ManagedCursorContainer managedCursorContainer = (ManagedCursorContainer) ledger.getCursors(); + CursorInfo oldestMarkDeleteCursorInfo = managedCursorContainer.getCursorWithOldestPosition(); + + // If we have no durable cursor since `ledger.getCursors()` only managed durable cursors + 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); + } return CompletableFuture.completedFuture(false); } + PositionImpl oldestMarkDeletePosition = oldestMarkDeleteCursorInfo.getPosition(); + + TimeBasedBacklogQuotaCheckResult lastCheckResult = timeBasedBacklogQuotaCheckResult; + if (lastCheckResult != null + && oldestMarkDeletePosition.compareTo(lastCheckResult.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(), + oldestMarkDeleteCursorInfo.getCursor().getName(), + lastCheckResult.getPositionPublishTimestampInMillis(), + oldestMarkDeleteCursorInfo.getVersion()); + + updateResultIfNewer(updatedResult); + if (log.isDebugEnabled()) { + log.debug("[{}] Time-based backlog quota check. Updating cached result for position {}, " + + "since cursor causing it has changed from {} to {}", + topicName, + oldestMarkDeletePosition, + lastCheckResult.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); + } + if (brokerService.pulsar().getConfiguration().isPreciseTimeBasedBacklogQuotaCheck()) { CompletableFuture future = new CompletableFuture<>(); // Check if first unconsumed message(first message after mark delete position) // for slowest cursor's has expired. - PositionImpl position = ((ManagedLedgerImpl) ledger).getNextValidPosition(((ManagedCursorContainer) - ledger.getCursors()).getSlowestReaderPosition()); + PositionImpl position = ((ManagedLedgerImpl) ledger).getNextValidPosition(oldestMarkDeletePosition); ((ManagedLedgerImpl) ledger).asyncReadEntry(position, new AsyncCallbacks.ReadEntryCallback() { @Override 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); - if (expired && log.isDebugEnabled()) { - log.debug("Time based backlog quota exceeded, oldest entry in cursor {}'s backlog" - + "exceeded quota {}", ((ManagedLedgerImpl) ledger).getSlowestConsumer().getName(), - backlogQuotaLimitInSecond); + if (log.isDebugEnabled()) { + log.debug("[{}] Time based backlog quota check. Oldest unacked entry read from BK. " + + "Oldest entry in cursor {}'s backlog: {}. " + + "Oldest mark-delete position: {}. " + + "Quota {}. Last check result position [{}]. " + + "Expired: {}, entryTimestamp: {}", + topicName, + oldestMarkDeleteCursorInfo.getCursor().getName(), + position, + oldestMarkDeletePosition, + backlogQuotaLimitInSecond, + lastCheckResult.getOldestCursorMarkDeletePosition(), + expired, + entryTimestamp); } future.complete(expired); } catch (Exception e) { @@ -3146,9 +3279,19 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { }, null); return future; } else { - PositionImpl slowestPosition = ((ManagedCursorContainer) ledger.getCursors()).getSlowestReaderPosition(); try { - return slowestReaderTimeBasedBacklogQuotaCheck(slowestPosition); + EstimateTimeBasedBacklogQuotaCheckResult checkResult = + estimatedTimeBasedBacklogQuotaCheck(oldestMarkDeletePosition); + if (checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp() != null) { + updateResultIfNewer( + new TimeBasedBacklogQuotaCheckResult( + oldestMarkDeleteCursorInfo.getPosition(), + oldestMarkDeleteCursorInfo.getCursor().getName(), + checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp(), + oldestMarkDeleteCursorInfo.getVersion())); + } + + return CompletableFuture.completedFuture(checkResult.isTruncateBacklogToMatchQuota()); } catch (Exception e) { log.error("[{}][{}] Error reading entry for precise time based backlog check", topicName, e); return CompletableFuture.completedFuture(false); @@ -3156,33 +3299,47 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { } } - private CompletableFuture slowestReaderTimeBasedBacklogQuotaCheck(PositionImpl slowestPosition) + private EstimateTimeBasedBacklogQuotaCheckResult estimatedTimeBasedBacklogQuotaCheck( + PositionImpl markDeletePosition) throws ExecutionException, InterruptedException { int backlogQuotaLimitInSecond = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); - Long ledgerId = slowestPosition.getLedgerId(); - if (((ManagedLedgerImpl) ledger).getLedgersInfo().lastKey().equals(ledgerId)) { - return CompletableFuture.completedFuture(false); + ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) ledger; + + // The ledger timestamp is only known when ledger is closed, hence when the mark-delete + // is at active ledger (open) we can't estimate it. + if (managedLedger.getLedgersInfo().lastKey().equals(markDeletePosition.getLedgerId())) { + return new EstimateTimeBasedBacklogQuotaCheckResult(false, null); } - int result; + org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo - ledgerInfo = ledger.getLedgerInfo(ledgerId).get(); - if (ledgerInfo != null && ledgerInfo.hasTimestamp() && ledgerInfo.getTimestamp() > 0 - && ((ManagedLedgerImpl) ledger).getClock().millis() - ledgerInfo.getTimestamp() - > backlogQuotaLimitInSecond * 1000 && (result = slowestPosition.compareTo( - new PositionImpl(ledgerInfo.getLedgerId(), ledgerInfo.getEntries() - 1))) <= 0) { - if (result < 0) { - if (log.isDebugEnabled()) { - log.debug("Time based backlog quota exceeded, quota {}, age of ledger " - + "slowest cursor currently on {}", backlogQuotaLimitInSecond * 1000, - ((ManagedLedgerImpl) ledger).getClock().millis() - ledgerInfo.getTimestamp()); - } - return CompletableFuture.completedFuture(true); - } else { - return slowestReaderTimeBasedBacklogQuotaCheck( - ((ManagedLedgerImpl) ledger).getNextValidPosition(slowestPosition)); + markDeletePositionLedgerInfo = ledger.getLedgerInfo(markDeletePosition.getLedgerId()).get(); + + org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo positionToCheckLedgerInfo = + markDeletePositionLedgerInfo; + + // if the mark-delete position is the last entry it means all entries for + // that ledger are acknowledged + if (markDeletePosition.getEntryId() == markDeletePositionLedgerInfo.getEntries() - 1) { + PositionImpl positionToCheck = managedLedger.getNextValidPosition(markDeletePosition); + positionToCheckLedgerInfo = ledger.getLedgerInfo(positionToCheck.getLedgerId()).get(); + } + + if (positionToCheckLedgerInfo != null + && positionToCheckLedgerInfo.hasTimestamp() + && positionToCheckLedgerInfo.getTimestamp() > 0) { + long estimateMsgAgeMs = managedLedger.getClock().millis() - positionToCheckLedgerInfo.getTimestamp(); + boolean shouldTruncateBacklog = estimateMsgAgeMs > SECONDS.toMillis(backlogQuotaLimitInSecond); + if (log.isDebugEnabled()) { + log.debug("Time based backlog quota exceeded, quota {}[ms], age of ledger " + + "slowest cursor currently on {}[ms]", backlogQuotaLimitInSecond * 1000, + estimateMsgAgeMs); } + + return new EstimateTimeBasedBacklogQuotaCheckResult( + shouldTruncateBacklog, + positionToCheckLedgerInfo.getTimestamp()); } else { - return CompletableFuture.completedFuture(false); + return new EstimateTimeBasedBacklogQuotaCheckResult(false, null); } } @@ -3573,7 +3730,7 @@ private synchronized void fence() { final int timeout = brokerService.pulsar().getConfiguration().getTopicFencingTimeoutSeconds(); if (timeout > 0) { this.fencedTopicMonitoringTask = brokerService.executor().schedule(this::closeFencedTopicForcefully, - timeout, TimeUnit.SECONDS); + timeout, SECONDS); } } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java new file mode 100644 index 0000000000000..f79d053a9790d --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopicMetrics.java @@ -0,0 +1,50 @@ +/* + * 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.atomic.LongAdder; +import lombok.Getter; + +@SuppressWarnings("LombokGetterMayBeUsed") +public class PersistentTopicMetrics { + + @Getter + private final BacklogQuotaMetrics backlogQuotaMetrics = new BacklogQuotaMetrics(); + + public static class BacklogQuotaMetrics { + private final LongAdder timeBasedBacklogQuotaExceededEvictionCount = new LongAdder(); + private final LongAdder sizeBasedBacklogQuotaExceededEvictionCount = new LongAdder(); + + public void recordTimeBasedBacklogEviction() { + timeBasedBacklogQuotaExceededEvictionCount.increment(); + } + + public void recordSizeBasedBacklogEviction() { + sizeBasedBacklogQuotaExceededEvictionCount.increment(); + } + + public long getSizeBasedBacklogQuotaExceededEvictionCount() { + return sizeBasedBacklogQuotaExceededEvictionCount.longValue(); + } + + public long getTimeBasedBacklogQuotaExceededEvictionCount() { + return timeBasedBacklogQuotaExceededEvictionCount.longValue(); + } + } +} \ No newline at end of file diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedBrokerStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedBrokerStats.java index 715231d3c6ee1..037fb29a999e3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedBrokerStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedBrokerStats.java @@ -33,7 +33,10 @@ public class AggregatedBrokerStats { public double storageReadRate; public double storageReadCacheMissesRate; public long msgBacklog; + public long sizeBasedBacklogQuotaExceededEvictionCount; + public long timeBasedBacklogQuotaExceededEvictionCount; + @SuppressWarnings("DuplicatedCode") void updateStats(TopicStats stats) { topicsCount++; subscriptionsCount += stats.subscriptionsCount; @@ -49,8 +52,11 @@ void updateStats(TopicStats stats) { storageReadRate += stats.managedLedgerStats.storageReadRate; storageReadCacheMissesRate += stats.managedLedgerStats.storageReadCacheMissesRate; msgBacklog += stats.msgBacklog; + timeBasedBacklogQuotaExceededEvictionCount += stats.timeBasedBacklogQuotaExceededEvictionCount; + sizeBasedBacklogQuotaExceededEvictionCount += stats.sizeBasedBacklogQuotaExceededEvictionCount; } + @SuppressWarnings("DuplicatedCode") public void reset() { topicsCount = 0; subscriptionsCount = 0; @@ -66,5 +72,7 @@ public void reset() { storageReadRate = 0; storageReadCacheMissesRate = 0; msgBacklog = 0; + sizeBasedBacklogQuotaExceededEvictionCount = 0; + timeBasedBacklogQuotaExceededEvictionCount = 0; } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java index d0dc4fe2a7e7d..3975cd89cfa6b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java @@ -51,6 +51,9 @@ public class AggregatedNamespaceStats { long backlogQuotaLimit; long backlogQuotaLimitTime; + public long sizeBasedBacklogQuotaExceededEvictionCount; + public long timeBasedBacklogQuotaExceededEvictionCount; + public Map replicationStats = new HashMap<>(); public Map subscriptionStats = new HashMap<>(); @@ -68,6 +71,7 @@ public class AggregatedNamespaceStats { Map bucketDelayedIndexStats = new HashMap<>(); + @SuppressWarnings("DuplicatedCode") void updateStats(TopicStats stats) { topicsCount++; @@ -105,6 +109,9 @@ void updateStats(TopicStats stats) { backlogQuotaLimit = Math.max(backlogQuotaLimit, stats.backlogQuotaLimit); backlogQuotaLimitTime = Math.max(backlogQuotaLimitTime, stats.backlogQuotaLimitTime); + sizeBasedBacklogQuotaExceededEvictionCount += stats.sizeBasedBacklogQuotaExceededEvictionCount; + timeBasedBacklogQuotaExceededEvictionCount += stats.timeBasedBacklogQuotaExceededEvictionCount; + managedLedgerStats.storageWriteRate += stats.managedLedgerStats.storageWriteRate; managedLedgerStats.storageReadRate += stats.managedLedgerStats.storageReadRate; managedLedgerStats.storageReadCacheMissesRate += stats.managedLedgerStats.storageReadCacheMissesRate; @@ -172,6 +179,7 @@ void updateStats(TopicStats stats) { compactionLatencyBuckets.addAll(stats.compactionLatencyBuckets); } + @SuppressWarnings("DuplicatedCode") public void reset() { managedLedgerStats.reset(); topicsCount = 0; @@ -201,6 +209,9 @@ public void reset() { replicationStats.clear(); subscriptionStats.clear(); + sizeBasedBacklogQuotaExceededEvictionCount = 0; + timeBasedBacklogQuotaExceededEvictionCount = 0; + compactionRemovedEventCount = 0; compactionSucceedCount = 0; compactionFailedCount = 0; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java index 4e72fa0d72b16..3728c3edd1e8b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java @@ -32,7 +32,10 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; -import org.apache.pulsar.common.policies.data.BacklogQuota; +import org.apache.pulsar.broker.service.persistent.PersistentTopicMetrics; +import org.apache.pulsar.broker.service.persistent.PersistentTopicMetrics.BacklogQuotaMetrics; +import org.apache.pulsar.broker.stats.prometheus.metrics.PrometheusLabels; +import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; import org.apache.pulsar.common.policies.data.stats.NonPersistentSubscriptionStatsImpl; import org.apache.pulsar.common.policies.data.stats.NonPersistentTopicStatsImpl; @@ -159,14 +162,15 @@ private static void aggregateTopicStats(TopicStats stats, SubscriptionStatsImpl subsStats.bucketDelayedIndexStats = subscriptionStats.bucketDelayedIndexStats; } + @SuppressWarnings("OptionalUsedAsFieldOrParameterType") private static void getTopicStats(Topic topic, TopicStats stats, boolean includeConsumerMetrics, boolean includeProducerMetrics, boolean getPreciseBacklog, boolean subscriptionBacklogSize, Optional compactorMXBean) { stats.reset(); - if (topic instanceof PersistentTopic) { + if (topic instanceof PersistentTopic persistentTopic) { // Managed Ledger stats - ManagedLedger ml = ((PersistentTopic) topic).getManagedLedger(); + ManagedLedger ml = persistentTopic.getManagedLedger(); ManagedLedgerMBeanImpl mlStats = (ManagedLedgerMBeanImpl) ml.getStats(); stats.managedLedgerStats.storageSize = mlStats.getStoredMessagesSize(); @@ -174,9 +178,10 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include stats.managedLedgerStats.backlogSize = ml.getEstimatedBacklogSize(); stats.managedLedgerStats.offloadedStorageUsed = ml.getOffloadedSize(); stats.backlogQuotaLimit = topic - .getBacklogQuota(BacklogQuota.BacklogQuotaType.destination_storage).getLimitSize(); + .getBacklogQuota(BacklogQuotaType.destination_storage).getLimitSize(); stats.backlogQuotaLimitTime = topic - .getBacklogQuota(BacklogQuota.BacklogQuotaType.message_age).getLimitTime(); + .getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); + stats.backlogAgeSeconds = topic.getBestEffortOldestUnacknowledgedMessageAgeSeconds(); stats.managedLedgerStats.storageWriteLatencyBuckets .addAll(mlStats.getInternalAddEntryLatencyBuckets()); @@ -191,7 +196,17 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include stats.managedLedgerStats.storageWriteRate = mlStats.getAddEntryMessagesRate(); stats.managedLedgerStats.storageReadRate = mlStats.getReadEntriesRate(); stats.managedLedgerStats.storageReadCacheMissesRate = mlStats.getReadEntriesOpsCacheMissesRate(); + + // Topic Stats + PersistentTopicMetrics persistentTopicMetrics = persistentTopic.getPersistentTopicMetrics(); + + BacklogQuotaMetrics backlogQuotaMetrics = persistentTopicMetrics.getBacklogQuotaMetrics(); + stats.sizeBasedBacklogQuotaExceededEvictionCount = + backlogQuotaMetrics.getSizeBasedBacklogQuotaExceededEvictionCount(); + stats.timeBasedBacklogQuotaExceededEvictionCount = + backlogQuotaMetrics.getTimeBasedBacklogQuotaExceededEvictionCount(); } + TopicStatsImpl tStatus = topic.getStats(getPreciseBacklog, subscriptionBacklogSize, false); stats.msgInCounter = tStatus.msgInCounter; stats.bytesInCounter = tStatus.bytesInCounter; @@ -334,6 +349,14 @@ private static void printBrokerStats(PrometheusMetricStreams stream, String clus writeMetric(stream, "pulsar_broker_storage_read_rate", brokerStats.storageReadRate, cluster); writeMetric(stream, "pulsar_broker_storage_read_cache_misses_rate", brokerStats.storageReadCacheMissesRate, cluster); + + writePulsarBacklogQuotaMetricBrokerLevel(stream, + "pulsar_broker_storage_backlog_quota_exceeded_evictions_total", + brokerStats.sizeBasedBacklogQuotaExceededEvictionCount, cluster, BacklogQuotaType.destination_storage); + writePulsarBacklogQuotaMetricBrokerLevel(stream, + "pulsar_broker_storage_backlog_quota_exceeded_evictions_total", + brokerStats.timeBasedBacklogQuotaExceededEvictionCount, cluster, BacklogQuotaType.message_age); + writeMetric(stream, "pulsar_broker_msg_backlog", brokerStats.msgBacklog, cluster); } @@ -372,6 +395,7 @@ private static void printNamespaceStats(PrometheusMetricStreams stream, Aggregat stats.managedLedgerStats.storageLogicalSize, cluster, namespace); writeMetric(stream, "pulsar_storage_backlog_size", stats.managedLedgerStats.backlogSize, cluster, namespace); + writeMetric(stream, "pulsar_storage_offloaded_size", stats.managedLedgerStats.offloadedStorageUsed, cluster, namespace); @@ -392,6 +416,14 @@ private static void printNamespaceStats(PrometheusMetricStreams stream, Aggregat }); writePulsarMsgBacklog(stream, stats.msgBacklog, cluster, namespace); + writePulsarBacklogQuotaMetricNamespaceLevel(stream, + "pulsar_storage_backlog_quota_exceeded_evictions_total", + stats.sizeBasedBacklogQuotaExceededEvictionCount, cluster, namespace, + BacklogQuotaType.destination_storage); + writePulsarBacklogQuotaMetricNamespaceLevel(stream, + "pulsar_storage_backlog_quota_exceeded_evictions_total", + stats.timeBasedBacklogQuotaExceededEvictionCount, cluster, namespace, + BacklogQuotaType.message_age); stats.managedLedgerStats.storageWriteLatencyBuckets.refresh(); long[] latencyBuckets = stats.managedLedgerStats.storageWriteLatencyBuckets.getBuckets(); @@ -471,6 +503,25 @@ private static void printNamespaceStats(PrometheusMetricStreams stream, Aggregat replStats -> replStats.replicationDelayInSeconds, cluster, namespace); } + @SuppressWarnings("SameParameterValue") + private static void writePulsarBacklogQuotaMetricBrokerLevel(PrometheusMetricStreams stream, String metricName, + Number value, String cluster, + BacklogQuotaType backlogQuotaType) { + String quotaTypeLabelValue = PrometheusLabels.backlogQuotaTypeLabel(backlogQuotaType); + stream.writeSample(metricName, value, "cluster", cluster, + "quota_type", quotaTypeLabelValue); + } + + @SuppressWarnings("SameParameterValue") + private static void writePulsarBacklogQuotaMetricNamespaceLevel(PrometheusMetricStreams stream, String metricName, + Number value, String cluster, String namespace, + BacklogQuotaType backlogQuotaType) { + String quotaTypeLabelValue = PrometheusLabels.backlogQuotaTypeLabel(backlogQuotaType); + stream.writeSample(metricName, value, "cluster", cluster, + "namespace", namespace, + "quota_type", quotaTypeLabelValue); + } + private static void writePulsarMsgBacklog(PrometheusMetricStreams stream, Number value, String cluster, String namespace) { stream.writeSample("pulsar_msg_backlog", value, "cluster", cluster, "namespace", namespace, 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 33ef05df9ebc4..27288291d2969 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 @@ -25,6 +25,8 @@ import org.apache.bookkeeper.mledger.util.StatsBuckets; import org.apache.commons.lang3.ArrayUtils; import org.apache.pulsar.broker.service.Consumer; +import org.apache.pulsar.broker.stats.prometheus.metrics.PrometheusLabels; +import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; import org.apache.pulsar.common.policies.data.stats.TopicMetricBean; import org.apache.pulsar.compaction.CompactionRecord; import org.apache.pulsar.compaction.CompactorMXBean; @@ -52,6 +54,7 @@ class TopicStats { long backlogQuotaLimit; long backlogQuotaLimitTime; + long backlogAgeSeconds; ManagedLedgerStats managedLedgerStats = new ManagedLedgerStats(); @@ -73,6 +76,11 @@ class TopicStats { Map bucketDelayedIndexStats = new HashMap<>(); + public long sizeBasedBacklogQuotaExceededEvictionCount; + public long timeBasedBacklogQuotaExceededEvictionCount; + + + @SuppressWarnings("DuplicatedCode") public void reset() { subscriptionsCount = 0; producersCount = 0; @@ -111,8 +119,13 @@ public void reset() { compactionLatencyBuckets.reset(); delayedMessageIndexSizeInBytes = 0; bucketDelayedIndexStats.clear(); + + timeBasedBacklogQuotaExceededEvictionCount = 0; + sizeBasedBacklogQuotaExceededEvictionCount = 0; + backlogAgeSeconds = -1; } + @SuppressWarnings("OptionalUsedAsFieldOrParameterType") public static void printTopicStats(PrometheusMetricStreams stream, TopicStats stats, Optional compactorMXBean, String cluster, String namespace, String topic, boolean splitTopicAndPartitionIndexLabel) { @@ -165,6 +178,14 @@ public static void printTopicStats(PrometheusMetricStreams stream, TopicStats st cluster, namespace, topic, splitTopicAndPartitionIndexLabel); writeMetric(stream, "pulsar_storage_backlog_quota_limit_time", stats.backlogQuotaLimitTime, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_storage_backlog_age_seconds", stats.backlogAgeSeconds, + cluster, namespace, topic, splitTopicAndPartitionIndexLabel); + writeBacklogQuotaMetric(stream, "pulsar_storage_backlog_quota_exceeded_evictions_total", + stats.sizeBasedBacklogQuotaExceededEvictionCount, cluster, namespace, topic, + splitTopicAndPartitionIndexLabel, BacklogQuotaType.destination_storage); + writeBacklogQuotaMetric(stream, "pulsar_storage_backlog_quota_exceeded_evictions_total", + stats.timeBasedBacklogQuotaExceededEvictionCount, cluster, namespace, topic, + splitTopicAndPartitionIndexLabel, BacklogQuotaType.message_age); writeMetric(stream, "pulsar_delayed_message_index_size_bytes", stats.delayedMessageIndexSizeInBytes, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); @@ -442,6 +463,17 @@ private static void writeMetric(PrometheusMetricStreams stream, String metricNam writeTopicMetric(stream, metricName, value, cluster, namespace, topic, splitTopicAndPartitionIndexLabel); } + @SuppressWarnings("SameParameterValue") + private static void writeBacklogQuotaMetric(PrometheusMetricStreams stream, String metricName, Number value, + String cluster, String namespace, String topic, + boolean splitTopicAndPartitionIndexLabel, + BacklogQuotaType backlogQuotaType) { + + String quotaTypeLabelValue = PrometheusLabels.backlogQuotaTypeLabel(backlogQuotaType); + writeTopicMetric(stream, metricName, value, cluster, namespace, topic, splitTopicAndPartitionIndexLabel, + "quota_type", quotaTypeLabelValue); + } + private static void writeMetric(PrometheusMetricStreams stream, String metricName, Number value, String cluster, String namespace, String topic, String remoteCluster, boolean splitTopicAndPartitionIndexLabel) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusLabels.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusLabels.java new file mode 100644 index 0000000000000..9a2c520731468 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/metrics/PrometheusLabels.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.stats.prometheus.metrics; + +import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; + +public class PrometheusLabels { + + public static String backlogQuotaTypeLabel(BacklogQuotaType backlogQuotaType) { + if (backlogQuotaType == BacklogQuotaType.message_age) { + return "time"; + } else /* destination_storage */ { + return "size"; + } + } +} 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 3918dcbe86d66..f194dfe1340e9 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 @@ -18,6 +18,13 @@ */ package org.apache.pulsar.broker.service; +import static java.util.Map.entry; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType.destination_storage; +import static org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType.message_age; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.AssertionsForClassTypes.within; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; @@ -33,15 +40,18 @@ import java.util.UUID; import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import lombok.Cleanup; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metrics; import org.apache.pulsar.client.admin.GetStatsOptions; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -73,6 +83,9 @@ @Test(groups = "broker") public class BacklogQuotaManagerTest { + private static final Logger log = LoggerFactory.getLogger(BacklogQuotaManagerTest.class); + + public static final String CLUSTER_NAME = "usc"; PulsarService pulsar; ServiceConfiguration config; @@ -80,6 +93,7 @@ public class BacklogQuotaManagerTest { PulsarAdmin admin; LocalBookkeeperEnsemble bkEnsemble; + PrometheusMetricsClient prometheusMetricsClient; private static final int TIME_TO_CHECK_BACKLOG_QUOTA = 2; private static final int MAX_ENTRIES_PER_LEDGER = 5; @@ -117,7 +131,7 @@ void setup() throws Exception { config.setMetadataStoreUrl("zk:127.0.0.1:" + bkEnsemble.getZookeeperPort()); config.setAdvertisedAddress("localhost"); config.setWebServicePort(Optional.of(0)); - config.setClusterName("usc"); + config.setClusterName(CLUSTER_NAME); config.setBrokerShutdownTimeoutMs(0L); config.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); config.setBrokerServicePort(Optional.of(0)); @@ -136,6 +150,7 @@ void setup() throws Exception { adminUrl = new URL("http://127.0.0.1" + ":" + pulsar.getListenPortHTTP().get()); admin = PulsarAdmin.builder().serviceHttpUrl(adminUrl.toString()).build(); + prometheusMetricsClient = new PrometheusMetricsClient("127.0.0.1", pulsar.getListenPortHTTP().get()); admin.clusters().createCluster("usc", ClusterData.builder().serviceUrl(adminUrl.toString()).build()); admin.tenants().createTenant("prop", @@ -190,7 +205,7 @@ private void rolloverStats() { } /** - * Readers should not effect backlog quota + * Readers should not affect backlog quota */ @Test public void testBacklogQuotaWithReader() throws Exception { @@ -202,18 +217,18 @@ public void testBacklogQuotaWithReader() throws Exception { .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) .build()); - try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS).build();) { + try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS).build()) { final String topic1 = "persistent://prop/ns-quota/topic1" + UUID.randomUUID(); final int numMsgs = 20; Reader reader = client.newReader().topic(topic1).receiverQueueSize(1).startMessageId(MessageId.latest).create(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { content[0] = (byte) (content[0] + 1); - MessageId msgId = producer.send(content); + producer.send(content); } Thread.sleep((TIME_TO_CHECK_BACKLOG_QUOTA + 1) * 1000); @@ -259,7 +274,7 @@ public void testBacklogQuotaWithReader() throws Exception { // check reader can still read with out error while (true) { - Message msg = reader.readNext(5, TimeUnit.SECONDS); + Message msg = reader.readNext(5, SECONDS); if (msg == null) { break; } @@ -284,10 +299,11 @@ public void testTriggerBacklogQuotaSizeWithReader() throws Exception { .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) .build()); - try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS).build();) { + try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS).build();) { final String topic1 = "persistent://prop/ns-quota/topic1" + UUID.randomUUID(); final int numMsgs = 20; Reader reader = client.newReader().topic(topic1).receiverQueueSize(1).startMessageId(MessageId.latest).create(); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { @@ -327,10 +343,10 @@ public void testTriggerBacklogQuotaSizeWithReader() throws Exception { // check if it's the expected ledger id given MAX_ENTRIES_PER_LEDGER assertEquals(internalStats.ledgers.get(0).ledgerId, finalMessageId.getLedgerId() + 1); }); - // check reader can still read with out error + // check reader can still read without error while (true) { - Message msg = reader.readNext(5, TimeUnit.SECONDS); + Message msg = reader.readNext(5, SECONDS); if (msg == null) { break; } @@ -341,6 +357,282 @@ public void testTriggerBacklogQuotaSizeWithReader() throws Exception { } } + @Test + public void backlogsStatsPrecise() throws PulsarAdminException, PulsarClientException, InterruptedException { + config.setPreciseTimeBasedBacklogQuotaCheck(true); + final String namespace = "prop/ns-quota"; + assertEquals(admin.namespaces().getBacklogQuotaMap(namespace), new HashMap<>()); + final int sizeLimitBytes = 15 * 1024 * 1024; + final int timeLimitSeconds = 123; + admin.namespaces().setBacklogQuota( + namespace, + BacklogQuota.builder() + .limitSize(sizeLimitBytes) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build(), + destination_storage); + admin.namespaces().setBacklogQuota( + namespace, + BacklogQuota.builder() + .limitTime(timeLimitSeconds) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build(), + message_age); + + 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); + + assertThat(topicStats.getBacklogQuotaLimitSize()).isEqualTo(sizeLimitBytes); + assertThat(topicStats.getBacklogQuotaLimitTime()).isEqualTo(timeLimitSeconds); + long expectedMessageAgeSeconds = MILLISECONDS.toSeconds(System.currentTimeMillis() - oldestMessage.getPublishTime()); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()) + .isCloseTo(expectedMessageAgeSeconds, within(1L)); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName2); + + 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)); + + // Move subscription 2 away from being the oldest mark delete + // S2/S1 + // 0 1 + Message firstOldestMessage = consumer2.receive(); + consumer2.acknowledge(firstOldestMessage); + // We only read and not ack, since we just need its publish-timestamp for later assert + Message secondOldestMessage = consumer2.receive(); + + // Switch subscription 1 to be where subscription 2 was in terms of oldest mark delete + // S1 S2 + // 0 1 + consumer1.seek(MessageId.earliest); + + log.info("Subscription 1 moved to be the oldest"); + + c1MarkDeletePositionBefore = waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); + + metrics = prometheusMetricsClient.getMetrics(); + long actualAge = (long) metrics.findByNameAndLabels( + "pulsar_storage_backlog_age_seconds", "topic", topic1) + .get(0).value; + + expectedMessageAgeSeconds = MILLISECONDS.toSeconds(System.currentTimeMillis() - oldestMessage.getPublishTime()); + assertThat(actualAge).isCloseTo(expectedMessageAgeSeconds, within(2L)); + + topicStats = getTopicStats(topic1); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName1); + + long entriesReadBefore = getReadEntries(topic1); + + // Move subscription 1 passed subscription 2 + for (int i = 0; i < 3; i++) { + Message message = consumer1.receive(); + log.info("Subscription 1 about to ack message ID {}", message.getMessageId()); + consumer1.acknowledge(message); + } + + log.info("Subscription 1 moved 3 messages. Now subscription 2 is the oldest"); + waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); + + // Cache shouldn't be used, since position has changed + long readEntries = getReadEntries(topic1); + assertThat(readEntries).isGreaterThan(entriesReadBefore); + + topicStats = getTopicStats(topic1); + expectedMessageAgeSeconds = MILLISECONDS.toSeconds(System.currentTimeMillis() - secondOldestMessage.getPublishTime()); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isCloseTo(expectedMessageAgeSeconds, within(2L)); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName2); + + waitForQuotaCheckToRunTwice(); + + // Cache should be used, since position hasn't changed + assertThat(getReadEntries(topic1)).isEqualTo(readEntries); + } + } + + private long getReadEntries(String topic1) { + return ((PersistentTopic) pulsar.getBrokerService().getTopicReference(topic1).get()) + .getManagedLedger().getStats().getEntriesReadTotalCount(); + } + + @Test + public void backlogsStatsNotPrecise() throws PulsarAdminException, PulsarClientException, InterruptedException { + config.setPreciseTimeBasedBacklogQuotaCheck(false); + config.setManagedLedgerMaxEntriesPerLedger(6); + final String namespace = "prop/ns-quota"; + assertEquals(admin.namespaces().getBacklogQuotaMap(namespace), new HashMap<>()); + final int sizeLimitBytes = 15 * 1024 * 1024; + final int timeLimitSeconds = 123; + admin.namespaces().setBacklogQuota( + namespace, + BacklogQuota.builder() + .limitSize(sizeLimitBytes) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build(), + destination_storage); + admin.namespaces().setBacklogQuota( + namespace, + BacklogQuota.builder() + .limitTime(timeLimitSeconds) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build(), + message_age); + + 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 = "brandNewC1"; + final String subName2 = "brandNewC2"; + final int numMsgs = 5; + + Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1) + .acknowledgmentGroupTime(0, SECONDS) + .isAckReceiptEnabled(true) + .subscribe(); + Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2) + .acknowledgmentGroupTime(0, SECONDS) + .isAckReceiptEnabled(true) + .subscribe(); + Producer producer = createProducer(client, topic1); + + byte[] content = new byte[1024]; + for (int i = 0; i < numMsgs; i++) { + Thread.sleep(500); + producer.send(content); + } + + String c1MarkDeletePositionBefore = + admin.topics().getInternalStats(topic1).cursors.get(subName1).markDeletePosition; + + consumer1.acknowledge(consumer1.receive()); + log.info("Moved subscription 1, by 1 message"); + c1MarkDeletePositionBefore = waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); + + TopicStats topicStats = getTopicStats(topic1); + + // We have only one ledger, and it is not closed yet, so we can't tell the age until it is closed + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isNull(); + + Metrics metrics = prometheusMetricsClient.getMetrics(); + Metric backlogAgeMetric = + metrics.findSingleMetricByNameAndLabels("pulsar_storage_backlog_age_seconds", + Pair.of("topic", topic1)); + assertThat(backlogAgeMetric.value).isEqualTo(-1); + + unloadAndLoadTopic(topic1, producer); + long unloadTime = System.currentTimeMillis(); + + waitForQuotaCheckToRunTwice(); + + topicStats = getTopicStats(topic1); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName2); + // age is measured against the ledger closing time + long expectedAge = MILLISECONDS.toSeconds(System.currentTimeMillis() - unloadTime); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isCloseTo(expectedAge, within(1L)); + + String c2MarkDeletePositionBefore = + admin.topics().getInternalStats(topic1).cursors.get(subName2).markDeletePosition; + Message message; + for (int i = 0; i < numMsgs-1; i++) { + consumer1.acknowledge(consumer1.receive()); + message = consumer2.receive(); + consumer2.acknowledge(message); + } + // At this point subscription 2 is the oldest + + waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForMarkDeletePositionToChange(topic1, subName2, c2MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); + + topicStats = getTopicStats(topic1); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isEqualTo(subName2); + expectedAge = MILLISECONDS.toSeconds(System.currentTimeMillis() - unloadTime); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isCloseTo(expectedAge, within(1L)); + } + config.setManagedLedgerMaxEntriesPerLedger(MAX_ENTRIES_PER_LEDGER); + } + + private void unloadAndLoadTopic(String topic, Producer producer) throws PulsarAdminException, + PulsarClientException { + admin.topics().unload(topic); + // This will load the topic + producer.send("Bla".getBytes()); + Awaitility.await().pollInterval(100, MILLISECONDS).atMost(5, SECONDS) + .until(() -> admin.topics().getInternalStats(topic).numberOfEntries > 0); + } + + private void waitForQuotaCheckToRunTwice() { + final long initialQuotaCheckCount = getQuotaCheckCount(); + Awaitility.await() + .pollInterval(1, SECONDS) + .atMost(TIME_TO_CHECK_BACKLOG_QUOTA*3, SECONDS) + .until(() -> getQuotaCheckCount() > initialQuotaCheckCount + 1); + } + + /** + * @return The new mark delete position + */ + private String waitForMarkDeletePositionToChange(String topic, + String subscriptionName, + String previousMarkDeletePosition) { + return Awaitility.await().pollInterval(1, SECONDS).atMost(5, SECONDS).until( + () -> admin.topics().getInternalStats(topic).cursors.get(subscriptionName).markDeletePosition, + markDeletePosition -> markDeletePosition != null && !markDeletePosition.equals(previousMarkDeletePosition)); + } + + private long getQuotaCheckCount() { + Metrics metrics = prometheusMetricsClient.getMetrics(); + return (long) metrics.findByNameAndLabels( + "pulsar_storage_backlog_quota_check_duration_seconds_count", + "cluster", CLUSTER_NAME) + .get(0).value; + } + /** * Time based backlog quota won't affect reader since broker doesn't keep track of consuming position for reader * and can't do message age check against the quota. @@ -356,7 +648,7 @@ public void testTriggerBacklogTimeQuotaWithReader() throws Exception { .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) .build()); - try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS).build();) { + try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS).build();) { final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); final int numMsgs = 9; Reader reader = client.newReader().topic(topic1).receiverQueueSize(1).startMessageId(MessageId.latest).create(); @@ -402,7 +694,7 @@ public void testTriggerBacklogTimeQuotaWithReader() throws Exception { // check reader can still read without error while (true) { - Message msg = reader.readNext(5, TimeUnit.SECONDS); + Message msg = reader.readNext(5, SECONDS); if (msg == null) { break; } @@ -423,7 +715,7 @@ public void testConsumerBacklogEvictionSizeQuota() throws Exception { .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) .build()); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); @@ -433,7 +725,7 @@ public void testConsumerBacklogEvictionSizeQuota() throws Exception { Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { producer.send(content); @@ -446,6 +738,8 @@ public void testConsumerBacklogEvictionSizeQuota() throws Exception { TopicStats stats = getTopicStats(topic1); assertTrue(stats.getBacklogSize() < 10 * 1024, "Storage size is [" + stats.getStorageSize() + "]"); + assertThat(evictionCountMetric("prop/ns-quota", topic1, "size")).isEqualTo(1); + assertThat(evictionCountMetric("size")).isEqualTo(1); } @Test @@ -456,10 +750,10 @@ public void testConsumerBacklogEvictionTimeQuotaPrecise() throws Exception { BacklogQuota.builder() .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); config.setPreciseTimeBasedBacklogQuotaCheck(true); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); final String topic1 = "persistent://prop/ns-quota/topic3" + UUID.randomUUID(); @@ -469,7 +763,7 @@ public void testConsumerBacklogEvictionTimeQuotaPrecise() throws Exception { Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { producer.send(content); @@ -488,8 +782,32 @@ public void testConsumerBacklogEvictionTimeQuotaPrecise() throws Exception { // All messages for both subscription should be cleaned up from backlog by backlog monitor task. assertEquals(stats.getSubscriptions().get(subName1).getMsgBacklog(), 0); assertEquals(stats.getSubscriptions().get(subName2).getMsgBacklog(), 0); + assertThat(evictionCountMetric("prop/ns-quota", topic1, "time")).isEqualTo(1); + assertThat(evictionCountMetric("time")).isEqualTo(1); + } + + @SuppressWarnings("SameParameterValue") + private long evictionCountMetric(String namespace, String topic, String quotaType) { + Metrics metrics = prometheusMetricsClient.getMetrics(); + Metric topicEvictionsTotal = metrics.findSingleMetricByNameAndLabels( + "pulsar_storage_backlog_quota_exceeded_evictions_total", + Pair.of("topic", topic), + Pair.of("quota_type", quotaType), + Pair.of("namespace", namespace), + Pair.of("cluster", CLUSTER_NAME)); + return (long) topicEvictionsTotal.value; + } + + private long evictionCountMetric(String quotaType) { + Metrics metrics = prometheusMetricsClient.getMetrics(); + Metric topicEvictionsTotal = metrics.findSingleMetricByNameAndLabels( + "pulsar_broker_storage_backlog_quota_exceeded_evictions_total", + Pair.of("quota_type", quotaType), + Pair.of("cluster", CLUSTER_NAME)); + return (long) topicEvictionsTotal.value; } + @Test(timeOut = 60000) public void testConsumerBacklogEvictionTimeQuota() throws Exception { assertEquals(admin.namespaces().getBacklogQuotaMap("prop/ns-quota"), @@ -498,9 +816,9 @@ public void testConsumerBacklogEvictionTimeQuota() throws Exception { BacklogQuota.builder() .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); final String topic1 = "persistent://prop/ns-quota/topic3" + UUID.randomUUID(); @@ -510,7 +828,7 @@ public void testConsumerBacklogEvictionTimeQuota() throws Exception { Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { producer.send(content); @@ -548,9 +866,9 @@ public void testConsumerBacklogEvictionTimeQuotaWithPartEviction() throws Except BacklogQuota.builder() .limitTime(5) // set limit time as 5 seconds .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); final String topic1 = "persistent://prop/ns-quota/topic3" + UUID.randomUUID(); @@ -560,7 +878,7 @@ public void testConsumerBacklogEvictionTimeQuotaWithPartEviction() throws Except Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { producer.send(content); @@ -602,17 +920,17 @@ public void testConsumerBacklogEvictionTimeQuotaWithEmptyLedger() throws Excepti BacklogQuota.builder() .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); config.setPreciseTimeBasedBacklogQuotaCheck(true); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); final String topic = "persistent://prop/ns-quota/topic4" + UUID.randomUUID(); final String subName = "c1"; Consumer consumer = client.newConsumer().topic(topic).subscriptionName(subName).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic); + Producer producer = createProducer(client, topic); producer.send(new byte[1024]); consumer.receive(); @@ -660,7 +978,7 @@ public void testConsumerBacklogEvictionWithAckSizeQuota() throws Exception { Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { producer.send(content); @@ -684,7 +1002,7 @@ public void testConsumerBacklogEvictionWithAckTimeQuotaPrecise() throws Exceptio BacklogQuota.builder() .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); config.setPreciseTimeBasedBacklogQuotaCheck(true); @Cleanup PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).build(); @@ -696,7 +1014,7 @@ public void testConsumerBacklogEvictionWithAckTimeQuotaPrecise() throws Exceptio Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { @@ -734,7 +1052,7 @@ private Producer createProducer(PulsarClient client, String topic) throws PulsarClientException { return client.newProducer() .enableBatching(false) - .sendTimeout(2, TimeUnit.SECONDS) + .sendTimeout(2, SECONDS) .topic(topic) .create(); } @@ -753,7 +1071,7 @@ public void testConsumerBacklogEvictionWithAckTimeQuota() throws Exception { Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; List> messagesToAcknowledge = new ArrayList<>(); @@ -794,7 +1112,7 @@ public void testConsumerBacklogEvictionWithAckTimeQuota() throws Exception { BacklogQuota.builder() .limitTime(2 * TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.consumer_backlog_eviction) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); Awaitility.await() .pollInterval(Duration.ofSeconds(1)) @@ -828,10 +1146,10 @@ public void testConcurrentAckAndEviction() throws Exception { final CountDownLatch counter = new CountDownLatch(2); final AtomicBoolean gotException = new AtomicBoolean(false); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); @Cleanup - PulsarClient client2 = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client2 = PulsarClient.builder().serviceUrl(adminUrl.toString()).statsInterval(0, SECONDS) .build(); Consumer consumer1 = client2.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client2.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); @@ -871,7 +1189,7 @@ public void testConcurrentAckAndEviction() throws Exception { consumerThread.start(); // test hangs without timeout since there is nothing to consume due to eviction - counter.await(20, TimeUnit.SECONDS); + counter.await(20, SECONDS); assertFalse(gotException.get()); Thread.sleep((TIME_TO_CHECK_BACKLOG_QUOTA + 1) * 1000); rolloverStats(); @@ -900,13 +1218,13 @@ public void testNoEviction() throws Exception { final AtomicBoolean gotException = new AtomicBoolean(false); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); final Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); @Cleanup final PulsarClient client2 = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); Thread producerThread = new Thread(() -> { try { @@ -964,16 +1282,16 @@ public void testEvictionMulti() throws Exception { final AtomicBoolean gotException = new AtomicBoolean(false); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); final Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); @Cleanup final PulsarClient client3 = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); @Cleanup final PulsarClient client2 = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); Thread producerThread1 = new Thread(() -> { try { @@ -1037,7 +1355,7 @@ public void testEvictionMulti() throws Exception { producerThread2.start(); consumerThread1.start(); consumerThread2.start(); - counter.await(20, TimeUnit.SECONDS); + counter.await(20, SECONDS); assertFalse(gotException.get()); Thread.sleep((TIME_TO_CHECK_BACKLOG_QUOTA + 1) * 1000); rolloverStats(); @@ -1057,7 +1375,7 @@ public void testAheadProducerOnHold() throws Exception { .build()); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final String topic1 = "persistent://prop/quotahold/hold"; final String subName1 = "c1hold"; final int numMsgs = 10; @@ -1099,7 +1417,7 @@ public void testAheadProducerOnHoldTimeout() throws Exception { .build()); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final String topic1 = "persistent://prop/quotahold/holdtimeout"; final String subName1 = "c1holdtimeout"; boolean gotException = false; @@ -1137,7 +1455,7 @@ public void testProducerException() throws Exception { .build()); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final String topic1 = "persistent://prop/quotahold/except"; final String subName1 = "c1except"; boolean gotException = false; @@ -1182,7 +1500,7 @@ public void testProducerExceptionAndThenUnblockSizeQuota(boolean dedupTestSet) t .build()); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final String topic1 = "persistent://prop/quotahold/exceptandunblock"; final String subName1 = "c1except"; boolean gotException = false; @@ -1266,11 +1584,11 @@ public void testProducerExceptionAndThenUnblockTimeQuotaPrecise() throws Excepti BacklogQuota.builder() .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); config.setPreciseTimeBasedBacklogQuotaCheck(true); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final String topic1 = "persistent://prop/quotahold/exceptandunblock2"; final String subName1 = "c1except"; boolean gotException = false; @@ -1332,10 +1650,10 @@ public void testProducerExceptionAndThenUnblockTimeQuota() throws Exception { BacklogQuota.builder() .limitTime(TIME_TO_CHECK_BACKLOG_QUOTA) .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) - .build(), BacklogQuota.BacklogQuotaType.message_age); + .build(), message_age); @Cleanup final PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) - .statsInterval(0, TimeUnit.SECONDS).build(); + .statsInterval(0, SECONDS).build(); final String topic1 = "persistent://prop/quotahold/exceptandunblock2"; final String subName1 = "c1except"; boolean gotException = false; @@ -1406,7 +1724,7 @@ public void testBacklogQuotaInGB(boolean backlogQuotaSizeGB) throws Exception { admin = PulsarAdmin.builder().serviceHttpUrl(adminUrl.toString()).build(); @Cleanup - PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).statsInterval(0, TimeUnit.SECONDS) + PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).statsInterval(0, SECONDS) .build(); final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); @@ -1416,7 +1734,7 @@ public void testBacklogQuotaInGB(boolean backlogQuotaSizeGB) throws Exception { Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1).subscribe(); Consumer consumer2 = client.newConsumer().topic(topic1).subscriptionName(subName2).subscribe(); - org.apache.pulsar.client.api.Producer producer = createProducer(client, topic1); + Producer producer = createProducer(client, topic1); byte[] content = new byte[1024]; for (int i = 0; i < numMsgs; i++) { producer.send(content); 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 16bce498e913f..ad7728319c9a7 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 @@ -21,6 +21,7 @@ import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgs; import static org.apache.pulsar.broker.BrokerTestUtil.spyWithClassAndConstructorArgsRecordingInvocations; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyInt; @@ -1847,14 +1848,14 @@ public void testBrokerClosedProducerClientRecreatesProducerThenSendCommand() thr ByteBuf clientCommand1 = Commands.newProducer(successTopicName, 1 /* producer id */, 1 /* request id */, producerName, Collections.emptyMap(), false); channel.writeInbound(clientCommand1); - assertTrue(getResponse() instanceof CommandProducerSuccess); + assertThat(getResponse()).isInstanceOf(CommandProducerSuccess.class); // Call disconnect method on producer to trigger activity similar to unloading Producer producer = serverCnx.getProducers().get(1).get(); assertNotNull(producer); producer.disconnect(); channel.runPendingTasks(); - assertTrue(getResponse() instanceof CommandCloseProducer); + assertThat(getResponse()).isInstanceOf(CommandCloseProducer.class); // Send message and expect no response sendMessage(); 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 b11946069c9dd..fd08f284bbf99 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 @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service; +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; @@ -128,6 +129,29 @@ public void testSeek() throws Exception { assertEquals(sub.getNumberOfEntriesInBacklog(false), 0); } + @Test + public void testSeekIsByReceive() throws PulsarClientException { + final String topicName = "persistent://prop/use/ns-abc/testSeek"; + + Producer producer = pulsarClient.newProducer().topic(topicName).create(); + + String subscriptionName = "my-subscription"; + org.apache.pulsar.client.api.Consumer consumer = pulsarClient.newConsumer().topic(topicName) + .subscriptionName(subscriptionName) + .subscribe(); + + List messageIds = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + String message = "my-message-" + i; + MessageId msgId = producer.send(message.getBytes()); + messageIds.add(msgId); + } + + consumer.seek(messageIds.get(5)); + Message message = consumer.receive(); + assertThat(message.getMessageId()).isEqualTo(messageIds.get(6)); + } + @Test public void testSeekForBatch() throws Exception { final String topicName = "persistent://prop/use/ns-abcd/testSeekForBatch"; 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 981feb002871e..ff8e418c024a0 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 @@ -19,6 +19,8 @@ package org.apache.pulsar.broker.service.persistent; import static org.apache.bookkeeper.mledger.impl.ManagedCursorImpl.CURSOR_INTERNAL_PROPERTY_PREFIX; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; @@ -42,7 +44,6 @@ import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.delayed.BucketDelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.service.Dispatcher; -import org.apache.pulsar.broker.stats.PrometheusMetricsTest; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; @@ -219,9 +220,9 @@ public void testBucketDelayedIndexMetrics() throws Exception { ByteArrayOutputStream output = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsar, true, true, true, output); String metricsStr = output.toString(StandardCharsets.UTF_8); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metricsMap = parseMetrics(metricsStr); - List bucketsMetrics = + List bucketsMetrics = metricsMap.get("pulsar_delayed_message_index_bucket_total").stream() .filter(metric -> metric.tags.get("topic").equals(topic)).toList(); MutableInt bucketsSum = new MutableInt(); @@ -230,12 +231,12 @@ public void testBucketDelayedIndexMetrics() throws Exception { bucketsSum.add(metric.value); }); assertEquals(6, bucketsSum.intValue()); - Optional bucketsTopicMetric = + Optional bucketsTopicMetric = bucketsMetrics.stream().filter(metric -> !metric.tags.containsKey("subscription")).findFirst(); assertTrue(bucketsTopicMetric.isPresent()); assertEquals(bucketsSum.intValue(), bucketsTopicMetric.get().value); - List loadedIndexMetrics = + List loadedIndexMetrics = metricsMap.get("pulsar_delayed_message_index_loaded").stream() .filter(metric -> metric.tags.get("topic").equals(topic)).toList(); MutableInt loadedIndexSum = new MutableInt(); @@ -244,12 +245,12 @@ public void testBucketDelayedIndexMetrics() throws Exception { loadedIndexSum.add(metric.value); }).count(); assertEquals(2, count); - Optional loadedIndexTopicMetrics = + Optional loadedIndexTopicMetrics = bucketsMetrics.stream().filter(metric -> !metric.tags.containsKey("subscription")).findFirst(); assertTrue(loadedIndexTopicMetrics.isPresent()); assertEquals(loadedIndexSum.intValue(), loadedIndexTopicMetrics.get().value); - List snapshotSizeBytesMetrics = + List snapshotSizeBytesMetrics = metricsMap.get("pulsar_delayed_message_index_bucket_snapshot_size_bytes").stream() .filter(metric -> metric.tags.get("topic").equals(topic)).toList(); MutableInt snapshotSizeBytesSum = new MutableInt(); @@ -259,12 +260,12 @@ public void testBucketDelayedIndexMetrics() throws Exception { snapshotSizeBytesSum.add(metric.value); }).count(); assertEquals(2, count); - Optional snapshotSizeBytesTopicMetrics = + Optional snapshotSizeBytesTopicMetrics = snapshotSizeBytesMetrics.stream().filter(metric -> !metric.tags.containsKey("subscription")).findFirst(); assertTrue(snapshotSizeBytesTopicMetrics.isPresent()); assertEquals(snapshotSizeBytesSum.intValue(), snapshotSizeBytesTopicMetrics.get().value); - List opCountMetrics = + List opCountMetrics = metricsMap.get("pulsar_delayed_message_index_bucket_op_count").stream() .filter(metric -> metric.tags.get("topic").equals(topic)).toList(); MutableInt opCountMetricsSum = new MutableInt(); @@ -276,14 +277,14 @@ public void testBucketDelayedIndexMetrics() throws Exception { opCountMetricsSum.add(metric.value); }).count(); assertEquals(2, count); - Optional opCountTopicMetrics = + Optional opCountTopicMetrics = opCountMetrics.stream() .filter(metric -> metric.tags.get("state").equals("succeed") && metric.tags.get("type") .equals("create") && !metric.tags.containsKey("subscription")).findFirst(); assertTrue(opCountTopicMetrics.isPresent()); assertEquals(opCountMetricsSum.intValue(), opCountTopicMetrics.get().value); - List opLatencyMetrics = + List opLatencyMetrics = metricsMap.get("pulsar_delayed_message_index_bucket_op_latency_ms").stream() .filter(metric -> metric.tags.get("topic").equals(topic)).toList(); MutableInt opLatencyMetricsSum = new MutableInt(); @@ -295,7 +296,7 @@ public void testBucketDelayedIndexMetrics() throws Exception { opLatencyMetricsSum.add(metric.value); }).count(); assertTrue(count >= 2); - Optional opLatencyTopicMetrics = + Optional opLatencyTopicMetrics = opCountMetrics.stream() .filter(metric -> metric.tags.get("type").equals("create") && !metric.tags.containsKey("subscription")).findFirst(); @@ -304,9 +305,9 @@ public void testBucketDelayedIndexMetrics() throws Exception { ByteArrayOutputStream namespaceOutput = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsar, false, true, true, namespaceOutput); - Multimap namespaceMetricsMap = PrometheusMetricsTest.parseMetrics(namespaceOutput.toString(StandardCharsets.UTF_8)); + Multimap namespaceMetricsMap = parseMetrics(namespaceOutput.toString(StandardCharsets.UTF_8)); - Optional namespaceMetric = + Optional namespaceMetric = namespaceMetricsMap.get("pulsar_delayed_message_index_bucket_total").stream().findFirst(); assertTrue(namespaceMetric.isPresent()); assertEquals(6, namespaceMetric.get().value); 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 8130c818e3a8a..d523586c2e2d3 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 @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker.service.persistent; +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.ArgumentMatchers.anyInt; import static org.mockito.Mockito.doAnswer; @@ -71,7 +73,6 @@ import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TopicPoliciesService; -import org.apache.pulsar.broker.stats.PrometheusMetricsTest; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -448,14 +449,14 @@ public void testDelayedDeliveryTrackerMemoryUsageMetric(String topic, boolean ex PrometheusMetricsTestUtil.generate(pulsar, exposeTopicLevelMetrics, true, true, output); String metricsStr = output.toString(StandardCharsets.UTF_8); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); - Collection metrics = metricsMap.get("pulsar_delayed_message_index_size_bytes"); + Multimap metricsMap = parseMetrics(metricsStr); + Collection metrics = metricsMap.get("pulsar_delayed_message_index_size_bytes"); Assert.assertTrue(metrics.size() > 0); int topicLevelNum = 0; int namespaceLevelNum = 0; int subscriptionLevelNum = 0; - for (PrometheusMetricsTest.Metric metric : metrics) { + for (Metric metric : metrics) { if (exposeTopicLevelMetrics && metric.tags.get("topic").equals(topic)) { Assert.assertTrue(metric.value > 0); topicLevelNum++; 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 913419a7f8a2e..9e455f60b6e41 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 @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker.service.schema; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.testng.Assert.fail; import static org.testng.AssertJUnit.assertEquals; import static org.testng.AssertJUnit.assertFalse; @@ -45,7 +47,6 @@ import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.schema.SchemaRegistry.SchemaAndMetadata; -import org.apache.pulsar.broker.stats.PrometheusMetricsTest; import org.apache.pulsar.client.impl.schema.KeyValueSchemaInfo; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; @@ -121,29 +122,29 @@ public void testSchemaRegistryMetrics() throws Exception { PrometheusMetricsTestUtil.generate(pulsar, false, false, false, output); output.flush(); String metricsStr = output.toString(StandardCharsets.UTF_8); - Multimap metrics = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection delMetrics = metrics.get("pulsar_schema_del_ops_failed_total"); + Collection delMetrics = metrics.get("pulsar_schema_del_ops_failed_total"); Assert.assertEquals(delMetrics.size(), 0); - Collection getMetrics = metrics.get("pulsar_schema_get_ops_failed_total"); + Collection getMetrics = metrics.get("pulsar_schema_get_ops_failed_total"); Assert.assertEquals(getMetrics.size(), 0); - Collection putMetrics = metrics.get("pulsar_schema_put_ops_failed_total"); + Collection putMetrics = metrics.get("pulsar_schema_put_ops_failed_total"); Assert.assertEquals(putMetrics.size(), 0); - Collection deleteLatency = metrics.get("pulsar_schema_del_ops_latency_count"); - for (PrometheusMetricsTest.Metric metric : deleteLatency) { + Collection deleteLatency = metrics.get("pulsar_schema_del_ops_latency_count"); + for (Metric metric : deleteLatency) { Assert.assertEquals(metric.tags.get("namespace"), namespace); Assert.assertTrue(metric.value > 0); } - Collection getLatency = metrics.get("pulsar_schema_get_ops_latency_count"); - for (PrometheusMetricsTest.Metric metric : getLatency) { + Collection getLatency = metrics.get("pulsar_schema_get_ops_latency_count"); + for (Metric metric : getLatency) { Assert.assertEquals(metric.tags.get("namespace"), namespace); Assert.assertTrue(metric.value > 0); } - Collection putLatency = metrics.get("pulsar_schema_put_ops_latency_count"); - for (PrometheusMetricsTest.Metric metric : putLatency) { + Collection putLatency = metrics.get("pulsar_schema_put_ops_latency_count"); + for (Metric metric : putLatency) { Assert.assertEquals(metric.tags.get("namespace"), namespace); Assert.assertTrue(metric.value > 0); } 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 de65d5db56475..512a5cfcab661 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 @@ -19,6 +19,8 @@ package org.apache.pulsar.broker.stats; 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.mockito.Mockito.mock; import static org.testng.Assert.assertNotEquals; import static org.testng.AssertJUnit.assertEquals; @@ -336,11 +338,11 @@ private void testMessageAckRateMetric(String topicName, boolean exposeTopicLevel PrometheusMetricsTestUtil.generate(pulsar, exposeTopicLevelMetrics, true, true, output); String metricStr = output.toString(StandardCharsets.UTF_8); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricStr); - Collection ackRateMetric = metricsMap.get("pulsar_consumer_msg_ack_rate"); + Multimap metricsMap = parseMetrics(metricStr); + Collection ackRateMetric = metricsMap.get("pulsar_consumer_msg_ack_rate"); String rateOutMetricName = exposeTopicLevelMetrics ? "pulsar_consumer_msg_rate_out" : "pulsar_rate_out"; - Collection rateOutMetric = metricsMap.get(rateOutMetricName); + Collection rateOutMetric = metricsMap.get(rateOutMetricName); Assert.assertTrue(ackRateMetric.size() > 0); Assert.assertTrue(rateOutMetric.size() > 0); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java index 7368d42355c1b..726bde3f3d0a9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/MetadataStoreStatsTest.java @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker.stats; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import com.google.common.collect.Multimap; import java.io.ByteArrayOutputStream; import java.util.Collection; @@ -101,12 +103,12 @@ public void testMetadataStoreStats() throws Exception { ByteArrayOutputStream output = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsar, false, false, false, false, output); String metricsStr = output.toString(); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metricsMap = parseMetrics(metricsStr); String metricsDebugMessage = "Assertion failed with metrics:\n" + metricsStr + "\n"; - Collection opsLatency = metricsMap.get("pulsar_metadata_store_ops_latency_ms" + "_sum"); - Collection putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total"); + Collection opsLatency = metricsMap.get("pulsar_metadata_store_ops_latency_ms" + "_sum"); + Collection putBytes = metricsMap.get("pulsar_metadata_store_put_bytes" + "_total"); Assert.assertTrue(opsLatency.size() > 1, metricsDebugMessage); Assert.assertTrue(putBytes.size() > 1, metricsDebugMessage); @@ -116,7 +118,7 @@ public void testMetadataStoreStats() throws Exception { expectedMetadataStoreName.add(MetadataStoreConfig.CONFIGURATION_METADATA_STORE); AtomicInteger matchCount = new AtomicInteger(0); - for (PrometheusMetricsTest.Metric m : opsLatency) { + for (Metric m : opsLatency) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (!isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { @@ -150,7 +152,7 @@ public void testMetadataStoreStats() throws Exception { Assert.assertEquals(matchCount.get(), expectedMetadataStoreName.size() * 6); matchCount = new AtomicInteger(0); - for (PrometheusMetricsTest.Metric m : putBytes) { + for (Metric m : putBytes) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (!isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { @@ -191,12 +193,12 @@ public void testBatchMetadataStoreMetrics() throws Exception { ByteArrayOutputStream output = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsar, false, false, false, false, output); String metricsStr = output.toString(); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metricsMap = parseMetrics(metricsStr); - Collection executorQueueSize = metricsMap.get("pulsar_batch_metadata_store_executor_queue_size"); - Collection opsWaiting = metricsMap.get("pulsar_batch_metadata_store_queue_wait_time_ms" + "_sum"); - Collection batchExecuteTime = metricsMap.get("pulsar_batch_metadata_store_batch_execute_time_ms" + "_sum"); - Collection opsPerBatch = metricsMap.get("pulsar_batch_metadata_store_batch_size" + "_sum"); + Collection executorQueueSize = metricsMap.get("pulsar_batch_metadata_store_executor_queue_size"); + Collection opsWaiting = metricsMap.get("pulsar_batch_metadata_store_queue_wait_time_ms" + "_sum"); + Collection batchExecuteTime = metricsMap.get("pulsar_batch_metadata_store_batch_execute_time_ms" + "_sum"); + Collection opsPerBatch = metricsMap.get("pulsar_batch_metadata_store_batch_size" + "_sum"); String metricsDebugMessage = "Assertion failed with metrics:\n" + metricsStr + "\n"; @@ -210,7 +212,7 @@ public void testBatchMetadataStoreMetrics() throws Exception { expectedMetadataStoreName.add(MetadataStoreConfig.CONFIGURATION_METADATA_STORE); AtomicInteger matchCount = new AtomicInteger(0); - for (PrometheusMetricsTest.Metric m : executorQueueSize) { + for (Metric m : executorQueueSize) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { @@ -221,7 +223,7 @@ public void testBatchMetadataStoreMetrics() throws Exception { Assert.assertEquals(matchCount.get(), expectedMetadataStoreName.size()); matchCount = new AtomicInteger(0); - for (PrometheusMetricsTest.Metric m : opsWaiting) { + for (Metric m : opsWaiting) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { @@ -232,7 +234,7 @@ public void testBatchMetadataStoreMetrics() throws Exception { Assert.assertEquals(matchCount.get(), expectedMetadataStoreName.size()); matchCount = new AtomicInteger(0); - for (PrometheusMetricsTest.Metric m : batchExecuteTime) { + for (Metric m : batchExecuteTime) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { @@ -243,7 +245,7 @@ public void testBatchMetadataStoreMetrics() throws Exception { Assert.assertEquals(matchCount.get(), expectedMetadataStoreName.size()); matchCount = new AtomicInteger(0); - for (PrometheusMetricsTest.Metric m : opsPerBatch) { + for (Metric m : opsPerBatch) { Assert.assertEquals(m.tags.get("cluster"), "test", metricsDebugMessage); String metadataStoreName = m.tags.get("name"); if (isExpectedLabel(metadataStoreName, expectedMetadataStoreName, matchCount)) { 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 244d9eaa81274..7cca1e876b4b2 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 @@ -21,14 +21,14 @@ import static com.google.common.base.Preconditions.checkArgument; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; -import com.google.common.base.MoreObjects; import com.google.common.base.Splitter; -import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; import io.jsonwebtoken.SignatureAlgorithm; import io.prometheus.client.Collector; @@ -53,7 +53,6 @@ import java.util.Properties; import java.util.Random; import java.util.Set; -import java.util.TreeMap; import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; @@ -1920,66 +1919,6 @@ public void testMetricsGroupedByTypeDefinitions() throws Exception { p2.close(); } - /** - * Hacky parsing of Prometheus text format. Should be good enough for unit tests - */ - public static Multimap parseMetrics(String metrics) { - Multimap parsed = ArrayListMultimap.create(); - - // Example of lines are - // jvm_threads_current{cluster="standalone",} 203.0 - // or - // pulsar_subscriptions_count{cluster="standalone", namespace="public/default", - // topic="persistent://public/default/test-2"} 0.0 - Pattern pattern = Pattern.compile("^(\\w+)\\{([^\\}]+)\\}\\s([+-]?[\\d\\w\\.-]+)$"); - Pattern tagsPattern = Pattern.compile("(\\w+)=\"([^\"]+)\"(,\\s?)?"); - - Splitter.on("\n").split(metrics).forEach(line -> { - if (line.isEmpty() || line.startsWith("#")) { - return; - } - - Matcher matcher = pattern.matcher(line); - assertTrue(matcher.matches(), "line " + line + " does not match pattern " + pattern); - String name = matcher.group(1); - - Metric m = new Metric(); - String numericValue = matcher.group(3); - if (numericValue.equalsIgnoreCase("-Inf")) { - m.value = Double.NEGATIVE_INFINITY; - } else if (numericValue.equalsIgnoreCase("+Inf")) { - m.value = Double.POSITIVE_INFINITY; - } else { - m.value = Double.parseDouble(numericValue); - } - String tags = matcher.group(2); - Matcher tagsMatcher = tagsPattern.matcher(tags); - while (tagsMatcher.find()) { - String tag = tagsMatcher.group(1); - String value = tagsMatcher.group(2); - m.tags.put(tag, value); - } - - parsed.put(name, m); - }); - - return parsed; - } - - public static class Metric { - public Map tags = new TreeMap<>(); - public double value; - - @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("tags", tags).add("value", value).toString(); - } - } - - /** - * Test both subscription and topic name with special characters. - * @throws Exception - */ @Test public void testEscapeLabelValue() throws Exception { String ns1 = "prop/ns-abc1"; 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 83e6f43cbafcc..3e71d8f211101 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 @@ -19,6 +19,8 @@ package org.apache.pulsar.broker.stats; 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.mockito.Mockito.mock; import com.google.common.collect.Multimap; import java.io.ByteArrayOutputStream; @@ -84,7 +86,7 @@ protected void cleanup() throws Exception { @Test public void testConsumersAfterMarkDelete() throws PulsarClientException, PulsarAdminException { final String topicName = "persistent://my-property/my-ns/testConsumersAfterMarkDelete-" - + UUID.randomUUID().toString(); + + UUID.randomUUID(); final String subName = "my-sub"; Consumer consumer1 = pulsarClient.newConsumer() @@ -233,15 +235,15 @@ public void testSubscriptionStats(final String topic, final String subName, bool ByteArrayOutputStream output = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsar, enableTopicStats, false, false, output); String metricsStr = output.toString(); - Multimap metrics = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection throughFilterMetrics = + Collection throughFilterMetrics = metrics.get("pulsar_subscription_filter_processed_msg_count"); - Collection acceptedMetrics = + Collection acceptedMetrics = metrics.get("pulsar_subscription_filter_accepted_msg_count"); - Collection rejectedMetrics = + Collection rejectedMetrics = metrics.get("pulsar_subscription_filter_rejected_msg_count"); - Collection rescheduledMetrics = + Collection rescheduledMetrics = metrics.get("pulsar_subscription_filter_rescheduled_msg_count"); if (enableTopicStats) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java index 85c6dd795d759..8d5cb9dc39148 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/TransactionMetricsTest.java @@ -19,7 +19,8 @@ package org.apache.pulsar.broker.stats; import static com.google.common.base.Preconditions.checkArgument; -import static org.apache.pulsar.broker.stats.PrometheusMetricsTest.parseMetrics; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; @@ -119,8 +120,8 @@ public void testTransactionCoordinatorMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); - Collection metric = metrics.get("pulsar_txn_active_count"); + Multimap metrics = parseMetrics(metricsStr); + Collection metric = metrics.get("pulsar_txn_active_count"); assertEquals(metric.size(), 2); metric.forEach(item -> { if ("0".equals(item.tags.get("coordinator_id"))) { @@ -187,9 +188,9 @@ public void testTransactionCoordinatorRateMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection metric = metrics.get("pulsar_txn_created_total"); + Collection metric = metrics.get("pulsar_txn_created_total"); assertEquals(metric.size(), 1); metric.forEach(item -> assertEquals(item.value, txnCount)); @@ -274,9 +275,9 @@ public void testManagedLedgerMetrics() throws Exception { PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection metric = metrics.get("pulsar_storage_size"); + Collection metric = metrics.get("pulsar_storage_size"); checkManagedLedgerMetrics(subName, 32, metric); checkManagedLedgerMetrics(MLTransactionLogImpl.TRANSACTION_SUBSCRIPTION_NAME, 252, metric); @@ -336,12 +337,12 @@ public void testManagedLedgerMetricsWhenPendingAckNotInit() throws Exception { PrometheusMetricsTestUtil.generate(pulsar, true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection metric = metrics.get("pulsar_storage_size"); + Collection metric = metrics.get("pulsar_storage_size"); checkManagedLedgerMetrics(subName, 32, metric); //No statistics of the pendingAck are generated when the pendingAck is not initialized. - for (PrometheusMetricsTest.Metric metric1 : metric) { + for (Metric metric1 : metric) { if (metric1.tags.containsValue(subName2)) { Assert.fail(); } @@ -431,9 +432,9 @@ public void testDuplicateMetricTypeDefinitions() throws Exception { } - private void checkManagedLedgerMetrics(String tag, double value, Collection metrics) { + private void checkManagedLedgerMetrics(String tag, double value, Collection metrics) { boolean exist = false; - for (PrometheusMetricsTest.Metric metric1 : metrics) { + for (Metric metric1 : metrics) { if (metric1.tags.containsValue(tag)) { assertEquals(metric1.value, value); exist = true; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregatorTest.java index e63f644f3d0e9..cf923df0411dd 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregatorTest.java @@ -32,6 +32,7 @@ import org.apache.pulsar.broker.service.Subscription; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.service.persistent.PersistentTopicMetrics; import org.apache.pulsar.common.policies.data.BacklogQuota; import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; import org.apache.pulsar.common.policies.data.stats.SubscriptionStatsImpl; @@ -102,6 +103,8 @@ public void testGenerateSubscriptionsStats() { when(topic.getReplicators()).thenReturn(ConcurrentOpenHashMap.newBuilder().build()); when(topic.getManagedLedger()).thenReturn(ml); when(topic.getBacklogQuota(Mockito.any())).thenReturn(Mockito.mock(BacklogQuota.class)); + PersistentTopicMetrics persistentTopicMetrics = new PersistentTopicMetrics(); + when(topic.getPersistentTopicMetrics()).thenReturn(persistentTopicMetrics); topicsMap.put("my-topic", topic); PrometheusMetricStreams metricStreams = Mockito.spy(new PrometheusMetricStreams()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java new file mode 100644 index 0000000000000..6fd509690278d --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/PrometheusMetricsClient.java @@ -0,0 +1,154 @@ +/* + * 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.assertj.core.api.Fail.fail; +import static org.testng.Assert.assertTrue; +import com.google.common.base.MoreObjects; +import com.google.common.base.Splitter; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.Multimap; +import io.restassured.RestAssured; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.apache.commons.lang3.tuple.Pair; + +public class PrometheusMetricsClient { + private final String host; + private final int port; + + public PrometheusMetricsClient(String host, int port) { + this.host = host; + this.port = port; + } + + @SuppressWarnings("HttpUrlsUsage") + public Metrics getMetrics() { + String metrics = RestAssured.given().baseUri("http://" + host).port(port).get("/metrics").asString(); + return new Metrics(parseMetrics(metrics)); + } + + /** + * Hacky parsing of Prometheus text format. Should be good enough for unit tests + */ + public static Multimap parseMetrics(String metrics) { + Multimap parsed = ArrayListMultimap.create(); + + // Example of lines are + // jvm_threads_current{cluster="standalone",} 203.0 + // or + // pulsar_subscriptions_count{cluster="standalone", namespace="public/default", + // topic="persistent://public/default/test-2"} 0.0 + Pattern pattern = Pattern.compile("^(\\w+)\\{([^}]+)}\\s([+-]?[\\d\\w.-]+)$"); + Pattern tagsPattern = Pattern.compile("(\\w+)=\"([^\"]+)\"(,\\s?)?"); + + Splitter.on("\n").split(metrics).forEach(line -> { + if (line.isEmpty() || line.startsWith("#")) { + return; + } + + Matcher matcher = pattern.matcher(line); + assertTrue(matcher.matches(), "line " + line + " does not match pattern " + pattern); + String name = matcher.group(1); + + Metric m = new Metric(); + String numericValue = matcher.group(3); + if (numericValue.equalsIgnoreCase("-Inf")) { + m.value = Double.NEGATIVE_INFINITY; + } else if (numericValue.equalsIgnoreCase("+Inf")) { + m.value = Double.POSITIVE_INFINITY; + } else { + m.value = Double.parseDouble(numericValue); + } + String tags = matcher.group(2); + Matcher tagsMatcher = tagsPattern.matcher(tags); + while (tagsMatcher.find()) { + String tag = tagsMatcher.group(1); + String value = tagsMatcher.group(2); + m.tags.put(tag, value); + } + + parsed.put(name, m); + }); + + return parsed; + } + + public static class Metric { + public Map tags = new TreeMap<>(); + public double value; + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("tags", tags).add("value", value).toString(); + } + + public boolean contains(String labelName, String labelValue) { + String value = tags.get(labelName); + return value != null && value.equals(labelValue); + } + } + + public static class Metrics { + final Multimap nameToDataPoints; + + public Metrics(Multimap nameToDataPoints) { + this.nameToDataPoints = nameToDataPoints; + } + + public List findByNameAndLabels(String metricName, String labelName, String labelValue) { + return nameToDataPoints.get(metricName) + .stream() + .filter(metric -> metric.contains(labelName, labelValue)) + .toList(); + } + + @SafeVarargs + public final List findByNameAndLabels(String metricName, Pair... nameValuePairs) { + return nameToDataPoints.get(metricName) + .stream() + .filter(metric -> { + for (Pair nameValuePair : nameValuePairs) { + String labelName = nameValuePair.getLeft(); + String labelValue = nameValuePair.getRight(); + if (!metric.contains(labelName, labelValue)) { + return false; + } + } + return true; + }) + .toList(); + } + + @SafeVarargs + public final Metric findSingleMetricByNameAndLabels(String metricName, Pair... nameValuePairs) { + List metricByNameAndLabels = findByNameAndLabels(metricName, nameValuePairs); + if (metricByNameAndLabels.size() != 1) { + fail("Expected to find 1 metric, but found the following: "+metricByNameAndLabels + + ". Metrics are = "+nameToDataPoints.get(metricName)+". Labels requested = "+ Arrays.toString( + nameValuePairs)); + } + return metricByNameAndLabels.get(0); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java index 0211b860cc4fe..1bc298ac601c8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionBufferClientTest.java @@ -18,28 +18,36 @@ */ package org.apache.pulsar.broker.transaction.buffer; +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.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyString; +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 static org.testng.Assert.fail; import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; import io.netty.util.HashedWheelTimer; import io.netty.util.concurrent.DefaultThreadFactory; -import lombok.Cleanup; import java.io.ByteArrayOutputStream; +import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ExecutionException; - import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; -import org.apache.pulsar.broker.stats.PrometheusMetricsTest; +import java.util.concurrent.TimeUnit; +import lombok.Cleanup; import org.apache.pulsar.broker.transaction.TransactionTestBase; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferClientImpl; import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferHandlerImpl; @@ -70,14 +78,6 @@ import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import java.lang.reflect.Field; -import java.util.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.TimeUnit; -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 static org.testng.Assert.fail; @Test(groups = "broker") public class TransactionBufferClientTest extends TransactionTestBase { @@ -228,28 +228,28 @@ public void testTransactionBufferMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsarServiceList.get(0), true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metricsMap = parseMetrics(metricsStr); - Collection abortFailed = metricsMap.get("pulsar_txn_tb_client_abort_failed_total"); - Collection commitFailed = metricsMap.get("pulsar_txn_tb_client_commit_failed_total"); - Collection abortLatencyCount = + Collection abortFailed = metricsMap.get("pulsar_txn_tb_client_abort_failed_total"); + Collection commitFailed = metricsMap.get("pulsar_txn_tb_client_commit_failed_total"); + Collection abortLatencyCount = metricsMap.get("pulsar_txn_tb_client_abort_latency_count"); - Collection commitLatencyCount = + Collection commitLatencyCount = metricsMap.get("pulsar_txn_tb_client_commit_latency_count"); - Collection pending = metricsMap.get("pulsar_txn_tb_client_pending_requests"); + Collection pending = metricsMap.get("pulsar_txn_tb_client_pending_requests"); assertEquals(abortFailed.stream().mapToDouble(metric -> metric.value).sum(), 0); assertEquals(commitFailed.stream().mapToDouble(metric -> metric.value).sum(), 0); for (int i = 0; i < partitions; i++) { String topic = partitionedTopicName.getPartition(i).toString(); - Optional optional = abortLatencyCount.stream() + Optional optional = abortLatencyCount.stream() .filter(metric -> metric.tags.get("topic").equals(topic)).findFirst(); assertTrue(optional.isPresent()); assertEquals(optional.get().value, 1D); - Optional optional1 = commitLatencyCount.stream() + Optional optional1 = commitLatencyCount.stream() .filter(metric -> metric.tags.get("topic").equals(topic)).findFirst(); assertTrue(optional1.isPresent()); assertEquals(optional1.get().value, 1D); 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 f5ac68950b49d..93a2f274517d5 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 @@ -19,6 +19,8 @@ package org.apache.pulsar.broker.transaction.pendingack; +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.mock; @@ -28,6 +30,7 @@ import static org.testng.AssertJUnit.assertNotNull; import static org.testng.AssertJUnit.assertTrue; import static org.testng.AssertJUnit.fail; +import com.google.common.collect.Multimap; import java.io.ByteArrayOutputStream; import java.lang.reflect.Field; import java.lang.reflect.Method; @@ -40,7 +43,6 @@ import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import com.google.common.collect.Multimap; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.ManagedCursor; @@ -54,7 +56,6 @@ import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; -import org.apache.pulsar.broker.stats.PrometheusMetricsTest; import org.apache.pulsar.broker.transaction.TransactionTestBase; import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore; import org.apache.pulsar.broker.transaction.pendingack.impl.PendingAckHandleImpl; @@ -67,9 +68,9 @@ import org.apache.pulsar.client.api.Schema; 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.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; -import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.impl.transaction.TransactionImpl; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.SystemTopicNames; @@ -338,28 +339,28 @@ public void testPendingAckMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsarServiceList.get(0), true, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metricsMap = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metricsMap = parseMetrics(metricsStr); - Collection abortedCount = metricsMap.get("pulsar_txn_tp_aborted_count_total"); - Collection committedCount = metricsMap.get("pulsar_txn_tp_committed_count_total"); - Collection commitLatency = metricsMap.get("pulsar_txn_tp_commit_latency"); + Collection abortedCount = metricsMap.get("pulsar_txn_tp_aborted_count_total"); + Collection committedCount = metricsMap.get("pulsar_txn_tp_committed_count_total"); + Collection commitLatency = metricsMap.get("pulsar_txn_tp_commit_latency"); Assert.assertTrue(commitLatency.size() > 0); int count = 0; - for (PrometheusMetricsTest.Metric metric : commitLatency) { + for (Metric metric : commitLatency) { if (metric.tags.get("topic").endsWith(PENDING_ACK_REPLAY_TOPIC) && metric.value > 0) { count++; } } Assert.assertTrue(count > 0); - for (PrometheusMetricsTest.Metric metric : abortedCount) { + for (Metric metric : abortedCount) { if (metric.tags.get("subscription").equals(subName) && metric.tags.get("status").equals("succeed")) { assertTrue(metric.tags.get("topic").endsWith(PENDING_ACK_REPLAY_TOPIC)); assertTrue(metric.value > 0); } } - for (PrometheusMetricsTest.Metric metric : committedCount) { + for (Metric metric : committedCount) { if (metric.tags.get("subscription").equals(subName) && metric.tags.get("status").equals("succeed")) { assertTrue(metric.tags.get("topic").endsWith(PENDING_ACK_REPLAY_TOPIC)); assertTrue(metric.value > 0); 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 999f22dc7f92b..9b0f8d798145a 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 @@ -18,6 +18,8 @@ */ package org.apache.pulsar.broker.web; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.Metric; +import static org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient.parseMetrics; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; @@ -56,7 +58,6 @@ import org.apache.pulsar.PrometheusMetricsTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.broker.stats.PrometheusMetricsTest; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminBuilder; @@ -108,31 +109,31 @@ public void testWebExecutorMetrics() throws Exception { ByteArrayOutputStream statsOut = new ByteArrayOutputStream(); PrometheusMetricsTestUtil.generate(pulsar, false, false, false, statsOut); String metricsStr = statsOut.toString(); - Multimap metrics = PrometheusMetricsTest.parseMetrics(metricsStr); + Multimap metrics = parseMetrics(metricsStr); - Collection maxThreads = metrics.get("pulsar_web_executor_max_threads"); - Collection minThreads = metrics.get("pulsar_web_executor_min_threads"); - Collection activeThreads = metrics.get("pulsar_web_executor_active_threads"); - Collection idleThreads = metrics.get("pulsar_web_executor_idle_threads"); - Collection currentThreads = metrics.get("pulsar_web_executor_current_threads"); + Collection maxThreads = metrics.get("pulsar_web_executor_max_threads"); + Collection minThreads = metrics.get("pulsar_web_executor_min_threads"); + Collection activeThreads = metrics.get("pulsar_web_executor_active_threads"); + Collection idleThreads = metrics.get("pulsar_web_executor_idle_threads"); + Collection currentThreads = metrics.get("pulsar_web_executor_current_threads"); - for (PrometheusMetricsTest.Metric metric : maxThreads) { + for (Metric metric : maxThreads) { Assert.assertNotNull(metric.tags.get("cluster")); Assert.assertTrue(metric.value > 0); } - for (PrometheusMetricsTest.Metric metric : minThreads) { + for (Metric metric : minThreads) { Assert.assertNotNull(metric.tags.get("cluster")); Assert.assertTrue(metric.value > 0); } - for (PrometheusMetricsTest.Metric metric : activeThreads) { + for (Metric metric : activeThreads) { Assert.assertNotNull(metric.tags.get("cluster")); Assert.assertTrue(metric.value >= 0); } - for (PrometheusMetricsTest.Metric metric : idleThreads) { + for (Metric metric : idleThreads) { Assert.assertNotNull(metric.tags.get("cluster")); Assert.assertTrue(metric.value >= 0); } - for (PrometheusMetricsTest.Metric metric : currentThreads) { + for (Metric metric : currentThreads) { Assert.assertNotNull(metric.tags.get("cluster")); Assert.assertTrue(metric.value > 0); } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/TopicStats.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/TopicStats.java index 985e42b280eb9..ac50763b7e097 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/TopicStats.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/TopicStats.java @@ -64,6 +64,31 @@ public interface TopicStats { /** Get the publish time of the earliest message over all the backlogs. */ long getEarliestMsgPublishTimeInBacklogs(); + /** the size in bytes of the topic backlog quota. */ + long getBacklogQuotaLimitSize(); + + /** the topic backlog age quota, in seconds. */ + long getBacklogQuotaLimitTime(); + + /** + * Age of oldest unacknowledged message, as recorded in last backlog quota check interval. + *

    + * The age of the oldest unacknowledged (i.e. backlog) message, measured by the time elapsed from its published + * time, in seconds. This value is recorded every backlog quota check interval, hence it represents the value + * seen in the last check. + *

    + */ + long getOldestBacklogMessageAgeSeconds(); + + /** + * The subscription name containing oldest unacknowledged message as recorded in last backlog quota check. + *

    + * The name of the subscription containing the oldest unacknowledged message. This value is recorded every backlog + * quota check interval, hence it represents the value seen in the last check. + *

    + */ + String getOldestBacklogMessageSubscriptionName(); + /** Space used to store the offloaded messages for the topic/. */ long getOffloadedStorageSize(); diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java index c67ad08c83631..f437b28fe1f8c 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java @@ -474,6 +474,9 @@ CompletableFuture reconsumeLaterCumulativeAsync(Message message, *
  • 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 * + *

    + * This effectively resets the acknowledgement state of the subscription: all messages up to and + * including messageId will be marked as acknowledged and the rest unacknowledged. * *

    Note: For multi-topics consumer, if `messageId` is a {@link TopicMessageId}, the seek operation will happen * on the owner topic of the message, which is returned by {@link TopicMessageId#getOwnerTopic()}. Otherwise, you diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java index e50620fb22398..c68bfdbd5d3be 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/policies/data/stats/TopicStatsImpl.java @@ -84,6 +84,31 @@ public class TopicStatsImpl implements TopicStats { /** Get estimated total unconsumed or backlog size in bytes. */ public long backlogSize; + /** the size in bytes of the topic backlog quota. */ + public long backlogQuotaLimitSize; + + /** the topic backlog age quota, in seconds. */ + public long backlogQuotaLimitTime; + + /** + * Age of oldest unacknowledged message, as recorded in last backlog quota check interval. + *

    + * The age of the oldest unacknowledged (i.e. backlog) message, measured by the time elapsed from its published + * time, in seconds. This value is recorded every backlog quota check interval, hence it represents the value + * seen in the last check. + *

    + */ + public long oldestBacklogMessageAgeSeconds; + + /** + * The subscription name containing oldest unacknowledged message as recorded in last backlog quota check. + *

    + * The name of the subscription containing the oldest unacknowledged message. This value is recorded every backlog + * quota check interval, hence it represents the value seen in the last check. + *

    + */ + public String oldestBacklogMessageSubscriptionName; + /** The number of times the publishing rate limit was triggered. */ public long publishRateLimitedTimes; @@ -221,6 +246,10 @@ public void reset() { this.compaction.reset(); this.ownerBroker = null; this.bucketDelayedIndexStats.clear(); + this.backlogQuotaLimitSize = 0; + this.backlogQuotaLimitTime = 0; + this.oldestBacklogMessageAgeSeconds = -1; + this.oldestBacklogMessageSubscriptionName = null; } // if the stats are added for the 1st time, we will need to make a copy of these stats and add it to the current @@ -250,6 +279,12 @@ public TopicStatsImpl add(TopicStats ts) { this.ongoingTxnCount = stats.ongoingTxnCount; this.abortedTxnCount = stats.abortedTxnCount; this.committedTxnCount = stats.committedTxnCount; + this.backlogQuotaLimitTime = stats.backlogQuotaLimitTime; + this.backlogQuotaLimitSize = stats.backlogQuotaLimitSize; + if (stats.oldestBacklogMessageAgeSeconds > this.oldestBacklogMessageAgeSeconds) { + this.oldestBacklogMessageAgeSeconds = stats.oldestBacklogMessageAgeSeconds; + this.oldestBacklogMessageSubscriptionName = stats.oldestBacklogMessageSubscriptionName; + } stats.bucketDelayedIndexStats.forEach((k, v) -> { TopicMetricBean topicMetricBean = From f54d72dc7405931ac8fa4a8342b21bcbc76b78cd Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Tue, 19 Dec 2023 12:00:54 +0800 Subject: [PATCH 514/699] [improve][broker] checkTopicExists supports checking partitioned topic without index (#21701) Signed-off-by: Zixuan Liu (cherry picked from commit 8b8048c4b605650720878fa5a549f36376a3aa79) --- .../broker/namespace/NamespaceService.java | 66 +++++++++---------- .../namespace/NamespaceServiceTest.java | 27 ++++++++ 2 files changed, 59 insertions(+), 34 deletions(-) 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 873a4493c8bde..626a187f46388 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 @@ -1313,42 +1313,40 @@ public CompletableFuture> getOwnedTopicListForNamespaceBundle(Names } public CompletableFuture checkTopicExists(TopicName topic) { - if (topic.isPersistent()) { - if (topic.isPartitioned()) { - return pulsar.getBrokerService() - .fetchPartitionedTopicMetadataAsync(TopicName.get(topic.getPartitionedTopicName())) - .thenCompose(metadata -> { - // Allow creating the non-partitioned persistent topic that name includes `-partition-` - if (metadata.partitions == 0 - || topic.getPartitionIndex() < metadata.partitions) { - return pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); - } - return CompletableFuture.completedFuture(false); - }); - } else { - return pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); - } + CompletableFuture future; + // If the topic is persistent and the name includes `-partition-`, find the topic from the managed/ledger. + if (topic.isPersistent() && topic.isPartitioned()) { + future = pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); } else { - if (topic.isPartitioned()) { - final TopicName partitionedTopicName = TopicName.get(topic.getPartitionedTopicName()); - return pulsar.getBrokerService() - .fetchPartitionedTopicMetadataAsync(partitionedTopicName) - .thenApply((metadata) -> topic.getPartitionIndex() < metadata.partitions); - } else { - // only checks and don't do any topic creating and loading. - CompletableFuture> topicFuture = - pulsar.getBrokerService().getTopics().get(topic.toString()); - if (topicFuture == null) { - return CompletableFuture.completedFuture(false); - } else { - return topicFuture.thenApply(Optional::isPresent).exceptionally(throwable -> { - LOG.warn("[{}] topicFuture completed with exception when checkTopicExists, {}", - topic, throwable.getMessage()); - return false; - }); - } - } + future = CompletableFuture.completedFuture(false); } + + return future.thenCompose(found -> { + if (found != null && found) { + return CompletableFuture.completedFuture(true); + } + + return pulsar.getBrokerService() + .fetchPartitionedTopicMetadataAsync(TopicName.get(topic.getPartitionedTopicName())) + .thenCompose(metadata -> { + if (metadata.partitions > 0) { + return CompletableFuture.completedFuture(true); + } + + if (topic.isPersistent()) { + return pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); + } else { + // The non-partitioned non-persistent topic only exist in the broker topics. + CompletableFuture> nonPersistentTopicFuture = + pulsar.getBrokerService().getTopics().get(topic.toString()); + if (nonPersistentTopicFuture == null) { + return CompletableFuture.completedFuture(false); + } else { + return nonPersistentTopicFuture.thenApply(Optional::isPresent); + } + } + }); + }); } public CompletableFuture> getListOfTopics(NamespaceName namespaceName, Mode mode) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java index f3fabc910346d..2a8a849ef9c06 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java @@ -44,6 +44,7 @@ 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.ExecutorService; import java.util.concurrent.TimeUnit; @@ -72,6 +73,7 @@ import org.apache.pulsar.common.naming.NamespaceBundles; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.ServiceUnitId; +import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.BundlesData; import org.apache.pulsar.common.policies.data.LocalPolicies; @@ -94,6 +96,7 @@ import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Test(groups = "flaky") @@ -800,6 +803,30 @@ public void testModularLoadManagerRemoveBundleAndLoad() throws Exception { assertFalse(getResult.isPresent()); } + @DataProvider(name = "topicDomain") + public Object[] topicDomain() { + return new Object[]{ + TopicDomain.persistent.value(), + TopicDomain.non_persistent.value() + }; + } + + @Test(dataProvider = "topicDomain") + public void testCheckTopicExists(String topicDomain) throws Exception { + String topic = topicDomain + "://prop/ns-abc/" + UUID.randomUUID(); + admin.topics().createNonPartitionedTopic(topic); + Awaitility.await().untilAsserted(() -> { + assertTrue(pulsar.getNamespaceService().checkTopicExists(TopicName.get(topic)).get()); + }); + + String partitionedTopic = topicDomain + "://prop/ns-abc/" + UUID.randomUUID(); + admin.topics().createPartitionedTopic(partitionedTopic, 5); + Awaitility.await().untilAsserted(() -> { + assertTrue(pulsar.getNamespaceService().checkTopicExists(TopicName.get(partitionedTopic)).get()); + assertTrue(pulsar.getNamespaceService().checkTopicExists(TopicName.get(partitionedTopic + "-partition-2")).get()); + }); + } + /** * 1. Manually trigger "LoadReportUpdaterTask" * 2. Registry another new zk-node-listener "waitForBrokerChangeNotice". From 6b7e7272de58fa91f8c9f3bd8fcb1cce92ba634d Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 17 May 2024 01:18:49 +0800 Subject: [PATCH 515/699] [improve] [test] Add a test to guarantee the TNX topics will not be replicated (#22721) (cherry picked from commit 4e132d3f2919767cef6fa935a59721937cb668e7) (cherry picked from commit b11ab69898fe6091cb5c196bc2418924b8ebf7a7) --- .../broker/service/OneWayReplicatorTest.java | 9 - .../service/OneWayReplicatorTestBase.java | 44 ++- .../broker/service/ReplicationTxnTest.java | 262 ++++++++++++++++++ 3 files changed, 297 insertions(+), 18 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicationTxnTest.java 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 c6a6b5e4fb8ca..2eee20906d259 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 @@ -94,15 +94,6 @@ public void cleanup() throws Exception { super.cleanup(); } - private void waitReplicatorStarted(String topicName) { - Awaitility.await().untilAsserted(() -> { - Optional topicOptional2 = pulsar2.getBrokerService().getTopic(topicName, false).get(); - assertTrue(topicOptional2.isPresent()); - PersistentTopic persistentTopic2 = (PersistentTopic) topicOptional2.get(); - assertFalse(persistentTopic2.getProducers().isEmpty()); - }); - } - private void waitReplicatorStopped(String topicName) { Awaitility.await().untilAsserted(() -> { Optional topicOptional2 = pulsar2.getBrokerService().getTopic(topicName, false).get(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java index b4eed00c4470f..317e43306e356 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java @@ -19,6 +19,8 @@ package org.apache.pulsar.broker.service; import static org.apache.pulsar.compaction.Compactor.COMPACTION_SUBSCRIPTION; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; import com.google.common.collect.Sets; import java.net.URL; import java.time.Duration; @@ -29,6 +31,7 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.policies.data.ClusterData; @@ -55,7 +58,7 @@ public abstract class OneWayReplicatorTestBase extends TestRetrySupport { protected ZookeeperServerTest brokerConfigZk1; protected LocalBookkeeperEnsemble bkEnsemble1; protected PulsarService pulsar1; - protected BrokerService ns1; + protected BrokerService broker1; protected PulsarAdmin admin1; protected PulsarClient client1; @@ -66,7 +69,7 @@ public abstract class OneWayReplicatorTestBase extends TestRetrySupport { protected ZookeeperServerTest brokerConfigZk2; protected LocalBookkeeperEnsemble bkEnsemble2; protected PulsarService pulsar2; - protected BrokerService ns2; + protected BrokerService broker2; protected PulsarAdmin admin2; protected PulsarClient client2; @@ -89,23 +92,29 @@ protected void startBrokers() throws Exception { setConfigDefaults(config1, cluster1, bkEnsemble1, brokerConfigZk1); pulsar1 = new PulsarService(config1); pulsar1.start(); - ns1 = pulsar1.getBrokerService(); - + broker1 = pulsar1.getBrokerService(); url1 = new URL(pulsar1.getWebServiceAddress()); urlTls1 = new URL(pulsar1.getWebServiceAddressTls()); - admin1 = PulsarAdmin.builder().serviceHttpUrl(url1.toString()).build(); - client1 = PulsarClient.builder().serviceUrl(url1.toString()).build(); // Start region 2 setConfigDefaults(config2, cluster2, bkEnsemble2, brokerConfigZk2); pulsar2 = new PulsarService(config2); pulsar2.start(); - ns2 = pulsar2.getBrokerService(); - + broker2 = pulsar2.getBrokerService(); url2 = new URL(pulsar2.getWebServiceAddress()); urlTls2 = new URL(pulsar2.getWebServiceAddressTls()); + } + + protected void startAdminClient() throws Exception { + admin1 = PulsarAdmin.builder().serviceHttpUrl(url1.toString()).build(); admin2 = PulsarAdmin.builder().serviceHttpUrl(url2.toString()).build(); - client2 = PulsarClient.builder().serviceUrl(url2.toString()).build(); + } + + protected void startPulsarClient() throws Exception{ + ClientBuilder clientBuilder1 = PulsarClient.builder().serviceUrl(url1.toString()); + client1 = initClient(clientBuilder1); + ClientBuilder clientBuilder2 = PulsarClient.builder().serviceUrl(url2.toString()); + client2 = initClient(clientBuilder2); } protected void createDefaultTenantsAndClustersAndNamespace() throws Exception { @@ -196,8 +205,12 @@ protected void setup() throws Exception { startBrokers(); + startAdminClient(); + createDefaultTenantsAndClustersAndNamespace(); + startPulsarClient(); + Thread.sleep(100); log.info("--- OneWayReplicatorTestBase::setup completed ---"); } @@ -287,4 +300,17 @@ protected void cleanup() throws Exception { config1 = new ServiceConfiguration(); config2 = new ServiceConfiguration(); } + + protected void waitReplicatorStarted(String topicName) { + Awaitility.await().untilAsserted(() -> { + Optional topicOptional2 = pulsar2.getBrokerService().getTopic(topicName, false).get(); + assertTrue(topicOptional2.isPresent()); + PersistentTopic persistentTopic2 = (PersistentTopic) topicOptional2.get(); + assertFalse(persistentTopic2.getProducers().isEmpty()); + }); + } + + protected PulsarClient initClient(ClientBuilder clientBuilder) throws Exception { + return clientBuilder.build(); + } } 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 new file mode 100644 index 0000000000000..3caf4a1f2398c --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicationTxnTest.java @@ -0,0 +1,262 @@ +/* + * 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.common.naming.NamespaceName.SYSTEM_NAMESPACE; +import static org.apache.pulsar.transaction.coordinator.impl.MLTransactionLogImpl.TRANSACTION_LOG_PREFIX; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; +import com.google.common.collect.Sets; +import java.util.Collections; +import java.util.Optional; +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.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.persistent.GeoPersistentReplicator; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.transaction.pendingack.impl.MLPendingAckStore; +import org.apache.pulsar.client.api.ClientBuilder; +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.transaction.Transaction; +import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.apache.pulsar.common.naming.NamespaceName; +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.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; +import org.apache.pulsar.zookeeper.ZookeeperServerTest; +import org.awaitility.reflect.WhiteboxImpl; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class ReplicationTxnTest extends OneWayReplicatorTestBase { + + private boolean transactionBufferSegmentedSnapshotEnabled = false; + private int txnLogPartitions = 4; + + @Override + @BeforeClass(alwaysRun = true, timeOut = 300000) + public void setup() throws Exception { + super.setup(); + } + + @Override + @AfterClass(alwaysRun = true, timeOut = 300000) + public void cleanup() throws Exception { + super.cleanup(); + } + + @Override + protected PulsarClient initClient(ClientBuilder clientBuilder) throws Exception { + return clientBuilder.enableTransaction(true).build(); + } + + @Override + protected void setConfigDefaults(ServiceConfiguration config, String clusterName, + LocalBookkeeperEnsemble bookkeeperEnsemble, ZookeeperServerTest brokerConfigZk) { + super.setConfigDefaults(config, clusterName, bookkeeperEnsemble, brokerConfigZk); + config.setSystemTopicEnabled(true); + config.setTopicLevelPoliciesEnabled(true); + config.setTransactionCoordinatorEnabled(true); + config.setTransactionLogBatchedWriteEnabled(true); + config.setTransactionPendingAckBatchedWriteEnabled(true); + config.setTransactionBufferSegmentedSnapshotEnabled(transactionBufferSegmentedSnapshotEnabled); + } + + @Override + protected void createDefaultTenantsAndClustersAndNamespace() throws Exception { + super.createDefaultTenantsAndClustersAndNamespace(); + + // Create resource that transaction function relies on. + admin1.tenants().createTenant(SYSTEM_NAMESPACE.getTenant(), new TenantInfoImpl(Collections.emptySet(), + Sets.newHashSet(cluster1, cluster2))); + admin1.namespaces().createNamespace(SYSTEM_NAMESPACE.toString(), 4); + pulsar1.getPulsarResources().getNamespaceResources().getPartitionedTopicResources().createPartitionedTopic( + SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, new PartitionedTopicMetadata(txnLogPartitions)); + //admin1.topics().createPartitionedTopic(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN.toString(), 4); + + admin2.tenants().createTenant(SYSTEM_NAMESPACE.getTenant(), new TenantInfoImpl(Collections.emptySet(), + Sets.newHashSet(cluster1, cluster2))); + admin2.namespaces().createNamespace(SYSTEM_NAMESPACE.toString(), 4); + pulsar2.getPulsarResources().getNamespaceResources().getPartitionedTopicResources().createPartitionedTopic( + SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, new PartitionedTopicMetadata(txnLogPartitions)); + } + + private void pubAndSubOneMsg(String topic, String subscription) throws Exception { + Consumer consumer1 = client1.newConsumer(Schema.STRING).topic(topic).subscriptionName(subscription) + .isAckReceiptEnabled(true).subscribe(); + Producer producer1 = client1.newProducer(Schema.STRING).topic(topic).create(); + producer1.newMessage().value("msg1").send(); + // start txn. + Transaction txn = client1.newTransaction().withTransactionTimeout(1, TimeUnit.MINUTES).build().get(); + // consume. + Message c1Msg1 = consumer1.receive(5, TimeUnit.SECONDS); + assertNotNull(c1Msg1); + assertEquals(c1Msg1.getValue(), "msg1"); + consumer1.acknowledgeAsync(c1Msg1.getMessageId(), txn).join(); + // send. + producer1.newMessage(txn).value("msg2").send(); + // commit. + txn.commit().get(); + + // Consume the msg with TXN. + Message c1Msg2 = consumer1.receive(5, TimeUnit.SECONDS); + assertNotNull(c1Msg2); + assertEquals(c1Msg2.getValue(), "msg2"); + consumer1.acknowledgeAsync(c1Msg2.getMessageId()).join(); + + // Consume messages on the remote cluster. + Consumer consumer2 = client2.newConsumer(Schema.STRING).topic(topic).subscriptionName(subscription).subscribe(); + Message c2Msg1 = consumer2.receive(15, TimeUnit.SECONDS); + assertNotNull(c2Msg1); + MessageMetadata msgMetadata1 = WhiteboxImpl.getInternalState(c2Msg1, "msgMetadata"); + // Verify: the messages replicated has no TXN id. + assertFalse(msgMetadata1.hasTxnidMostBits()); + assertFalse(msgMetadata1.hasTxnidLeastBits()); + consumer2.acknowledge(c2Msg1); + Message c2Msg2 = consumer2.receive(15, TimeUnit.SECONDS); + assertNotNull(c2Msg2); + MessageMetadata msgMetadata2 = WhiteboxImpl.getInternalState(c2Msg2, "msgMetadata"); + // Verify: the messages replicated has no TXN id. + assertFalse(msgMetadata2.hasTxnidMostBits()); + assertFalse(msgMetadata2.hasTxnidLeastBits()); + consumer2.acknowledge(c2Msg2); + + // cleanup. + producer1.close(); + consumer1.close(); + consumer2.close(); + } + + private void verifyNoReplicator(BrokerService broker, TopicName topicName) throws Exception { + String tpStr = topicName.toString(); + CompletableFuture> future = broker.getTopic(tpStr, true); + if (future == null) { + return; + } + PersistentTopic persistentTopic = (PersistentTopic) future.join().get(); + assertTrue(persistentTopic.getReplicators().isEmpty()); + } + + @Test + public void testTxnLogNotBeReplicated() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp"); + final String subscription = "s1"; + admin1.topics().createNonPartitionedTopic(topic); + waitReplicatorStarted(topic); + admin1.topics().createSubscription(topic, subscription, MessageId.earliest); + admin2.topics().createSubscription(topic, subscription, MessageId.earliest); + // Pub & Sub. + pubAndSubOneMsg(topic, subscription); + // To cover more cases, sleep 3s. + Thread.sleep(3000); + + // Verify: messages on the TXN system topic did not been replicated. + // __transaction_log_: it only uses ML, will not create topic. + for (int i = 0; i < txnLogPartitions; i++) { + TopicName txnLog = TopicName.get(TopicDomain.persistent.value(), + NamespaceName.SYSTEM_NAMESPACE, TRANSACTION_LOG_PREFIX + i); + assertNotNull(pulsar1.getManagedLedgerFactory() + .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() + .getManagedLedgerInfo(pendingAck.getPersistenceNamingEncoding())); + assertFalse(broker1.getTopics().containsKey(pendingAck.toString())); + // __transaction_buffer_snapshot. + verifyNoReplicator(broker1, TopicName.get(TopicDomain.persistent.value(), + TopicName.get(topic).getNamespaceObject(), + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT)); + verifyNoReplicator(broker1, TopicName.get(TopicDomain.persistent.value(), + TopicName.get(topic).getNamespaceObject(), + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_SEGMENTS)); + verifyNoReplicator(broker1, TopicName.get(TopicDomain.persistent.value(), + TopicName.get(topic).getNamespaceObject(), + SystemTopicNames.TRANSACTION_BUFFER_SNAPSHOT_INDEXES)); + + // cleanup. + cleanupTopics(() -> { + admin1.topics().delete(topic); + admin2.topics().delete(topic); + try { + admin1.topics().delete(pendingAck.toString()); + } catch (Exception ex) {} + try { + admin2.topics().delete(pendingAck.toString()); + } catch (Exception ex) {} + }); + } + + @Test + public void testOngoingMessagesWillNotBeReplicated() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp"); + final String subscription = "s1"; + admin1.topics().createNonPartitionedTopic(topic); + waitReplicatorStarted(topic); + admin1.topics().createSubscription(topic, subscription, MessageId.earliest); + admin2.topics().createSubscription(topic, subscription, MessageId.earliest); + // Pub without commit. + Producer producer1 = client1.newProducer(Schema.STRING).topic(topic).create(); + Transaction txn = client1.newTransaction().withTransactionTimeout(1, TimeUnit.HOURS).build().get(); + producer1.newMessage(txn).value("msg1").send(); + // Verify: receive nothing on the remote cluster. + Consumer consumer2 = client2.newConsumer(Schema.STRING).topic(topic).subscriptionName(subscription).subscribe(); + Message msg = consumer2.receive(15, TimeUnit.SECONDS); + assertNull(msg); + // Verify: the repl cursor is not end of the topic. + PersistentTopic persistentTopic = (PersistentTopic) broker1.getTopic(topic, false).join().get(); + GeoPersistentReplicator replicator = + (GeoPersistentReplicator) persistentTopic.getReplicators().values().iterator().next(); + assertTrue(replicator.getCursor().hasMoreEntries()); + + // cleanup. + producer1.close(); + consumer2.close(); + cleanupTopics(() -> { + admin1.topics().delete(topic); + admin2.topics().delete(topic); + TopicName pendingAck = TopicName.get( + MLPendingAckStore.getTransactionPendingAckStoreSuffix(topic, subscription)); + try { + admin1.topics().delete(pendingAck.toString()); + } catch (Exception ex) {} + try { + admin2.topics().delete(pendingAck.toString()); + } catch (Exception ex) {} + }); + } +} From 323013aaa9db90d889acb166ea2ceaecd809399a Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Sun, 19 May 2024 11:28:21 +0800 Subject: [PATCH 516/699] [fix] [broker] fix deadlock when disable topic level Geo-Replication (#22738) (cherry picked from commit 6372b9c8f5448757cdb56dc70aaf0ebb52a1063e) (cherry picked from commit 1a8ba1356a18e6794a411239e411202c77965c4d) --- .../admin/impl/PersistentTopicsBase.java | 5 +- .../broker/service/OneWayReplicatorTest.java | 59 ++++++ .../service/OneWayReplicatorTestBase.java | 172 +++++++++++++++--- .../OneWayReplicatorUsingGlobalZKTest.java | 97 ++++++++++ 4 files changed, 305 insertions(+), 28 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java 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 8c5851c9d82fb..d64b6c6e37de0 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 @@ -3416,11 +3416,12 @@ protected CompletableFuture internalSetReplicationClusters(List cl Set replicationClusters = Sets.newHashSet(clusterIds); return validateTopicPolicyOperationAsync(topicName, PolicyName.REPLICATION, PolicyOperation.WRITE) .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) - .thenCompose(__ -> { + .thenAccept(__ -> { if (CollectionUtils.isEmpty(clusterIds)) { throw new RestException(Status.PRECONDITION_FAILED, "ClusterIds should not be null or empty"); } - Set clusters = clusters(); + }).thenCompose(__ -> clustersAsync()) + .thenCompose(clusters -> { List> futures = new ArrayList<>(replicationClusters.size()); for (String clusterId : replicationClusters) { if (!clusters.contains(clusterId)) { 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 2eee20906d259..cc2666af3c13f 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 @@ -658,4 +658,63 @@ public void testUnFenceTopicToReuse() throws Exception { admin2.topics().delete(topicName); }); } + + @Test + public void testDeleteNonPartitionedTopic() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); + admin1.topics().createNonPartitionedTopic(topicName); + + // Verify replicator works. + verifyReplicationWorks(topicName); + + // Disable replication. + setTopicLevelClusters(topicName, Arrays.asList(cluster1), admin1, pulsar1); + setTopicLevelClusters(topicName, Arrays.asList(cluster2), admin2, pulsar2); + + // Delete topic. + admin1.topics().delete(topicName); + admin2.topics().delete(topicName); + + // Verify the topic was deleted. + assertFalse(pulsar1.getPulsarResources().getTopicResources() + .persistentTopicExists(TopicName.get(topicName)).join()); + assertFalse(pulsar2.getPulsarResources().getTopicResources() + .persistentTopicExists(TopicName.get(topicName)).join()); + } + + @Test + public void testDeletePartitionedTopic() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); + admin1.topics().createPartitionedTopic(topicName, 2); + + // Verify replicator works. + verifyReplicationWorks(topicName); + + // Disable replication. + setTopicLevelClusters(topicName, Arrays.asList(cluster1), admin1, pulsar1); + setTopicLevelClusters(topicName, Arrays.asList(cluster2), admin2, pulsar2); + + // Delete topic. + admin1.topics().deletePartitionedTopic(topicName); + if (!usingGlobalZK) { + admin2.topics().deletePartitionedTopic(topicName); + } + + // Verify the topic was deleted. + assertFalse(pulsar1.getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .partitionedTopicExists(TopicName.get(topicName))); + assertFalse(pulsar2.getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .partitionedTopicExists(TopicName.get(topicName))); + if (!usingGlobalZK) { + // So far, the topic partitions on the remote cluster are needed to delete manually when using global ZK. + assertFalse(pulsar1.getPulsarResources().getTopicResources() + .persistentTopicExists(TopicName.get(topicName).getPartition(0)).join()); + assertFalse(pulsar2.getPulsarResources().getTopicResources() + .persistentTopicExists(TopicName.get(topicName).getPartition(0)).join()); + assertFalse(pulsar1.getPulsarResources().getTopicResources() + .persistentTopicExists(TopicName.get(topicName).getPartition(1)).join()); + assertFalse(pulsar2.getPulsarResources().getTopicResources() + .persistentTopicExists(TopicName.get(topicName).getPartition(1)).join()); + } + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java index 317e43306e356..6a84432890cb5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java @@ -21,21 +21,33 @@ import static org.apache.pulsar.compaction.Compactor.COMPACTION_SUBSCRIPTION; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.assertEquals; import com.google.common.collect.Sets; import java.net.URL; import java.time.Duration; +import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; +import java.util.List; import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.ClientBuilder; +import org.apache.pulsar.client.api.Consumer; +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.common.naming.SystemTopicNames; +import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.TopicPolicies; import org.apache.pulsar.common.policies.data.TopicType; import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.apache.pulsar.tests.TestRetrySupport; @@ -52,6 +64,9 @@ public abstract class OneWayReplicatorTestBase extends TestRetrySupport { protected final String nonReplicatedNamespace = defaultTenant + "/ns1"; protected final String cluster1 = "r1"; + + protected boolean usingGlobalZK = false; + protected URL url1; protected URL urlTls1; protected ServiceConfiguration config1 = new ServiceConfiguration(); @@ -77,8 +92,12 @@ protected void startZKAndBK() throws Exception { // Start ZK. brokerConfigZk1 = new ZookeeperServerTest(0); brokerConfigZk1.start(); - brokerConfigZk2 = new ZookeeperServerTest(0); - brokerConfigZk2.start(); + if (usingGlobalZK) { + brokerConfigZk2 = brokerConfigZk1; + } else { + brokerConfigZk2 = new ZookeeperServerTest(0); + brokerConfigZk2.start(); + } // Start BK. bkEnsemble1 = new LocalBookkeeperEnsemble(3, 0, () -> 0); @@ -132,30 +151,32 @@ protected void createDefaultTenantsAndClustersAndNamespace() throws Exception { .brokerServiceUrlTls(pulsar2.getBrokerServiceUrlTls()) .brokerClientTlsEnabled(false) .build()); - admin2.clusters().createCluster(cluster1, ClusterData.builder() - .serviceUrl(url1.toString()) - .serviceUrlTls(urlTls1.toString()) - .brokerServiceUrl(pulsar1.getBrokerServiceUrl()) - .brokerServiceUrlTls(pulsar1.getBrokerServiceUrlTls()) - .brokerClientTlsEnabled(false) - .build()); - admin2.clusters().createCluster(cluster2, ClusterData.builder() - .serviceUrl(url2.toString()) - .serviceUrlTls(urlTls2.toString()) - .brokerServiceUrl(pulsar2.getBrokerServiceUrl()) - .brokerServiceUrlTls(pulsar2.getBrokerServiceUrlTls()) - .brokerClientTlsEnabled(false) - .build()); - admin1.tenants().createTenant(defaultTenant, new TenantInfoImpl(Collections.emptySet(), Sets.newHashSet(cluster1, cluster2))); - admin2.tenants().createTenant(defaultTenant, new TenantInfoImpl(Collections.emptySet(), - Sets.newHashSet(cluster1, cluster2))); - admin1.namespaces().createNamespace(replicatedNamespace, Sets.newHashSet(cluster1, cluster2)); - admin2.namespaces().createNamespace(replicatedNamespace); admin1.namespaces().createNamespace(nonReplicatedNamespace); - admin2.namespaces().createNamespace(nonReplicatedNamespace); + + if (!usingGlobalZK) { + admin2.clusters().createCluster(cluster1, ClusterData.builder() + .serviceUrl(url1.toString()) + .serviceUrlTls(urlTls1.toString()) + .brokerServiceUrl(pulsar1.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar1.getBrokerServiceUrlTls()) + .brokerClientTlsEnabled(false) + .build()); + admin2.clusters().createCluster(cluster2, ClusterData.builder() + .serviceUrl(url2.toString()) + .serviceUrlTls(urlTls2.toString()) + .brokerServiceUrl(pulsar2.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar2.getBrokerServiceUrlTls()) + .brokerClientTlsEnabled(false) + .build()); + admin2.tenants().createTenant(defaultTenant, new TenantInfoImpl(Collections.emptySet(), + Sets.newHashSet(cluster1, cluster2))); + admin2.namespaces().createNamespace(replicatedNamespace); + admin2.namespaces().createNamespace(nonReplicatedNamespace); + } + } protected void cleanupTopics(CleanupTopicAction cleanupTopicAction) throws Exception { @@ -163,6 +184,9 @@ protected void cleanupTopics(CleanupTopicAction cleanupTopicAction) throws Excep } protected void cleanupTopics(String namespace, CleanupTopicAction cleanupTopicAction) throws Exception { + if (usingGlobalZK) { + throw new IllegalArgumentException("The method cleanupTopics does not support for global ZK"); + } waitChangeEventsInit(namespace); admin1.namespaces().setNamespaceReplicationClusters(namespace, Collections.singleton(cluster1)); admin1.namespaces().unload(namespace); @@ -242,11 +266,15 @@ protected void cleanup() throws Exception { // delete namespaces. waitChangeEventsInit(replicatedNamespace); admin1.namespaces().setNamespaceReplicationClusters(replicatedNamespace, Sets.newHashSet(cluster1)); + if (!usingGlobalZK) { + admin2.namespaces().setNamespaceReplicationClusters(replicatedNamespace, Sets.newHashSet(cluster2)); + } admin1.namespaces().deleteNamespace(replicatedNamespace); - admin2.namespaces().setNamespaceReplicationClusters(replicatedNamespace, Sets.newHashSet(cluster2)); - admin2.namespaces().deleteNamespace(replicatedNamespace); admin1.namespaces().deleteNamespace(nonReplicatedNamespace); - admin2.namespaces().deleteNamespace(nonReplicatedNamespace); + if (!usingGlobalZK) { + admin2.namespaces().deleteNamespace(replicatedNamespace); + admin2.namespaces().deleteNamespace(nonReplicatedNamespace); + } // shutdown. markCurrentSetupNumberCleaned(); @@ -291,7 +319,7 @@ protected void cleanup() throws Exception { brokerConfigZk1.stop(); brokerConfigZk1 = null; } - if (brokerConfigZk2 != null) { + if (!usingGlobalZK && brokerConfigZk2 != null) { brokerConfigZk2.stop(); brokerConfigZk2 = null; } @@ -313,4 +341,96 @@ protected void waitReplicatorStarted(String topicName) { protected PulsarClient initClient(ClientBuilder clientBuilder) throws Exception { return clientBuilder.build(); } + + protected void verifyReplicationWorks(String topic) throws Exception { + final String subscription = "__subscribe_1"; + final String msgValue = "__msg1"; + Producer producer1 = client1.newProducer(Schema.STRING).topic(topic).create(); + Consumer consumer2 = client2.newConsumer(Schema.STRING).topic(topic).isAckReceiptEnabled(true) + .subscriptionName(subscription).subscribe(); + producer1.newMessage().value(msgValue).send(); + pulsar1.getBrokerService().checkReplicationPolicies(); + assertEquals(consumer2.receive(10, TimeUnit.SECONDS).getValue(), msgValue); + consumer2.unsubscribe(); + producer1.close(); + } + + protected void setTopicLevelClusters(String topic, List clusters, PulsarAdmin admin, + PulsarService pulsar) throws Exception { + Set expected = new HashSet<>(clusters); + TopicName topicName = TopicName.get(TopicName.get(topic).getPartitionedTopicName()); + int partitions = ensurePartitionsAreSame(topic); + admin.topics().setReplicationClusters(topic, clusters); + Awaitility.await().untilAsserted(() -> { + TopicPolicies policies = pulsar.getTopicPoliciesService().getTopicPolicies(topicName); + assertEquals(new HashSet<>(policies.getReplicationClusters()), expected); + if (partitions == 0) { + checkNonPartitionedTopicLevelClusters(topicName.toString(), clusters, admin, pulsar.getBrokerService()); + } else { + for (int i = 0; i < partitions; i++) { + checkNonPartitionedTopicLevelClusters(topicName.getPartition(i).toString(), clusters, admin, + pulsar.getBrokerService()); + } + } + }); + } + + protected void checkNonPartitionedTopicLevelClusters(String topic, List clusters, PulsarAdmin admin, + BrokerService broker) throws Exception { + CompletableFuture> future = broker.getTopic(topic, false); + if (future == null) { + return; + } + Optional optional = future.join(); + if (optional == null || !optional.isPresent()) { + return; + } + PersistentTopic persistentTopic = (PersistentTopic) optional.get(); + Set expected = new HashSet<>(clusters); + Set act = new HashSet<>(persistentTopic.getTopicPolicies().get().getReplicationClusters()); + assertEquals(act, expected); + } + + protected int ensurePartitionsAreSame(String topic) throws Exception { + TopicName topicName = TopicName.get(TopicName.get(topic).getPartitionedTopicName()); + boolean isPartitionedTopic1 = pulsar1.getPulsarResources().getNamespaceResources() + .getPartitionedTopicResources().partitionedTopicExists(topicName); + boolean isPartitionedTopic2 = pulsar2.getPulsarResources().getNamespaceResources() + .getPartitionedTopicResources().partitionedTopicExists(topicName); + if (isPartitionedTopic1 != isPartitionedTopic2) { + throw new IllegalArgumentException(String.format("Can not delete topic." + + " isPartitionedTopic1: %s, isPartitionedTopic2: %s", + isPartitionedTopic1, isPartitionedTopic2)); + } + if (!isPartitionedTopic1) { + return 0; + } + int partitions1 = pulsar1.getPulsarResources().getNamespaceResources() + .getPartitionedTopicResources().getPartitionedTopicMetadataAsync(topicName).join().get().partitions; + int partitions2 = pulsar2.getPulsarResources().getNamespaceResources() + .getPartitionedTopicResources().getPartitionedTopicMetadataAsync(topicName).join().get().partitions; + if (partitions1 != partitions2) { + throw new IllegalArgumentException(String.format("Can not delete topic." + + " partitions1: %s, partitions2: %s", + partitions1, partitions2)); + } + return partitions1; + } + + protected void deleteTopicAfterDisableTopicLevelReplication(String topic) throws Exception { + setTopicLevelClusters(topic, Arrays.asList(cluster1), admin1, pulsar1); + setTopicLevelClusters(topic, Arrays.asList(cluster1), admin2, pulsar2); + admin2.topics().setReplicationClusters(topic, Arrays.asList(cluster2)); + + int partitions = ensurePartitionsAreSame(topic); + + TopicName topicName = TopicName.get(TopicName.get(topic).getPartitionedTopicName()); + if (partitions != 0) { + admin1.topics().deletePartitionedTopic(topicName.toString()); + admin2.topics().deletePartitionedTopic(topicName.toString()); + } else { + admin1.topics().delete(topicName.toString()); + admin2.topics().delete(topicName.toString()); + } + } } 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 new file mode 100644 index 0000000000000..d827235bc326d --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorUsingGlobalZKTest.java @@ -0,0 +1,97 @@ +/* + * 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 lombok.extern.slf4j.Slf4j; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class OneWayReplicatorUsingGlobalZKTest extends OneWayReplicatorTest { + + @Override + @BeforeClass(alwaysRun = true, timeOut = 300000) + public void setup() throws Exception { + super.usingGlobalZK = true; + super.setup(); + } + + @Override + @AfterClass(alwaysRun = true, timeOut = 300000) + public void cleanup() throws Exception { + super.cleanup(); + } + + @Test(enabled = false) + public void testReplicatorProducerStatInTopic() throws Exception { + super.testReplicatorProducerStatInTopic(); + } + + @Test(enabled = false) + public void testCreateRemoteConsumerFirst() throws Exception { + super.testReplicatorProducerStatInTopic(); + } + + @Test(enabled = false) + public void testTopicCloseWhenInternalProducerCloseErrorOnce() throws Exception { + super.testReplicatorProducerStatInTopic(); + } + + @Test(enabled = false) + public void testConcurrencyOfUnloadBundleAndRecreateProducer() throws Exception { + super.testConcurrencyOfUnloadBundleAndRecreateProducer(); + } + + @Test(enabled = false) + public void testPartitionedTopicLevelReplication() throws Exception { + super.testPartitionedTopicLevelReplication(); + } + + @Test(enabled = false) + public void testPartitionedTopicLevelReplicationRemoteTopicExist() throws Exception { + super.testPartitionedTopicLevelReplicationRemoteTopicExist(); + } + + @Test(enabled = false) + public void testPartitionedTopicLevelReplicationRemoteConflictTopicExist() throws Exception { + super.testPartitionedTopicLevelReplicationRemoteConflictTopicExist(); + } + + @Test(enabled = false) + public void testConcurrencyOfUnloadBundleAndRecreateProducer2() throws Exception { + super.testConcurrencyOfUnloadBundleAndRecreateProducer2(); + } + + @Test(enabled = false) + public void testUnFenceTopicToReuse() throws Exception { + super.testUnFenceTopicToReuse(); + } + + @Test + public void testDeleteNonPartitionedTopic() throws Exception { + super.testDeleteNonPartitionedTopic(); + } + + @Test + public void testDeletePartitionedTopic() throws Exception { + super.testDeletePartitionedTopic(); + } +} From 8027d5c786faf5551212ef09355d5edb76bbf899 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Nicol=C3=B2=20Boschi?= Date: Mon, 20 May 2024 18:12:21 +0200 Subject: [PATCH 517/699] [fix][ml]: subscription props could be lost in case of missing ledger during recovery (#22637) (cherry picked from commit 5bbb62d9fc8845de8bbb61323453b8aad22c083d) --- .../org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 9fb570eee491e..3451167affbb6 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 @@ -540,7 +540,7 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac log.error("[{}] Error opening metadata ledger {} for cursor {}: {}", ledger.getName(), ledgerId, name, BKException.getMessage(rc)); // Rewind to oldest entry available - initialize(getRollbackPosition(info), Collections.emptyMap(), Collections.emptyMap(), callback); + initialize(getRollbackPosition(info), Collections.emptyMap(), cursorProperties, callback); return; } else if (rc != BKException.Code.OK) { log.warn("[{}] Error opening metadata ledger {} for cursor {}: {}", ledger.getName(), ledgerId, name, From 1baa094ca42e2f1adb51783b39b01ca8ce1b0507 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 21 May 2024 15:50:53 +0800 Subject: [PATCH 518/699] [fix] [ml] Add entry fail due to race condition about add entry failed/timeout and switch ledger (#22221) (cherry picked from commit b798e7f7d81e5b03f127a07b435c912109ff2e64) --- .../mledger/impl/ManagedLedgerImpl.java | 30 +++++++---- .../bookkeeper/mledger/impl/OpAddEntry.java | 39 ++++++++------ .../mledger/impl/ShadowManagedLedgerImpl.java | 3 ++ .../mledger/impl/ManagedLedgerTest.java | 53 ++++++++++++++++++- .../client/PulsarMockBookKeeper.java | 7 +++ .../client/PulsarMockLedgerHandle.java | 7 +++ 6 files changed, 113 insertions(+), 26 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 8eb83724b7f36..4cc883f140a46 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 @@ -59,6 +59,7 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLongFieldUpdater; import java.util.concurrent.atomic.AtomicReference; @@ -242,6 +243,9 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback { protected volatile long lastAddEntryTimeMs = 0; private long inactiveLedgerRollOverTimeMs = 0; + /** A signal that may trigger all the subsequent OpAddEntry of current ledger to be failed due to timeout. **/ + protected volatile AtomicBoolean currentLedgerTimeoutTriggered; + protected static final int DEFAULT_LEDGER_DELETE_RETRIES = 3; protected static final int DEFAULT_LEDGER_DELETE_BACKOFF_TIME_SEC = 60; private static final String MIGRATION_STATE_PROPERTY = "migrated"; @@ -534,6 +538,7 @@ public void operationFailed(MetaStoreException e) { STATE_UPDATER.set(this, State.LedgerOpened); updateLastLedgerCreatedTimeAndScheduleRolloverTask(); currentLedger = lh; + currentLedgerTimeoutTriggered = new AtomicBoolean(); lastConfirmedEntry = new PositionImpl(lh.getId(), -1); // bypass empty ledgers, find last ledger with Message if possible. @@ -776,7 +781,8 @@ public void asyncAddEntry(ByteBuf buffer, AddEntryCallback callback, Object ctx) // Jump to specific thread to avoid contention from writers writing from different threads executor.execute(() -> { - OpAddEntry addOperation = OpAddEntry.createNoRetainBuffer(this, buffer, callback, ctx); + OpAddEntry addOperation = OpAddEntry.createNoRetainBuffer(this, buffer, callback, ctx, + currentLedgerTimeoutTriggered); internalAsyncAddEntry(addOperation); }); } @@ -792,7 +798,8 @@ public void asyncAddEntry(ByteBuf buffer, int numberOfMessages, AddEntryCallback // Jump to specific thread to avoid contention from writers writing from different threads executor.execute(() -> { - OpAddEntry addOperation = OpAddEntry.createNoRetainBuffer(this, buffer, numberOfMessages, callback, ctx); + OpAddEntry addOperation = OpAddEntry.createNoRetainBuffer(this, buffer, numberOfMessages, callback, ctx, + currentLedgerTimeoutTriggered); internalAsyncAddEntry(addOperation); }); } @@ -844,6 +851,7 @@ protected synchronized void internalAsyncAddEntry(OpAddEntry addOperation) { // Write into lastLedger addOperation.setLedger(currentLedger); + addOperation.setTimeoutTriggered(currentLedgerTimeoutTriggered); ++currentLedgerEntries; currentLedgerSize += addOperation.data.readableBytes(); @@ -1587,6 +1595,7 @@ public void operationComplete(Void v, Stat stat) { LedgerHandle originalCurrentLedger = currentLedger; ledgers.put(lh.getId(), newLedger); currentLedger = lh; + currentLedgerTimeoutTriggered = new AtomicBoolean(); currentLedgerEntries = 0; currentLedgerSize = 0; updateLedgersIdsComplete(originalCurrentLedger); @@ -1670,9 +1679,11 @@ void createNewOpAddEntryForNewLedger() { if (existsOp != null) { // If op is used by another ledger handle, we need to close it and create a new one if (existsOp.ledger != null) { - existsOp.close(); - existsOp = OpAddEntry.createNoRetainBuffer(existsOp.ml, existsOp.data, - existsOp.getNumberOfMessages(), existsOp.callback, existsOp.ctx); + existsOp = existsOp.duplicateAndClose(currentLedgerTimeoutTriggered); + } else { + // This scenario should not happen. + log.warn("[{}] An OpAddEntry's ledger is empty.", name); + existsOp.setTimeoutTriggered(currentLedgerTimeoutTriggered); } existsOp.setLedger(currentLedger); pendingAddEntries.add(existsOp); @@ -4211,13 +4222,14 @@ private void checkAddTimeout() { } OpAddEntry opAddEntry = pendingAddEntries.peek(); if (opAddEntry != null) { - final long finalAddOpCount = opAddEntry.addOpCount; boolean isTimedOut = opAddEntry.lastInitTime != -1 && TimeUnit.NANOSECONDS.toSeconds(System.nanoTime() - opAddEntry.lastInitTime) >= timeoutSec; if (isTimedOut) { - log.error("Failed to add entry for ledger {} in time-out {} sec", - (opAddEntry.ledger != null ? opAddEntry.ledger.getId() : -1), timeoutSec); - opAddEntry.handleAddTimeoutFailure(opAddEntry.ledger, finalAddOpCount); + log.warn("[{}] Failed to add entry {}:{} in time-out {} sec", this.name, + opAddEntry.ledger != null ? opAddEntry.ledger.getId() : -1, + opAddEntry.entryId, timeoutSec); + currentLedgerTimeoutTriggered.set(true); + opAddEntry.handleAddFailure(opAddEntry.ledger); } } } 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 ae2beafb64374..acbb0da5a4e74 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 @@ -24,8 +24,10 @@ import io.netty.util.Recycler.Handle; import io.netty.util.ReferenceCountUtil; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLongFieldUpdater; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import lombok.Setter; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.AsyncCallback.AddCallback; import org.apache.bookkeeper.client.AsyncCallback.CloseCallback; @@ -45,7 +47,7 @@ public class OpAddEntry implements AddCallback, CloseCallback, Runnable { protected ManagedLedgerImpl ml; LedgerHandle ledger; - private long entryId; + long entryId; private int numberOfMessages; @SuppressWarnings("unused") @@ -68,6 +70,9 @@ public class OpAddEntry implements AddCallback, CloseCallback, Runnable { AtomicReferenceFieldUpdater.newUpdater(OpAddEntry.class, OpAddEntry.State.class, "state"); volatile State state; + @Setter + private AtomicBoolean timeoutTriggered; + enum State { OPEN, INITIATED, @@ -76,8 +81,8 @@ enum State { } public static OpAddEntry createNoRetainBuffer(ManagedLedgerImpl ml, ByteBuf data, AddEntryCallback callback, - Object ctx) { - OpAddEntry op = createOpAddEntryNoRetainBuffer(ml, data, callback, ctx); + Object ctx, AtomicBoolean timeoutTriggered) { + OpAddEntry op = createOpAddEntryNoRetainBuffer(ml, data, callback, ctx, timeoutTriggered); if (log.isDebugEnabled()) { log.debug("Created new OpAddEntry {}", op); } @@ -85,8 +90,9 @@ public static OpAddEntry createNoRetainBuffer(ManagedLedgerImpl ml, ByteBuf data } public static OpAddEntry createNoRetainBuffer(ManagedLedgerImpl ml, ByteBuf data, int numberOfMessages, - AddEntryCallback callback, Object ctx) { - OpAddEntry op = createOpAddEntryNoRetainBuffer(ml, data, callback, ctx); + AddEntryCallback callback, Object ctx, + AtomicBoolean timeoutTriggered) { + OpAddEntry op = createOpAddEntryNoRetainBuffer(ml, data, callback, ctx, timeoutTriggered); op.numberOfMessages = numberOfMessages; if (log.isDebugEnabled()) { log.debug("Created new OpAddEntry {}", op); @@ -95,7 +101,8 @@ public static OpAddEntry createNoRetainBuffer(ManagedLedgerImpl ml, ByteBuf data } private static OpAddEntry createOpAddEntryNoRetainBuffer(ManagedLedgerImpl ml, ByteBuf data, - AddEntryCallback callback, Object ctx) { + AddEntryCallback callback, Object ctx, + AtomicBoolean timeoutTriggered) { OpAddEntry op = RECYCLER.get(); op.ml = ml; op.ledger = null; @@ -109,6 +116,7 @@ private static OpAddEntry createOpAddEntryNoRetainBuffer(ManagedLedgerImpl ml, B op.startTime = System.nanoTime(); op.state = State.OPEN; op.payloadProcessorHandle = null; + op.timeoutTriggered = timeoutTriggered; ml.mbean.addAddEntrySample(op.dataLength); return op; } @@ -176,7 +184,9 @@ public void addComplete(int rc, final LedgerHandle lh, long entryId, Object ctx) if (!STATE_UPDATER.compareAndSet(OpAddEntry.this, State.INITIATED, State.COMPLETED)) { log.warn("[{}] The add op is terminal legacy callback for entry {}-{} adding.", ml.getName(), lh.getId(), entryId); - OpAddEntry.this.recycle(); + // Since there is a thread is coping this object, do not recycle this object to avoid other problems. + // For example: we recycled this object, other thread get a null "opAddEntry.{variable_name}". + // Recycling is not mandatory, JVM GC will collect it. return; } @@ -200,7 +210,7 @@ public void addComplete(int rc, final LedgerHandle lh, long entryId, Object ctx) lh == null ? -1 : lh.getId(), entryId, dataLength, rc); } - if (rc != BKException.Code.OK) { + if (rc != BKException.Code.OK || timeoutTriggered.get()) { handleAddFailure(lh); } else { // Trigger addComplete callback in a thread hashed on the managed ledger name @@ -307,13 +317,6 @@ private boolean checkAndCompleteOp(Object ctx) { return false; } - void handleAddTimeoutFailure(final LedgerHandle ledger, Object ctx) { - if (checkAndCompleteOp(ctx)) { - this.close(); - this.handleAddFailure(ledger); - } - } - /** * It handles add failure on the given ledger. it can be triggered when add-entry fails or times out. * @@ -333,8 +336,11 @@ void handleAddFailure(final LedgerHandle lh) { }); } - void close() { + OpAddEntry duplicateAndClose(AtomicBoolean timeoutTriggered) { STATE_UPDATER.set(OpAddEntry.this, State.CLOSED); + OpAddEntry duplicate = + OpAddEntry.createNoRetainBuffer(ml, data, getNumberOfMessages(), callback, ctx, timeoutTriggered); + return duplicate; } public State getState() { @@ -389,6 +395,7 @@ public void recycle() { startTime = -1; lastInitTime = -1; payloadProcessorHandle = null; + timeoutTriggered = null; recyclerHandle.recycle(this); } 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 8b2742d958783..ec5b006c4745b 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 @@ -25,6 +25,7 @@ import java.util.Map; import java.util.TreeMap; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Supplier; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.AsyncCallback; @@ -54,6 +55,8 @@ public ShadowManagedLedgerImpl(ManagedLedgerFactoryImpl factory, BookKeeper book String name, final Supplier> mlOwnershipChecker) { super(factory, bookKeeper, store, config, scheduledExecutor, name, mlOwnershipChecker); this.sourceMLName = config.getShadowSourceName(); + // ShadowManagedLedgerImpl does not implement add entry timeout yet, so this variable will always be false. + this.currentLedgerTimeoutTriggered = new AtomicBoolean(false); } /** 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 7a34947a6b7cc..eca9c28a5ef6b 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 @@ -146,6 +146,7 @@ import org.apache.pulsar.metadata.impl.FaultInjectionMetadataStore; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; +import org.eclipse.jetty.util.BlockingArrayQueue; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.DataProvider; @@ -3184,6 +3185,55 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { ledger.close(); } + @Test + public void testAddEntryResponseTimeout() throws Exception { + // Create ML with feature Add Entry Timeout Check. + final ManagedLedgerConfig config = new ManagedLedgerConfig().setAddEntryTimeoutSeconds(2); + final ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("ml1", config); + final ManagedCursor cursor = ledger.openCursor("c1"); + final CollectCtxAddEntryCallback collectCtxAddEntryCallback = new CollectCtxAddEntryCallback(); + + // Insert a response delay. + bkc.addEntryResponseDelay(8, TimeUnit.SECONDS); + + // Add two entries. + final byte[] msg1 = new byte[]{1}; + final byte[] msg2 = new byte[]{2}; + int ctx1 = 1; + int ctx2 = 2; + ledger.asyncAddEntry(msg1, collectCtxAddEntryCallback, ctx1); + ledger.asyncAddEntry(msg2, collectCtxAddEntryCallback, ctx2); + // Verify all write requests are completed. + Awaitility.await().untilAsserted(() -> { + assertEquals(collectCtxAddEntryCallback.addCompleteCtxList, Arrays.asList(1, 2)); + }); + Entry entry1 = cursor.readEntries(1).get(0); + assertEquals(entry1.getData(), msg1); + entry1.release(); + Entry entry2 = cursor.readEntries(1).get(0); + assertEquals(entry2.getData(), msg2); + entry2.release(); + + // cleanup. + factory.delete(ledger.name); + } + + private static class CollectCtxAddEntryCallback implements AddEntryCallback { + + public List addCompleteCtxList = new BlockingArrayQueue<>(); + public List addFailedCtxList = new BlockingArrayQueue<>(); + + @Override + public void addComplete(Position position, ByteBuf entryData, Object ctx) { + addCompleteCtxList.add(ctx); + } + + @Override + public void addFailed(ManagedLedgerException exception, Object ctx) { + addFailedCtxList.add(ctx); + } + } + /** * It verifies that if bk-client doesn't complete the add-entry in given time out then broker is resilient enough * to create new ledger and add entry successfully. @@ -3259,7 +3309,8 @@ public void avoidUseSameOpAddEntryBetweenDifferentLedger() throws Exception { List oldOps = new ArrayList<>(); for (int i = 0; i < 10; i++) { - OpAddEntry op = OpAddEntry.createNoRetainBuffer(ledger, ByteBufAllocator.DEFAULT.buffer(128).retain(), null, null); + OpAddEntry op = OpAddEntry.createNoRetainBuffer(ledger, + ByteBufAllocator.DEFAULT.buffer(128).retain(), null, null, new AtomicBoolean()); if (i > 4) { op.setLedger(mock(LedgerHandle.class)); } 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 f0d279ef25050..4516cfea01f05 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockBookKeeper.java @@ -18,6 +18,7 @@ */ package org.apache.bookkeeper.client; +import static com.google.common.base.Preconditions.checkArgument; import com.google.common.collect.Lists; import java.util.ArrayList; import java.util.Arrays; @@ -89,6 +90,7 @@ public static Collection getMockEnsemble() { } final Queue addEntryDelaysMillis = new ConcurrentLinkedQueue<>(); + final Queue addEntryResponseDelaysMillis = new ConcurrentLinkedQueue<>(); final List> failures = new ArrayList<>(); final List> addEntryFailures = new ArrayList<>(); @@ -367,6 +369,11 @@ public synchronized void addEntryDelay(long delay, TimeUnit unit) { addEntryDelaysMillis.add(unit.toMillis(delay)); } + public synchronized void addEntryResponseDelay(long delay, TimeUnit unit) { + checkArgument(delay >= 0, "The delay time must not be negative."); + addEntryResponseDelaysMillis.add(unit.toMillis(delay)); + } + static int getExceptionCode(Throwable t) { if (t instanceof BKException) { return ((BKException) t).getCode(); 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 dea33a0e67662..aa61e541d0d6b 100644 --- a/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java +++ b/testmocks/src/main/java/org/apache/bookkeeper/client/PulsarMockLedgerHandle.java @@ -197,6 +197,13 @@ public void asyncAddEntry(final ByteBuf data, final AddCallback cb, final Object cb.addComplete(PulsarMockBookKeeper.getExceptionCode(exception), PulsarMockLedgerHandle.this, LedgerHandle.INVALID_ENTRY_ID, ctx); } else { + Long responseDelayMillis = bk.addEntryResponseDelaysMillis.poll(); + if (responseDelayMillis != null) { + try { + Thread.sleep(responseDelayMillis); + } catch (InterruptedException e) { + } + } cb.addComplete(BKException.Code.OK, PulsarMockLedgerHandle.this, entryId, ctx); } }, bk.executor); From 3d57b6edc2a435ef207a2f5b6f6a6a5aee2d1065 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Sun, 26 May 2024 10:59:10 +0800 Subject: [PATCH 519/699] [fix][admin][part-1]Clearly define REST API on Open API (#22774) (cherry picked from commit 9d9b8dfa16d2f15142ea94467c086c016cb9e49d) (cherry picked from commit fdeeba597d1689f858a0eec072441872ad33c0ed) --- .../broker/admin/impl/BrokerStatsBase.java | 3 +++ .../pulsar/broker/admin/impl/BrokersBase.java | 19 +++++++++++-------- .../broker/admin/impl/ClustersBase.java | 10 +++++++--- .../pulsar/broker/admin/v2/Bookies.java | 16 +++++++++++++--- .../pulsar/broker/admin/v2/BrokerStats.java | 7 ++++++- 5 files changed, 40 insertions(+), 15 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java index 6d49dd81da13d..48577fc701486 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/BrokerStatsBase.java @@ -125,6 +125,9 @@ public AllocatorStats getAllocatorStats(@PathParam("allocator") String allocator @GET @Path("/bookieops") @ApiOperation(value = "Get pending bookie client op stats by namespace", + notes = "Returns a nested map structure which Swagger does not fully support for display. " + + "Structure: Map>." + + " Please refer to this structure for details.", response = PendingBookieOpsStats.class, // https://github.com/swagger-api/swagger-core/issues/449 // nested containers are not supported 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 61b354610ac20..eeb65590bec8a 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 @@ -219,7 +219,7 @@ public void updateDynamicConfiguration(@Suspended AsyncResponse asyncResponse, @ApiOperation(value = "Delete dynamic ServiceConfiguration into metadata only." + " This operation requires Pulsar super-user privileges.") - @ApiResponses(value = { @ApiResponse(code = 204, message = "Service configuration updated successfully"), + @ApiResponses(value = { @ApiResponse(code = 204, message = "Service configuration delete successfully"), @ApiResponse(code = 403, message = "You don't have admin permission to update service-configuration"), @ApiResponse(code = 412, message = "Invalid dynamic-config value"), @ApiResponse(code = 500, message = "Internal server error") }) @@ -240,7 +240,8 @@ public void deleteDynamicConfiguration( @GET @Path("/configuration/values") - @ApiOperation(value = "Get value of all dynamic configurations' value overridden on local config") + @ApiOperation(value = "Get value of all dynamic configurations' value overridden on local config", + response = String.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 403, message = "You don't have admin permission to view configuration"), @ApiResponse(code = 404, message = "Configuration not found"), @@ -258,7 +259,8 @@ public void getAllDynamicConfigurations(@Suspended AsyncResponse asyncResponse) @GET @Path("/configuration") - @ApiOperation(value = "Get all updatable dynamic configurations's name") + @ApiOperation(value = "Get all updatable dynamic configurations's name", + response = String.class, responseContainer = "List") @ApiResponses(value = { @ApiResponse(code = 403, message = "You don't have admin permission to get configuration")}) public void getDynamicConfigurationName(@Suspended AsyncResponse asyncResponse) { @@ -273,7 +275,8 @@ public void getDynamicConfigurationName(@Suspended AsyncResponse asyncResponse) @GET @Path("/configuration/runtime") - @ApiOperation(value = "Get all runtime configurations. This operation requires Pulsar super-user privileges.") + @ApiOperation(value = "Get all runtime configurations. This operation requires Pulsar super-user privileges.", + response = String.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) public void getRuntimeConfiguration(@Suspended AsyncResponse asyncResponse) { validateSuperUserAccessAsync() @@ -330,7 +333,7 @@ public void getInternalConfigurationData(@Suspended AsyncResponse asyncResponse) @Path("/backlog-quota-check") @ApiOperation(value = "An REST endpoint to trigger backlogQuotaCheck") @ApiResponses(value = { - @ApiResponse(code = 200, message = "Everything is OK"), + @ApiResponse(code = 204, message = "Everything is OK"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 500, message = "Internal server error")}) public void backlogQuotaCheck(@Suspended AsyncResponse asyncResponse) { @@ -368,15 +371,15 @@ public void isReady(@Suspended AsyncResponse asyncResponse) { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Cluster doesn't exist"), @ApiResponse(code = 500, message = "Internal server error")}) - @ApiParam(value = "Topic Version") public void healthCheck(@Suspended AsyncResponse asyncResponse, + @ApiParam(value = "Topic Version") @QueryParam("topicVersion") TopicVersion topicVersion) { validateSuperUserAccessAsync() .thenAccept(__ -> checkDeadlockedThreads()) .thenCompose(__ -> internalRunHealthCheck(topicVersion)) .thenAccept(__ -> { LOG.info("[{}] Successfully run health check.", clientAppId()); - asyncResponse.resume("ok"); + asyncResponse.resume(Response.ok("ok").build()); }).exceptionally(ex -> { LOG.error("[{}] Fail to run health check.", clientAppId(), ex); resumeAsyncResponseExceptionally(asyncResponse, ex); @@ -541,7 +544,7 @@ private CompletableFuture internalDeleteDynamicConfigurationOnMetadataAsyn @Path("/version") @ApiOperation(value = "Get version of current broker") @ApiResponses(value = { - @ApiResponse(code = 200, message = "Everything is OK"), + @ApiResponse(code = 200, message = "The Pulsar version", response = String.class), @ApiResponse(code = 500, message = "Internal server error")}) public String version() throws Exception { return PulsarVersion.getVersion(); 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 5d4ed54c33466..7c3d0290c54f0 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 @@ -132,7 +132,7 @@ public void getCluster(@Suspended AsyncResponse asyncResponse, notes = "This operation requires Pulsar superuser privileges, and the name cannot contain the '/' characters." ) @ApiResponses(value = { - @ApiResponse(code = 204, message = "Cluster has been created."), + @ApiResponse(code = 200, message = "Cluster has been created."), @ApiResponse(code = 400, message = "Bad request parameter."), @ApiResponse(code = 403, message = "You don't have admin permission to create the cluster."), @ApiResponse(code = 409, message = "Cluster already exists."), @@ -198,7 +198,7 @@ public void createCluster( value = "Update the configuration for a cluster.", notes = "This operation requires Pulsar superuser privileges.") @ApiResponses(value = { - @ApiResponse(code = 204, message = "Cluster has been updated."), + @ApiResponse(code = 200, message = "Cluster has been updated."), @ApiResponse(code = 400, message = "Bad request parameter."), @ApiResponse(code = 403, message = "Don't have admin permission or policies are read-only."), @ApiResponse(code = 404, message = "Cluster doesn't exist."), @@ -253,7 +253,7 @@ public void updateCluster( value = "Update the configuration for a cluster migration.", notes = "This operation requires Pulsar superuser privileges.") @ApiResponses(value = { - @ApiResponse(code = 204, message = "Cluster has been updated."), + @ApiResponse(code = 200, message = "Cluster has been updated."), @ApiResponse(code = 400, message = "Cluster url must not be empty."), @ApiResponse(code = 403, message = "Don't have admin permission or policies are read-only."), @ApiResponse(code = 404, message = "Cluster doesn't exist."), @@ -652,6 +652,7 @@ public void getBrokerWithNamespaceIsolationPolicy( notes = "This operation requires Pulsar superuser privileges." ) @ApiResponses(value = { + @ApiResponse(code = 204, message = "Set namespace isolation policy successfully."), @ApiResponse(code = 400, message = "Namespace isolation policy data is invalid."), @ApiResponse(code = 403, message = "Don't have admin permission or policies are read-only."), @ApiResponse(code = 404, message = "Namespace isolation policy doesn't exist."), @@ -762,6 +763,7 @@ private CompletableFuture filterAndUnloadMatchedNamespaceAsync(NamespaceIs notes = "This operation requires Pulsar superuser privileges." ) @ApiResponses(value = { + @ApiResponse(code = 204, message = "Delete namespace isolation policy successfully."), @ApiResponse(code = 403, message = "Don't have admin permission or policies are read only."), @ApiResponse(code = 404, message = "Namespace isolation policy doesn't exist."), @ApiResponse(code = 412, message = "Cluster doesn't exist."), @@ -809,6 +811,7 @@ public void deleteNamespaceIsolationPolicy( notes = "This operation requires Pulsar superuser privileges." ) @ApiResponses(value = { + @ApiResponse(code = 204, message = "Set the failure domain of the cluster successfully."), @ApiResponse(code = 403, message = "Don't have admin permission."), @ApiResponse(code = 404, message = "Failure domain doesn't exist."), @ApiResponse(code = 409, message = "Broker already exists in another domain."), @@ -944,6 +947,7 @@ public void getDomain( notes = "This operation requires Pulsar superuser privileges." ) @ApiResponses(value = { + @ApiResponse(code = 200, message = "Delete the failure domain of the cluster successfully"), @ApiResponse(code = 403, message = "Don't have admin permission or policy is read only"), @ApiResponse(code = 404, message = "FailureDomain doesn't exist"), @ApiResponse(code = 412, message = "Cluster doesn't exist"), diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Bookies.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Bookies.java index a50bc7515ff6f..ba02a22cbc2c8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Bookies.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Bookies.java @@ -20,6 +20,7 @@ import io.swagger.annotations.Api; import io.swagger.annotations.ApiOperation; +import io.swagger.annotations.ApiParam; import io.swagger.annotations.ApiResponse; import io.swagger.annotations.ApiResponses; import java.util.ArrayList; @@ -124,7 +125,10 @@ public void getBookieRackInfo(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/racks-info/{bookie}") @ApiOperation(value = "Removed the rack placement information for a specific bookie in the cluster") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission")}) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission") + }) public void deleteBookieRackInfo(@Suspended final AsyncResponse asyncResponse, @PathParam("bookie") String bookieAddress) throws Exception { validateSuperUserAccess(); @@ -153,11 +157,17 @@ public void deleteBookieRackInfo(@Suspended final AsyncResponse asyncResponse, @Path("/racks-info/{bookie}") @ApiOperation(value = "Updates the rack placement information for a specific bookie in the cluster (note." + " bookie address format:`address:port`)") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission")}) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission")} + ) public void updateBookieRackInfo(@Suspended final AsyncResponse asyncResponse, + @ApiParam(value = "The bookie address", required = true) @PathParam("bookie") String bookieAddress, + @ApiParam(value = "The group", required = true) @QueryParam("group") String group, - BookieInfo bookieInfo) throws Exception { + @ApiParam(value = "The bookie info", required = true) + BookieInfo bookieInfo) throws Exception { validateSuperUserAccess(); if (group == null) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/BrokerStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/BrokerStats.java index aba6cb1a0aba4..6f280e8d197f8 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/BrokerStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/BrokerStats.java @@ -61,7 +61,12 @@ public StreamingOutput getTopics2() throws Exception { + "sum of all of the resource usage percent is called broker-resource-availability" + "

    THIS API IS ONLY FOR USE BY TESTING FOR CONFIRMING NAMESPACE ALLOCATION ALGORITHM", response = ResourceUnit.class, responseContainer = "Map") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Returns broker resource availability as Map>." + + "Since `ResourceUnit` is an interface, its specific content is not determinable via class " + + "reflection. Refer to the source code or interface tests for detailed type definitions.", + response = Map.class), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 409, message = "Load-manager doesn't support operation") }) public Map> getBrokerResourceAvailability(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { From 37f16db3ab8e50266873cf8f14a19b248f086754 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Mon, 27 May 2024 21:37:24 +0800 Subject: [PATCH 520/699] [fix][admin] Clearly define REST API on Open API for Namesaces@v2 (#22775) (cherry picked from commit 9b3876df70f3b1d8bc01a34308d718c456f1781b) (cherry picked from commit 7558b2d3c4bd968734887087482bf76e113348d4) --- .../pulsar/broker/admin/v2/Namespaces.java | 440 +++++++++++++----- 1 file changed, 317 insertions(+), 123 deletions(-) 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 317ff49e5423c..259195056e326 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 @@ -24,6 +24,8 @@ import io.swagger.annotations.ApiParam; import io.swagger.annotations.ApiResponse; import io.swagger.annotations.ApiResponses; +import io.swagger.annotations.Example; +import io.swagger.annotations.ExampleProperty; import java.io.OutputStreamWriter; import java.nio.charset.StandardCharsets; import java.util.HashSet; @@ -59,9 +61,11 @@ import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType; import org.apache.pulsar.common.policies.data.BookieAffinityGroupData; import org.apache.pulsar.common.policies.data.DelayedDeliveryPolicies; +import org.apache.pulsar.common.policies.data.DispatchRate; import org.apache.pulsar.common.policies.data.EntryFilters; import org.apache.pulsar.common.policies.data.InactiveTopicPolicies; import org.apache.pulsar.common.policies.data.NamespaceOperation; +import org.apache.pulsar.common.policies.data.OffloadPolicies; import org.apache.pulsar.common.policies.data.OffloadPoliciesImpl; import org.apache.pulsar.common.policies.data.PersistencePolicies; import org.apache.pulsar.common.policies.data.Policies; @@ -73,6 +77,12 @@ import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; import org.apache.pulsar.common.policies.data.SubscribeRate; import org.apache.pulsar.common.policies.data.SubscriptionAuthMode; +import org.apache.pulsar.common.policies.data.TopicHashPositions; +import org.apache.pulsar.common.policies.data.impl.AutoSubscriptionCreationOverrideImpl; +import org.apache.pulsar.common.policies.data.impl.AutoTopicCreationOverrideImpl; +import org.apache.pulsar.common.policies.data.impl.BacklogQuotaImpl; +import org.apache.pulsar.common.policies.data.impl.BookieAffinityGroupDataImpl; +import org.apache.pulsar.common.policies.data.impl.BundlesDataImpl; import org.apache.pulsar.common.policies.data.impl.DispatchRateImpl; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.metadata.api.MetadataStoreException; @@ -152,7 +162,9 @@ public void getPolicies(@Suspended AsyncResponse response, @PUT @Path("/{tenant}/{namespace}") @ApiOperation(value = "Creates a new namespace with the specified policies") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster doesn't exist"), @ApiResponse(code = 409, message = "Namespace already exists"), @ApiResponse(code = 412, message = "Namespace name is not valid") }) @@ -180,6 +192,7 @@ public void createNamespace(@Suspended AsyncResponse response, @Path("/{tenant}/{namespace}") @ApiOperation(value = "Delete a namespace and all the topics under it.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @@ -208,6 +221,7 @@ public void deleteNamespace(@Suspended final AsyncResponse asyncResponse, @PathP @Path("/{tenant}/{namespace}/{bundle}") @ApiOperation(value = "Delete a namespace bundle and all the topics under it.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @@ -231,7 +245,10 @@ public void deleteNamespaceBundle(@Suspended AsyncResponse response, @PathParam( @GET @Path("/{tenant}/{namespace}/permissions") - @ApiOperation(value = "Retrieve the permissions for a namespace.") + @ApiOperation(value = "Retrieve the permissions for a namespace.", + notes = "Returns a nested map structure which Swagger does not fully support for display. " + + "Structure: Map>. Please refer to this structure for details.", + response = AuthAction.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Namespace is not empty") }) @@ -251,7 +268,10 @@ public void getPermissions(@Suspended AsyncResponse response, @GET @Path("/{tenant}/{namespace}/permissions/subscription") - @ApiOperation(value = "Retrieve the permissions for a subscription.") + @ApiOperation(value = "Retrieve the permissions for a subscription.", + notes = "Returns a nested map structure which Swagger does not fully support for display. " + + "Structure: Map>. Please refer to this structure for details.", + response = String.class, responseContainer = "Map") @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Namespace is not empty")}) @@ -273,7 +293,9 @@ public void getPermissionOnSubscription(@Suspended AsyncResponse response, @POST @Path("/{tenant}/{namespace}/permissions/{role}") @ApiOperation(value = "Grant a new permission to a role on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 501, message = "Authorization is not enabled")}) @@ -297,7 +319,9 @@ public void grantPermissionOnNamespace(@Suspended AsyncResponse asyncResponse, @Path("/{property}/{namespace}/permissions/subscription/{subscription}") @ApiOperation(hidden = true, value = "Grant a new permission to roles for a subscription." + "[Tenant admin is allowed to perform this operation]") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 501, message = "Authorization is not enabled") }) @@ -321,7 +345,9 @@ public void grantPermissionOnSubscription(@Suspended AsyncResponse asyncResponse @DELETE @Path("/{tenant}/{namespace}/permissions/{role}") @ApiOperation(value = "Revoke all permissions to a role on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void revokePermissionsOnNamespace(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -340,7 +366,9 @@ public void revokePermissionsOnNamespace(@Suspended AsyncResponse asyncResponse, @DELETE @Path("/{property}/{namespace}/permissions/{subscription}/{role}") @ApiOperation(hidden = true, value = "Revoke subscription admin-api access permission for a role.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Property or cluster or namespace doesn't exist") }) public void revokePermissionOnSubscription(@Suspended AsyncResponse asyncResponse, @PathParam("property") String property, @@ -360,7 +388,7 @@ public void revokePermissionOnSubscription(@Suspended AsyncResponse asyncRespons @GET @Path("/{tenant}/{namespace}/replication") @ApiOperation(value = "Get the replication clusters for a namespace.", - response = String.class, responseContainer = "List") + response = String.class, responseContainer = "Set") @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Namespace is not global")}) @@ -381,7 +409,9 @@ public void getNamespaceReplicationClusters(@Suspended AsyncResponse asyncRespon @POST @Path("/{tenant}/{namespace}/replication") @ApiOperation(value = "Set the replication clusters for a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Peer-cluster can't be part of replication-cluster"), @ApiResponse(code = 412, message = "Namespace is not global or invalid cluster ids") }) @@ -422,7 +452,9 @@ public void getNamespaceMessageTTL(@Suspended AsyncResponse asyncResponse, @Path @POST @Path("/{tenant}/{namespace}/messageTTL") @ApiOperation(value = "Set message TTL in seconds for namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid TTL") }) public void setNamespaceMessageTTL(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -442,7 +474,9 @@ public void setNamespaceMessageTTL(@Suspended AsyncResponse asyncResponse, @Path @DELETE @Path("/{tenant}/{namespace}/messageTTL") @ApiOperation(value = "Remove message TTL in seconds for namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid TTL")}) public void removeNamespaceMessageTTL(@Suspended AsyncResponse asyncResponse, @@ -460,7 +494,7 @@ public void removeNamespaceMessageTTL(@Suspended AsyncResponse asyncResponse, @GET @Path("/{tenant}/{namespace}/subscriptionExpirationTime") - @ApiOperation(value = "Get the subscription expiration time for the namespace") + @ApiOperation(value = "Get the subscription expiration time for the namespace", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void getSubscriptionExpirationTime(@Suspended AsyncResponse asyncResponse, @@ -482,7 +516,9 @@ public void getSubscriptionExpirationTime(@Suspended AsyncResponse asyncResponse @POST @Path("/{tenant}/{namespace}/subscriptionExpirationTime") @ApiOperation(value = "Set subscription expiration time in minutes for namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid expiration time")}) public void setSubscriptionExpirationTime(@Suspended AsyncResponse asyncResponse, @@ -505,7 +541,9 @@ public void setSubscriptionExpirationTime(@Suspended AsyncResponse asyncResponse @DELETE @Path("/{tenant}/{namespace}/subscriptionExpirationTime") @ApiOperation(value = "Remove subscription expiration time for namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist")}) public void removeSubscriptionExpirationTime(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -523,7 +561,7 @@ public void removeSubscriptionExpirationTime(@Suspended AsyncResponse asyncRespo @GET @Path("/{tenant}/{namespace}/deduplication") - @ApiOperation(value = "Get broker side deduplication for all topics in a namespace") + @ApiOperation(value = "Get broker side deduplication for all topics in a namespace", response = Boolean.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void getDeduplication(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -541,7 +579,9 @@ public void getDeduplication(@Suspended AsyncResponse asyncResponse, @PathParam( @POST @Path("/{tenant}/{namespace}/deduplication") @ApiOperation(value = "Enable or disable broker side deduplication for all topics in a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void modifyDeduplication(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -561,7 +601,9 @@ public void modifyDeduplication(@Suspended AsyncResponse asyncResponse, @PathPar @DELETE @Path("/{tenant}/{namespace}/deduplication") @ApiOperation(value = "Remove broker side deduplication for all topics in a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void removeDeduplication(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -578,7 +620,7 @@ public void removeDeduplication(@Suspended AsyncResponse asyncResponse, @PathPar @GET @Path("/{tenant}/{namespace}/autoTopicCreation") - @ApiOperation(value = "Get autoTopicCreation info in a namespace") + @ApiOperation(value = "Get autoTopicCreation info in a namespace", response = AutoTopicCreationOverrideImpl.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist")}) public void getAutoTopicCreation(@Suspended AsyncResponse asyncResponse, @@ -597,7 +639,9 @@ public void getAutoTopicCreation(@Suspended AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/autoTopicCreation") @ApiOperation(value = "Override broker's allowAutoTopicCreation setting for a namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 406, message = "The number of partitions should be less than or" + " equal to maxNumPartitionsPerPartitionedTopic"), @@ -633,7 +677,9 @@ public void setAutoTopicCreation( @DELETE @Path("/{tenant}/{namespace}/autoTopicCreation") @ApiOperation(value = "Remove override of broker's allowAutoTopicCreation in a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void removeAutoTopicCreation(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -661,7 +707,9 @@ public void removeAutoTopicCreation(@Suspended final AsyncResponse asyncResponse @POST @Path("/{tenant}/{namespace}/autoSubscriptionCreation") @ApiOperation(value = "Override broker's allowAutoSubscriptionCreation setting for a namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 400, message = "Invalid autoSubscriptionCreation override")}) public void setAutoSubscriptionCreation( @@ -691,7 +739,8 @@ public void setAutoSubscriptionCreation( @GET @Path("/{tenant}/{namespace}/autoSubscriptionCreation") - @ApiOperation(value = "Get autoSubscriptionCreation info in a namespace") + @ApiOperation(value = "Get autoSubscriptionCreation info in a namespace", + response = AutoSubscriptionCreationOverrideImpl.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist")}) public void getAutoSubscriptionCreation(@Suspended final AsyncResponse asyncResponse, @@ -710,7 +759,9 @@ public void getAutoSubscriptionCreation(@Suspended final AsyncResponse asyncResp @DELETE @Path("/{tenant}/{namespace}/autoSubscriptionCreation") @ApiOperation(value = "Remove override of broker's allowAutoSubscriptionCreation in a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void removeAutoSubscriptionCreation(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -736,7 +787,7 @@ public void removeAutoSubscriptionCreation(@Suspended final AsyncResponse asyncR @GET @Path("/{tenant}/{namespace}/bundles") - @ApiOperation(value = "Get the bundles split data.") + @ApiOperation(value = "Get the bundles split data.", response = BundlesDataImpl.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Namespace is not setup to split in bundles") }) @@ -768,6 +819,7 @@ public void getBundlesData(@Suspended final AsyncResponse asyncResponse, + " since it wouldresult in non-persistent message loss and" + " unexpected connection closure to the clients.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), @@ -800,6 +852,7 @@ public void unloadNamespace(@Suspended final AsyncResponse asyncResponse, @Path("/{tenant}/{namespace}/{bundle}/unload") @ApiOperation(value = "Unload a namespace bundle") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 403, message = "Don't have admin permission") }) @@ -829,6 +882,7 @@ public void unloadNamespaceBundle(@Suspended final AsyncResponse asyncResponse, @Path("/{tenant}/{namespace}/{bundle}/split") @ApiOperation(value = "Split a namespace bundle") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 403, message = "Don't have admin permission") }) @@ -865,7 +919,7 @@ public void splitNamespaceBundle( @GET @Path("/{tenant}/{namespace}/{bundle}/topicHashPositions") - @ApiOperation(value = "Get hash positions for topics") + @ApiOperation(value = "Get hash positions for topics", response = TopicHashPositions.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist")}) @@ -891,7 +945,9 @@ public void getTopicHashPositions( @POST @Path("/{property}/{namespace}/publishRate") @ApiOperation(hidden = true, value = "Set publish-rate throttling for all topics of the namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission") }) public void setPublishRate(@Suspended AsyncResponse asyncResponse, @PathParam("property") String property, @PathParam("namespace") String namespace, @ApiParam(value = "Publish rate for all topics of the specified namespace") PublishRate publishRate) { @@ -907,7 +963,9 @@ public void setPublishRate(@Suspended AsyncResponse asyncResponse, @PathParam("p @DELETE @Path("/{property}/{namespace}/publishRate") @ApiOperation(hidden = true, value = "Set publish-rate throttling for all topics of the namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission") }) public void removePublishRate(@Suspended AsyncResponse asyncResponse, @PathParam("property") String property, @PathParam("namespace") String namespace) { validateNamespaceName(property, namespace); @@ -925,7 +983,8 @@ public void removePublishRate(@Suspended AsyncResponse asyncResponse, @PathParam @Path("/{property}/{namespace}/publishRate") @ApiOperation(hidden = true, value = "Get publish-rate configured for the namespace, null means publish-rate not configured, " - + "-1 means msg-publish-rate or byte-publish-rate not configured in publish-rate yet") + + "-1 means msg-publish-rate or byte-publish-rate not configured in publish-rate yet", + response = PublishRate.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist")}) public void getPublishRate(@Suspended AsyncResponse asyncResponse, @@ -944,7 +1003,9 @@ public void getPublishRate(@Suspended AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/dispatchRate") @ApiOperation(value = "Set dispatch-rate throttling for all topics of the namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission") }) public void setDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @ApiParam(value = "Dispatch rate for all topics of the specified namespace") @@ -963,7 +1024,9 @@ public void setDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam(" @DELETE @Path("/{tenant}/{namespace}/dispatchRate") @ApiOperation(value = "Delete dispatch-rate throttling for all topics of the namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission") }) public void deleteDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { validateNamespaceName(tenant, namespace); @@ -980,7 +1043,8 @@ public void deleteDispatchRate(@Suspended AsyncResponse asyncResponse, @PathPara @GET @Path("/{tenant}/{namespace}/dispatchRate") @ApiOperation(value = "Get dispatch-rate configured for the namespace, null means dispatch-rate not configured, " - + "-1 means msg-dispatch-rate or byte-dispatch-rate not configured in dispatch-rate yet") + + "-1 means msg-dispatch-rate or byte-dispatch-rate not configured in dispatch-rate yet", + response = DispatchRate.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -997,7 +1061,9 @@ public void getDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam(" @POST @Path("/{tenant}/{namespace}/subscriptionDispatchRate") @ApiOperation(value = "Set Subscription dispatch-rate throttling for all topics of the namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission")}) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission")}) public void setSubscriptionDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -1019,7 +1085,7 @@ public void setSubscriptionDispatchRate(@Suspended AsyncResponse asyncResponse, @Path("/{tenant}/{namespace}/subscriptionDispatchRate") @ApiOperation(value = "Get subscription dispatch-rate configured for the namespace, null means subscription " + "dispatch-rate not configured, -1 means msg-dispatch-rate or byte-dispatch-rate not configured " - + "in dispatch-rate yet") + + "in dispatch-rate yet", response = DispatchRate.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist")}) public void getSubscriptionDispatchRate(@Suspended AsyncResponse asyncResponse, @@ -1039,7 +1105,9 @@ public void getSubscriptionDispatchRate(@Suspended AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/subscriptionDispatchRate") @ApiOperation(value = "Delete Subscription dispatch-rate throttling for all topics of the namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission") }) public void deleteSubscriptionDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -1057,7 +1125,9 @@ public void deleteSubscriptionDispatchRate(@Suspended AsyncResponse asyncRespons @DELETE @Path("/{tenant}/{namespace}/subscribeRate") @ApiOperation(value = "Delete subscribe-rate throttling for all topics of the namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission")}) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission")}) public void deleteSubscribeRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { validateNamespaceName(tenant, namespace); @@ -1074,7 +1144,9 @@ public void deleteSubscribeRate(@Suspended AsyncResponse asyncResponse, @PathPar @POST @Path("/{tenant}/{namespace}/subscribeRate") @ApiOperation(value = "Set subscribe-rate throttling for all topics of the namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission")}) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission")}) public void setSubscribeRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @ApiParam(value = "Subscribe rate for all topics of the specified namespace") @@ -1092,7 +1164,7 @@ public void setSubscribeRate(@Suspended AsyncResponse asyncResponse, @PathParam( @GET @Path("/{tenant}/{namespace}/subscribeRate") - @ApiOperation(value = "Get subscribe-rate configured for the namespace") + @ApiOperation(value = "Get subscribe-rate configured for the namespace", response = SubscribeRate.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist")}) public void getSubscribeRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -1110,7 +1182,9 @@ public void getSubscribeRate(@Suspended AsyncResponse asyncResponse, @PathParam( @DELETE @Path("/{tenant}/{namespace}/replicatorDispatchRate") @ApiOperation(value = "Remove replicator dispatch-rate throttling for all topics of the namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission")}) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission")}) public void removeReplicatorDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -1121,7 +1195,9 @@ public void removeReplicatorDispatchRate(@Suspended AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/replicatorDispatchRate") @ApiOperation(value = "Set replicator dispatch-rate throttling for all topics of the namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission")}) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission")}) public void setReplicatorDispatchRate(@Suspended AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -1135,7 +1211,7 @@ public void setReplicatorDispatchRate(@Suspended AsyncResponse asyncResponse, @Path("/{tenant}/{namespace}/replicatorDispatchRate") @ApiOperation(value = "Get replicator dispatch-rate configured for the namespace, null means replicator " + "dispatch-rate not configured, -1 means msg-dispatch-rate or byte-dispatch-rate not configured " - + "in dispatch-rate yet") + + "in dispatch-rate yet", response = DispatchRateImpl.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getReplicatorDispatchRate(@Suspended final AsyncResponse asyncResponse, @@ -1147,7 +1223,8 @@ public void getReplicatorDispatchRate(@Suspended final AsyncResponse asyncRespon @GET @Path("/{tenant}/{namespace}/backlogQuotaMap") - @ApiOperation(value = "Get backlog quota map on a namespace.") + @ApiOperation(value = "Get backlog quota map on a namespace.", + response = BacklogQuotaImpl.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getBacklogQuotaMap( @@ -1161,7 +1238,9 @@ public void getBacklogQuotaMap( @POST @Path("/{tenant}/{namespace}/backlogQuota") @ApiOperation(value = " Set a backlog quota for all the topics on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, @@ -1179,7 +1258,9 @@ public void setBacklogQuota( @DELETE @Path("/{tenant}/{namespace}/backlogQuota") @ApiOperation(value = "Remove a backlog quota policy from a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void removeBacklogQuota( @@ -1192,7 +1273,7 @@ public void removeBacklogQuota( @GET @Path("/{tenant}/{namespace}/retention") - @ApiOperation(value = "Get retention config on a namespace.") + @ApiOperation(value = "Get retention config on a namespace.", response = RetentionPolicies.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getRetention(@Suspended final AsyncResponse asyncResponse, @@ -1213,7 +1294,9 @@ public void getRetention(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/retention") @ApiOperation(value = " Set retention configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "Retention Quota must exceed backlog quota") }) @@ -1226,7 +1309,9 @@ public void setRetention(@PathParam("tenant") String tenant, @PathParam("namespa @DELETE @Path("/{tenant}/{namespace}/retention") @ApiOperation(value = " Remove retention configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "Retention Quota must exceed backlog quota") }) @@ -1239,7 +1324,9 @@ public void removeRetention(@PathParam("tenant") String tenant, @PathParam("name @POST @Path("/{tenant}/{namespace}/persistence") @ApiOperation(value = "Set the persistence configuration for all the topics on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 400, message = "Invalid persistence policies")}) @@ -1261,7 +1348,9 @@ public void setPersistence(@Suspended final AsyncResponse asyncResponse, @PathPa @DELETE @Path("/{tenant}/{namespace}/persistence") @ApiOperation(value = "Delete the persistence configuration for all topics on a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission") }) + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission") }) public void deletePersistence(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { validateNamespaceName(tenant, namespace); @@ -1279,6 +1368,7 @@ public void deletePersistence(@Suspended final AsyncResponse asyncResponse, @Pat @Path("/{tenant}/{namespace}/persistence/bookieAffinity") @ApiOperation(value = "Set the bookie-affinity-group to namespace-persistent policy.") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @@ -1292,7 +1382,8 @@ public void setBookieAffinityGroup(@PathParam("tenant") String tenant, @PathPara @GET @Path("/{property}/{namespace}/persistence/bookieAffinity") - @ApiOperation(value = "Get the bookie-affinity-group from namespace-local policy.") + @ApiOperation(value = "Get the bookie-affinity-group from namespace-local policy.", + response = BookieAffinityGroupDataImpl.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -1307,7 +1398,9 @@ public BookieAffinityGroupData getBookieAffinityGroup(@PathParam("property") Str @DELETE @Path("/{property}/{namespace}/persistence/bookieAffinity") @ApiOperation(value = "Delete the bookie-affinity-group from namespace-local policy.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void deleteBookieAffinityGroup(@PathParam("property") String property, @@ -1318,7 +1411,7 @@ public void deleteBookieAffinityGroup(@PathParam("property") String property, @GET @Path("/{tenant}/{namespace}/persistence") - @ApiOperation(value = "Get the persistence configuration for a namespace.") + @ApiOperation(value = "Get the persistence configuration for a namespace.", response = PersistencePolicies.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) @@ -1342,6 +1435,7 @@ public void getPersistence( @Path("/{tenant}/{namespace}/clearBacklog") @ApiOperation(value = "Clear backlog for all topics on a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin or operate permission on the namespace"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void clearNamespaceBacklog(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -1361,6 +1455,7 @@ public void clearNamespaceBacklog(@Suspended final AsyncResponse asyncResponse, @Path("/{tenant}/{namespace}/{bundle}/clearBacklog") @ApiOperation(value = "Clear backlog for all topics on a namespace bundle.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin or operate permission on the namespace"), @ApiResponse(code = 404, message = "Namespace does not exist") }) @@ -1375,6 +1470,7 @@ public void clearNamespaceBundleBacklog(@PathParam("tenant") String tenant, @Path("/{tenant}/{namespace}/clearBacklog/{subscription}") @ApiOperation(value = "Clear backlog for a given subscription on all topics on a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin or operate permission on the namespace"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void clearNamespaceBacklogForSubscription(@Suspended final AsyncResponse asyncResponse, @@ -1395,6 +1491,7 @@ public void clearNamespaceBacklogForSubscription(@Suspended final AsyncResponse @Path("/{tenant}/{namespace}/{bundle}/clearBacklog/{subscription}") @ApiOperation(value = "Clear backlog for a given subscription on all topics on a namespace bundle.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin or operate permission on the namespace"), @ApiResponse(code = 404, message = "Namespace does not exist") }) @@ -1410,6 +1507,7 @@ public void clearNamespaceBundleBacklogForSubscription(@PathParam("tenant") Stri @Path("/{tenant}/{namespace}/unsubscribe/{subscription}") @ApiOperation(value = "Unsubscribes the given subscription on all topics on a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin or operate permission on the namespacen"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void unsubscribeNamespace(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -1430,6 +1528,7 @@ public void unsubscribeNamespace(@Suspended final AsyncResponse asyncResponse, @ @Path("/{tenant}/{namespace}/{bundle}/unsubscribe/{subscription}") @ApiOperation(value = "Unsubscribes the given subscription on all topics on a namespace bundle.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin or operate permission on the namespace"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void unsubscribeNamespaceBundle(@PathParam("tenant") String tenant, @@ -1443,7 +1542,9 @@ public void unsubscribeNamespaceBundle(@PathParam("tenant") String tenant, @POST @Path("/{tenant}/{namespace}/subscriptionAuthMode") @ApiOperation(value = " Set a subscription auth mode for all the topics on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void setSubscriptionAuthMode(@PathParam("tenant") String tenant, @@ -1456,7 +1557,7 @@ public void setSubscriptionAuthMode(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/subscriptionAuthMode") - @ApiOperation(value = "Get subscription auth mode in a namespace") + @ApiOperation(value = "Get subscription auth mode in a namespace", response = SubscriptionAuthMode.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist")}) public void getSubscriptionAuthMode( @@ -1478,7 +1579,9 @@ public void getSubscriptionAuthMode( @POST @Path("/{tenant}/{namespace}/encryptionRequired") @ApiOperation(value = "Message encryption is required or not for all topics in a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), }) public void modifyEncryptionRequired( @@ -1492,7 +1595,7 @@ public void modifyEncryptionRequired( @GET @Path("/{tenant}/{namespace}/encryptionRequired") - @ApiOperation(value = "Get message encryption required status in a namespace") + @ApiOperation(value = "Get message encryption required status in a namespace", response = Boolean.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist")}) public void getEncryptionRequired(@Suspended AsyncResponse asyncResponse, @@ -1512,7 +1615,8 @@ public void getEncryptionRequired(@Suspended AsyncResponse asyncResponse, @GET @Path("/{tenant}/{namespace}/delayedDelivery") - @ApiOperation(value = "Get delayed delivery messages config on a namespace.") + @ApiOperation(value = "Get delayed delivery messages config on a namespace.", + response = DelayedDeliveryPolicies.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), }) @@ -1534,7 +1638,9 @@ public void getDelayedDeliveryPolicies(@Suspended final AsyncResponse asyncRespo @POST @Path("/{tenant}/{namespace}/delayedDelivery") @ApiOperation(value = "Set delayed delivery messages config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), }) public void setDelayedDeliveryPolicies(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -1547,7 +1653,9 @@ public void setDelayedDeliveryPolicies(@PathParam("tenant") String tenant, @DELETE @Path("/{tenant}/{namespace}/delayedDelivery") @ApiOperation(value = "Delete delayed delivery messages config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), }) public void removeDelayedDeliveryPolicies(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -1557,7 +1665,7 @@ public void removeDelayedDeliveryPolicies(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/inactiveTopicPolicies") - @ApiOperation(value = "Get inactive topic policies config on a namespace.") + @ApiOperation(value = "Get inactive topic policies config on a namespace.", response = InactiveTopicPolicies.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), }) @@ -1579,7 +1687,9 @@ public void getInactiveTopicPolicies(@Suspended final AsyncResponse asyncRespons @DELETE @Path("/{tenant}/{namespace}/inactiveTopicPolicies") @ApiOperation(value = "Remove inactive topic policies from a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void removeInactiveTopicPolicies(@PathParam("tenant") String tenant, @@ -1591,7 +1701,9 @@ public void removeInactiveTopicPolicies(@PathParam("tenant") String tenant, @POST @Path("/{tenant}/{namespace}/inactiveTopicPolicies") @ApiOperation(value = "Set inactive topic policies config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), }) public void setInactiveTopicPolicies(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -1603,7 +1715,7 @@ public void setInactiveTopicPolicies(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/maxProducersPerTopic") - @ApiOperation(value = "Get maxProducersPerTopic config on a namespace.") + @ApiOperation(value = "Get maxProducersPerTopic config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getMaxProducersPerTopic( @@ -1625,7 +1737,9 @@ public void getMaxProducersPerTopic( @POST @Path("/{tenant}/{namespace}/maxProducersPerTopic") @ApiOperation(value = " Set maxProducersPerTopic configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxProducersPerTopic value is not valid") }) @@ -1638,7 +1752,9 @@ public void setMaxProducersPerTopic(@PathParam("tenant") String tenant, @PathPar @DELETE @Path("/{tenant}/{namespace}/maxProducersPerTopic") @ApiOperation(value = "Remove maxProducersPerTopic configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void removeMaxProducersPerTopic(@PathParam("tenant") String tenant, @@ -1649,7 +1765,7 @@ public void removeMaxProducersPerTopic(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/deduplicationSnapshotInterval") - @ApiOperation(value = "Get deduplicationSnapshotInterval config on a namespace.") + @ApiOperation(value = "Get deduplicationSnapshotInterval config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getDeduplicationSnapshotInterval( @@ -1671,7 +1787,9 @@ public void getDeduplicationSnapshotInterval( @POST @Path("/{tenant}/{namespace}/deduplicationSnapshotInterval") @ApiOperation(value = "Set deduplicationSnapshotInterval config on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist")}) public void setDeduplicationSnapshotInterval(@PathParam("tenant") String tenant , @PathParam("namespace") String namespace @@ -1683,7 +1801,7 @@ public void setDeduplicationSnapshotInterval(@PathParam("tenant") String tenant @GET @Path("/{tenant}/{namespace}/maxConsumersPerTopic") - @ApiOperation(value = "Get maxConsumersPerTopic config on a namespace.") + @ApiOperation(value = "Get maxConsumersPerTopic config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getMaxConsumersPerTopic( @@ -1705,7 +1823,9 @@ public void getMaxConsumersPerTopic( @POST @Path("/{tenant}/{namespace}/maxConsumersPerTopic") @ApiOperation(value = " Set maxConsumersPerTopic configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxConsumersPerTopic value is not valid") }) @@ -1718,7 +1838,9 @@ public void setMaxConsumersPerTopic(@PathParam("tenant") String tenant, @PathPar @DELETE @Path("/{tenant}/{namespace}/maxConsumersPerTopic") @ApiOperation(value = "Remove maxConsumersPerTopic configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void removeMaxConsumersPerTopic(@PathParam("tenant") String tenant, @@ -1729,7 +1851,7 @@ public void removeMaxConsumersPerTopic(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/maxConsumersPerSubscription") - @ApiOperation(value = "Get maxConsumersPerSubscription config on a namespace.") + @ApiOperation(value = "Get maxConsumersPerSubscription config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getMaxConsumersPerSubscription( @@ -1751,7 +1873,9 @@ public void getMaxConsumersPerSubscription( @POST @Path("/{tenant}/{namespace}/maxConsumersPerSubscription") @ApiOperation(value = " Set maxConsumersPerSubscription configuration on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxConsumersPerSubscription value is not valid")}) @@ -1767,7 +1891,9 @@ public void setMaxConsumersPerSubscription(@PathParam("tenant") String tenant, @DELETE @Path("/{tenant}/{namespace}/maxConsumersPerSubscription") @ApiOperation(value = " Set maxConsumersPerSubscription configuration on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxConsumersPerSubscription value is not valid")}) @@ -1779,7 +1905,7 @@ public void removeMaxConsumersPerSubscription(@PathParam("tenant") String tenant @GET @Path("/{tenant}/{namespace}/maxUnackedMessagesPerConsumer") - @ApiOperation(value = "Get maxUnackedMessagesPerConsumer config on a namespace.") + @ApiOperation(value = "Get maxUnackedMessagesPerConsumer config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getMaxUnackedMessagesPerConsumer(@Suspended final AsyncResponse asyncResponse, @@ -1800,7 +1926,9 @@ public void getMaxUnackedMessagesPerConsumer(@Suspended final AsyncResponse asyn @POST @Path("/{tenant}/{namespace}/maxUnackedMessagesPerConsumer") @ApiOperation(value = " Set maxConsumersPerTopic configuration on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxUnackedMessagesPerConsumer value is not valid")}) @@ -1816,7 +1944,9 @@ public void setMaxUnackedMessagesPerConsumer(@PathParam("tenant") String tenant, @DELETE @Path("/{tenant}/{namespace}/maxUnackedMessagesPerConsumer") @ApiOperation(value = "Remove maxUnackedMessagesPerConsumer config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void removeMaxUnackedmessagesPerConsumer(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -1826,7 +1956,7 @@ public void removeMaxUnackedmessagesPerConsumer(@PathParam("tenant") String tena @GET @Path("/{tenant}/{namespace}/maxUnackedMessagesPerSubscription") - @ApiOperation(value = "Get maxUnackedMessagesPerSubscription config on a namespace.") + @ApiOperation(value = "Get maxUnackedMessagesPerSubscription config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getMaxUnackedmessagesPerSubscription( @@ -1848,7 +1978,9 @@ public void getMaxUnackedmessagesPerSubscription( @POST @Path("/{tenant}/{namespace}/maxUnackedMessagesPerSubscription") @ApiOperation(value = " Set maxUnackedMessagesPerSubscription configuration on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxUnackedMessagesPerSubscription value is not valid")}) @@ -1863,7 +1995,9 @@ public void setMaxUnackedMessagesPerSubscription( @DELETE @Path("/{tenant}/{namespace}/maxUnackedMessagesPerSubscription") @ApiOperation(value = "Remove maxUnackedMessagesPerSubscription config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void removeMaxUnackedmessagesPerSubscription(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -1873,7 +2007,7 @@ public void removeMaxUnackedmessagesPerSubscription(@PathParam("tenant") String @GET @Path("/{tenant}/{namespace}/maxSubscriptionsPerTopic") - @ApiOperation(value = "Get maxSubscriptionsPerTopic config on a namespace.") + @ApiOperation(value = "Get maxSubscriptionsPerTopic config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getMaxSubscriptionsPerTopic(@Suspended final AsyncResponse asyncResponse, @@ -1894,7 +2028,9 @@ public void getMaxSubscriptionsPerTopic(@Suspended final AsyncResponse asyncResp @POST @Path("/{tenant}/{namespace}/maxSubscriptionsPerTopic") @ApiOperation(value = " Set maxSubscriptionsPerTopic configuration on a namespace.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "maxUnackedMessagesPerSubscription value is not valid")}) @@ -1909,7 +2045,9 @@ public void setMaxSubscriptionsPerTopic( @DELETE @Path("/{tenant}/{namespace}/maxSubscriptionsPerTopic") @ApiOperation(value = "Remove maxSubscriptionsPerTopic configuration on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void removeMaxSubscriptionsPerTopic(@PathParam("tenant") String tenant, @@ -1921,7 +2059,9 @@ public void removeMaxSubscriptionsPerTopic(@PathParam("tenant") String tenant, @POST @Path("/{tenant}/{namespace}/antiAffinity") @ApiOperation(value = "Set anti-affinity group for a namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid antiAffinityGroup")}) public void setNamespaceAntiAffinityGroup(@PathParam("tenant") String tenant, @@ -1935,7 +2075,7 @@ public void setNamespaceAntiAffinityGroup(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/antiAffinity") - @ApiOperation(value = "Get anti-affinity group of a namespace.") + @ApiOperation(value = "Get anti-affinity group of a namespace.", response = String.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public String getNamespaceAntiAffinityGroup(@PathParam("tenant") String tenant, @@ -1947,7 +2087,9 @@ public String getNamespaceAntiAffinityGroup(@PathParam("tenant") String tenant, @DELETE @Path("/{tenant}/{namespace}/antiAffinity") @ApiOperation(value = "Remove anti-affinity group of a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void removeNamespaceAntiAffinityGroup(@PathParam("tenant") String tenant, @@ -1959,7 +2101,8 @@ public void removeNamespaceAntiAffinityGroup(@PathParam("tenant") String tenant, @GET @Path("{cluster}/antiAffinity/{group}") @ApiOperation(value = "Get all namespaces that are grouped by given anti-affinity group in a given cluster." - + " api can be only accessed by admin of any of the existing tenant") + + " api can be only accessed by admin of any of the existing tenant", + response = String.class, responseContainer = "List") @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 412, message = "Cluster not exist/Anti-affinity group can't be empty.")}) public List getAntiAffinityNamespaces(@PathParam("cluster") String cluster, @@ -1985,7 +2128,7 @@ private Policies getDefaultPolicesIfNull(Policies policies) { @Path("/{tenant}/{namespace}/compactionThreshold") @ApiOperation(value = "Maximum number of uncompacted bytes in topics before compaction is triggered.", notes = "The backlog size is compared to the threshold periodically. " - + "A threshold of 0 disabled automatic compaction") + + "A threshold of 0 disabled automatic compaction", response = Long.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist") }) public void getCompactionThreshold( @@ -2009,7 +2152,9 @@ public void getCompactionThreshold( @ApiOperation(value = "Set maximum number of uncompacted bytes in a topic before compaction is triggered.", notes = "The backlog size is compared to the threshold periodically. " + "A threshold of 0 disabled automatic compaction") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "compactionThreshold value is not valid")}) @@ -2027,7 +2172,9 @@ public void setCompactionThreshold(@PathParam("tenant") String tenant, @ApiOperation(value = "Delete maximum number of uncompacted bytes in a topic before compaction is triggered.", notes = "The backlog size is compared to the threshold periodically. " + "A threshold of 0 disabled automatic compaction") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void deleteCompactionThreshold(@PathParam("tenant") String tenant, @@ -2040,7 +2187,7 @@ public void deleteCompactionThreshold(@PathParam("tenant") String tenant, @Path("/{tenant}/{namespace}/offloadThreshold") @ApiOperation(value = "Maximum number of bytes stored on the pulsar cluster for a topic," + " before the broker will start offloading to longterm storage", - notes = "A negative value disables automatic offloading") + notes = "A negative value disables automatic offloading", response = Long.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist") }) public void getOffloadThreshold( @@ -2071,7 +2218,9 @@ public void getOffloadThreshold( + " before the broker will start offloading to longterm storage", notes = "-1 will revert to using the cluster default." + " A negative value disables automatic offloading. ") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "offloadThreshold value is not valid")}) @@ -2089,7 +2238,7 @@ public void setOffloadThreshold(@PathParam("tenant") String tenant, @Path("/{tenant}/{namespace}/offloadThresholdInSeconds") @ApiOperation(value = "Maximum number of bytes stored on the pulsar cluster for a topic," + " before the broker will start offloading to longterm storage", - notes = "A negative value disables automatic offloading") + notes = "A negative value disables automatic offloading", response = Long.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist") }) public void getOffloadThresholdInSeconds( @@ -2119,7 +2268,9 @@ public void getOffloadThresholdInSeconds( @ApiOperation(value = "Set maximum number of seconds stored on the pulsar cluster for a topic," + " before the broker will start offloading to longterm storage", notes = "A negative value disables automatic offloading") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "offloadThresholdInSeconds value is not valid") }) @@ -2143,7 +2294,7 @@ public void setOffloadThresholdInSeconds( + " from the Pulsar cluster's local storage (i.e. BookKeeper)", notes = "A negative value denotes that deletion has been completely disabled." + " 'null' denotes that the topics in the namespace will fall back to the" - + " broker default for deletion lag.") + + " broker default for deletion lag.", response = Long.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist") }) public void getOffloadDeletionLag( @@ -2173,7 +2324,9 @@ public void getOffloadDeletionLag( @ApiOperation(value = "Set number of milliseconds to wait before deleting a ledger segment which has been offloaded" + " from the Pulsar cluster's local storage (i.e. BookKeeper)", notes = "A negative value disables the deletion completely.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 412, message = "offloadDeletionLagMs value is not valid")}) @@ -2192,6 +2345,7 @@ public void setOffloadDeletionLag(@PathParam("tenant") String tenant, @ApiOperation(value = "Clear the namespace configured offload deletion lag. The topics in the namespace" + " will fallback to using the default configured deletion lag for the broker") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) public void clearOffloadDeletionLag(@PathParam("tenant") String tenant, @@ -2205,7 +2359,8 @@ public void clearOffloadDeletionLag(@PathParam("tenant") String tenant, @ApiOperation(value = "The strategy used to check the compatibility of new schemas," + " provided by producers, before automatically updating the schema", notes = "The value AutoUpdateDisabled prevents producers from updating the schema. " - + " If set to AutoUpdateDisabled, schemas must be updated through the REST api") + + " If set to AutoUpdateDisabled, schemas must be updated through the REST api", + response = SchemaAutoUpdateCompatibilityStrategy.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) @@ -2222,7 +2377,9 @@ public SchemaAutoUpdateCompatibilityStrategy getSchemaAutoUpdateCompatibilityStr + " provided by producers, before automatically updating the schema", notes = "The value AutoUpdateDisabled prevents producers from updating the schema. " + " If set to AutoUpdateDisabled, schemas must be updated through the REST api") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void setSchemaAutoUpdateCompatibilityStrategy( @@ -2236,7 +2393,8 @@ public void setSchemaAutoUpdateCompatibilityStrategy( @GET @Path("/{tenant}/{namespace}/schemaCompatibilityStrategy") - @ApiOperation(value = "The strategy of the namespace schema compatibility ") + @ApiOperation(value = "The strategy of the namespace schema compatibility ", + response = SchemaCompatibilityStrategy.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) @@ -2260,7 +2418,9 @@ public void getSchemaCompatibilityStrategy( @PUT @Path("/{tenant}/{namespace}/schemaCompatibilityStrategy") @ApiOperation(value = "Update the strategy used to check the compatibility of new schema") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void setSchemaCompatibilityStrategy( @@ -2274,7 +2434,7 @@ public void setSchemaCompatibilityStrategy( @GET @Path("/{tenant}/{namespace}/isAllowAutoUpdateSchema") - @ApiOperation(value = "The flag of whether allow auto update schema") + @ApiOperation(value = "The flag of whether allow auto update schema", response = Boolean.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) @@ -2304,7 +2464,9 @@ public void getIsAllowAutoUpdateSchema( @POST @Path("/{tenant}/{namespace}/isAllowAutoUpdateSchema") @ApiOperation(value = "Update flag of whether allow auto update schema") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void setIsAllowAutoUpdateSchema( @@ -2318,7 +2480,8 @@ public void setIsAllowAutoUpdateSchema( @GET @Path("/{tenant}/{namespace}/subscriptionTypesEnabled") - @ApiOperation(value = "The set of whether allow subscription types") + @ApiOperation(value = "The set of whether allow subscription types", + response = SubscriptionType.class, responseContainer = "Set") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification") }) @@ -2346,7 +2509,9 @@ public void getSubscriptionTypesEnabled( @POST @Path("/{tenant}/{namespace}/subscriptionTypesEnabled") @ApiOperation(value = "Update set of whether allow share sub type") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification")}) public void setSubscriptionTypesEnabled( @@ -2377,7 +2542,8 @@ public void removeSubscriptionTypesEnabled(@PathParam("tenant") String tenant, notes = "If the flag is set to true, when a producer without a schema attempts to produce to a topic" + " with schema in this namespace, the producer will be failed to connect. PLEASE be" + " carefully on using this, since non-java clients don't support schema.if you enable" - + " this setting, it will cause non-java clients failed to produce.") + + " this setting, it will cause non-java clients failed to produce.", + response = Boolean.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenants or Namespace doesn't exist") }) public void getSchemaValidtionEnforced( @@ -2412,7 +2578,9 @@ public void getSchemaValidtionEnforced( + " with schema in this namespace, the producer will be failed to connect. PLEASE be" + " carefully on using this, since non-java clients don't support schema.if you enable" + " this setting, it will cause non-java clients failed to produce.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or Namespace doesn't exist"), @ApiResponse(code = 412, message = "schemaValidationEnforced value is not valid")}) public void setSchemaValidationEnforced(@PathParam("tenant") String tenant, @@ -2427,8 +2595,9 @@ public void setSchemaValidationEnforced(@PathParam("tenant") String tenant, @POST @Path("/{tenant}/{namespace}/offloadPolicies") - @ApiOperation(value = " Set offload configuration on a namespace.") + @ApiOperation(value = "Set offload configuration on a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @@ -2452,6 +2621,7 @@ public void setOffloadPolicies(@PathParam("tenant") String tenant, @PathParam("n @Path("/{tenant}/{namespace}/removeOffloadPolicies") @ApiOperation(value = " Set offload configuration on a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @@ -2471,7 +2641,7 @@ public void removeOffloadPolicies(@PathParam("tenant") String tenant, @PathParam @GET @Path("/{tenant}/{namespace}/offloadPolicies") - @ApiOperation(value = "Get offload configuration on a namespace.") + @ApiOperation(value = "Get offload configuration on a namespace.", response = OffloadPolicies.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist")}) @@ -2493,7 +2663,7 @@ public void getOffloadPolicies( @GET @Path("/{tenant}/{namespace}/maxTopicsPerNamespace") - @ApiOperation(value = "Get maxTopicsPerNamespace config on a namespace.") + @ApiOperation(value = "Get maxTopicsPerNamespace config on a namespace.", response = Integer.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace does not exist") }) public void getMaxTopicsPerNamespace(@Suspended final AsyncResponse asyncResponse, @@ -2518,7 +2688,9 @@ public void getMaxTopicsPerNamespace(@Suspended final AsyncResponse asyncRespons @POST @Path("/{tenant}/{namespace}/maxTopicsPerNamespace") @ApiOperation(value = "Set maxTopicsPerNamespace config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void setMaxTopicsPerNamespace(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -2531,7 +2703,9 @@ public void setMaxTopicsPerNamespace(@PathParam("tenant") String tenant, @DELETE @Path("/{tenant}/{namespace}/maxTopicsPerNamespace") @ApiOperation(value = "Remove maxTopicsPerNamespace config on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void removeMaxTopicsPerNamespace(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -2542,7 +2716,9 @@ public void removeMaxTopicsPerNamespace(@PathParam("tenant") String tenant, @PUT @Path("/{tenant}/{namespace}/property/{key}/{value}") @ApiOperation(value = "Put a key value pair property on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void setProperty( @Suspended final AsyncResponse asyncResponse, @@ -2556,7 +2732,7 @@ public void setProperty( @GET @Path("/{tenant}/{namespace}/property/{key}") - @ApiOperation(value = "Get property value for a given key on a namespace.") + @ApiOperation(value = "Get property value for a given key on a namespace.", response = String.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void getProperty( @@ -2571,7 +2747,9 @@ public void getProperty( @DELETE @Path("/{tenant}/{namespace}/property/{key}") @ApiOperation(value = "Remove property value for a given key on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void removeProperty( @Suspended final AsyncResponse asyncResponse, @@ -2585,7 +2763,9 @@ public void removeProperty( @PUT @Path("/{tenant}/{namespace}/properties") @ApiOperation(value = "Put key value pairs property on a namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void setProperties( @Suspended final AsyncResponse asyncResponse, @@ -2599,7 +2779,8 @@ public void setProperties( @GET @Path("/{tenant}/{namespace}/properties") - @ApiOperation(value = "Get key value pair properties for a given namespace.") + @ApiOperation(value = "Get key value pair properties for a given namespace.", + response = String.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void getProperties( @@ -2613,7 +2794,9 @@ public void getProperties( @DELETE @Path("/{tenant}/{namespace}/properties") @ApiOperation(value = "Clear properties on a given namespace.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), }) public void clearProperties( @Suspended final AsyncResponse asyncResponse, @@ -2625,7 +2808,7 @@ public void clearProperties( @GET @Path("/{tenant}/{namespace}/resourcegroup") - @ApiOperation(value = "Get the resource group attached to the namespace") + @ApiOperation(value = "Get the resource group attached to the namespace", response = String.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") }) public void getNamespaceResourceGroup( @@ -2647,7 +2830,9 @@ public void getNamespaceResourceGroup( @POST @Path("/{tenant}/{namespace}/resourcegroup/{resourcegroup}") @ApiOperation(value = "Set resourcegroup for a namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid resourcegroup") }) public void setNamespaceResourceGroup(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace, @@ -2659,7 +2844,9 @@ public void setNamespaceResourceGroup(@PathParam("tenant") String tenant, @PathP @DELETE @Path("/{tenant}/{namespace}/resourcegroup") @ApiOperation(value = "Delete resourcegroup for a namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid resourcegroup")}) public void removeNamespaceResourceGroup(@PathParam("tenant") String tenant, @@ -2671,7 +2858,13 @@ public void removeNamespaceResourceGroup(@PathParam("tenant") String tenant, @GET @Path("/{tenant}/{namespace}/scanOffloadedLedgers") @ApiOperation(value = "Trigger the scan of offloaded Ledgers on the LedgerOffloader for the given namespace") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 200, message = "Successful get of offloaded ledger data", response = String.class, + examples = @Example(value = { @ExampleProperty(mediaType = "application/json", + value = "{\"objects\":[{\"key1\":\"value1\",\"key2\":\"value2\"}]," + + "\"total\":100,\"errors\":5,\"unknown\":3}") + })), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace doesn't exist") }) public Response scanOffloadedLedgers(@PathParam("tenant") String tenant, @PathParam("namespace") String namespace) { @@ -2720,7 +2913,7 @@ public void finished(int total, int errors, int unknown) throws Exception { @GET @Path("/{tenant}/{namespace}/entryFilters") - @ApiOperation(value = "Get maxConsumersPerSubscription config on a namespace.") + @ApiOperation(value = "Get maxConsumersPerSubscription config on a namespace.", response = EntryFilters.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace does not exist") }) public void getEntryFiltersPerTopic( @@ -2743,6 +2936,7 @@ public void getEntryFiltersPerTopic( @Path("/{tenant}/{namespace}/entryFilters") @ApiOperation(value = "Set entry filters for namespace") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 400, message = "Specified entry filters are not valid"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist") @@ -2764,7 +2958,9 @@ public void setEntryFiltersPerTopic(@Suspended AsyncResponse asyncResponse, @Pat @DELETE @Path("/{tenant}/{namespace}/entryFilters") @ApiOperation(value = "Remove entry filters for namespace") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), @ApiResponse(code = 412, message = "Invalid TTL")}) public void removeNamespaceEntryFilters(@Suspended AsyncResponse asyncResponse, @@ -2780,7 +2976,5 @@ public void removeNamespaceEntryFilters(@Suspended AsyncResponse asyncResponse, }); } - - private static final Logger log = LoggerFactory.getLogger(Namespaces.class); } From 80fd4823f6239b028fabd8c1f65f784cbbb891f8 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Mon, 27 May 2024 23:45:50 +0800 Subject: [PATCH 521/699] [fix][admin] Clearly define REST API on Open API for Topics (#22782) (cherry picked from commit c25d7b20b21e66f122f949b5a26fa32b433632b7) (cherry picked from commit 08ed1e2879c30bf84e186ad02c2b7d25b1f03b4e) --- .../broker/admin/v2/NonPersistentTopics.java | 19 +- .../broker/admin/v2/PersistentTopics.java | 319 ++++++++++++++---- 2 files changed, 264 insertions(+), 74 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java index f8fe30ccecb2c..cae7c651ce791 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java @@ -52,8 +52,10 @@ import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.web.RestException; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.EntryFilters; import org.apache.pulsar.common.policies.data.NamespaceOperation; +import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.TopicOperation; import org.apache.pulsar.common.policies.data.TopicStats; @@ -74,7 +76,7 @@ public class NonPersistentTopics extends PersistentTopics { @GET @Path("/{tenant}/{namespace}/{topic}/partitions") - @ApiOperation(value = "Get partitioned topic metadata.") + @ApiOperation(value = "Get partitioned topic metadata.", response = PartitionedTopicMetadata.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to manage resources on this tenant"), @@ -102,7 +104,7 @@ public void getPartitionedMetadata( @GET @Path("{tenant}/{namespace}/{topic}/internalStats") - @ApiOperation(value = "Get the internal stats for the topic.") + @ApiOperation(value = "Get the internal stats for the topic.", response = PersistentTopicInternalStats.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to manage resources on this tenant"), @@ -145,6 +147,7 @@ public void getInternalStats( @ApiOperation(value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to manage resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -306,6 +309,7 @@ public void getPartitionedStats( @Path("/{tenant}/{namespace}/{topic}/unload") @ApiOperation(value = "Unload a topic") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "This operation requires super-user access"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -491,6 +495,7 @@ public void getListFromBundle( @ApiOperation(value = "Truncate a topic.", notes = "NonPersistentTopic does not support truncate.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 412, message = "NonPersistentTopic does not support truncate.") }) public void truncateTopic( @@ -514,7 +519,7 @@ protected void validateAdminOperationOnTopic(TopicName topicName, boolean author @GET @Path("/{tenant}/{namespace}/{topic}/entryFilters") - @ApiOperation(value = "Get entry filters for a topic.") + @ApiOperation(value = "Get entry filters for a topic.", response = EntryFilters.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenants or Namespace doesn't exist") }) public void getEntryFilters(@Suspended AsyncResponse asyncResponse, @@ -542,7 +547,9 @@ public void getEntryFilters(@Suspended AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/entryFilters") @ApiOperation(value = "Set entry filters for specified topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -570,7 +577,9 @@ public void setEntryFilters(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/entryFilters") @ApiOperation(value = "Remove entry filters for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), 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 94fb1f53ac710..e619769d81ea0 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 @@ -49,7 +49,10 @@ import org.apache.pulsar.broker.admin.impl.PersistentTopicsBase; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.client.admin.LongRunningProcessStatus; +import org.apache.pulsar.client.admin.OffloadProcessStatus; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.ResetCursorData; @@ -162,7 +165,10 @@ public void getPartitionedTopicList( @ApiOperation(value = "Get permissions on a topic.", notes = "Retrieve the effective permissions for a topic." + " These permissions are defined by the permissions set at the" - + "namespace level combined (union) with any eventual specific permission set on the topic.") + + "namespace level combined (union) with any eventual specific permission set on the topic." + + "Returns a nested map structure which Swagger does not fully support for display. " + + "Structure: Map>. Please refer to this structure for details.", + response = AuthAction.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -195,6 +201,7 @@ public void getPermissionsOnTopic( @Path("/{tenant}/{namespace}/{topic}/permissions/{role}") @ApiOperation(value = "Grant a new permission to a role on a single topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -232,6 +239,7 @@ public void grantPermissionsOnTopic( + "level, but rather at the namespace level," + " this operation will return an error (HTTP status code 412).") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -263,6 +271,7 @@ public void revokePermissionsOnTopic( @ApiOperation(value = "Create a partitioned topic.", notes = "It needs to be called before creating a producer on a partitioned topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -305,6 +314,7 @@ public void createPartitionedTopic( @ApiOperation(value = "Create a non-partitioned topic.", notes = "This is the only REST endpoint from which non-partitioned topics could be created.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 404, message = "Tenant or namespace doesn't exist"), @@ -344,7 +354,7 @@ public void createNonPartitionedTopic( @GET @Path("/{tenant}/{namespace}/{topic}/offloadPolicies") - @ApiOperation(value = "Get offload policies on a topic.") + @ApiOperation(value = "Get offload policies on a topic.", response = OffloadPoliciesImpl.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), @ApiResponse(code = 500, message = "Internal server error"), }) @@ -370,7 +380,9 @@ public void getOffloadPolicies(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/offloadPolicies") @ApiOperation(value = "Set offload policies on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void setOffloadPolicies(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -394,7 +406,9 @@ public void setOffloadPolicies(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/offloadPolicies") @ApiOperation(value = "Delete offload policies on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void removeOffloadPolicies(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -441,7 +455,9 @@ public void getMaxUnackedMessagesOnConsumer(@Suspended final AsyncResponse async @POST @Path("/{tenant}/{namespace}/{topic}/maxUnackedMessagesOnConsumer") @ApiOperation(value = "Set max unacked messages per consumer config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void setMaxUnackedMessagesOnConsumer( @Suspended final AsyncResponse asyncResponse, @@ -467,7 +483,9 @@ public void setMaxUnackedMessagesOnConsumer( @DELETE @Path("/{tenant}/{namespace}/{topic}/maxUnackedMessagesOnConsumer") @ApiOperation(value = "Delete max unacked messages per consumer config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void deleteMaxUnackedMessagesOnConsumer(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -517,7 +535,9 @@ public void getDeduplicationSnapshotInterval(@Suspended final AsyncResponse asyn @POST @Path("/{tenant}/{namespace}/{topic}/deduplicationSnapshotInterval") @ApiOperation(value = "Set deduplicationSnapshotInterval config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void setDeduplicationSnapshotInterval( @Suspended final AsyncResponse asyncResponse, @@ -543,7 +563,9 @@ public void setDeduplicationSnapshotInterval( @DELETE @Path("/{tenant}/{namespace}/{topic}/deduplicationSnapshotInterval") @ApiOperation(value = "Delete deduplicationSnapshotInterval config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void deleteDeduplicationSnapshotInterval(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -590,7 +612,9 @@ public void getInactiveTopicPolicies(@Suspended final AsyncResponse asyncRespons @POST @Path("/{tenant}/{namespace}/{topic}/inactiveTopicPolicies") @ApiOperation(value = "Set inactive topic policies on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void setInactiveTopicPolicies(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -615,7 +639,9 @@ public void setInactiveTopicPolicies(@Suspended final AsyncResponse asyncRespons @DELETE @Path("/{tenant}/{namespace}/{topic}/inactiveTopicPolicies") @ApiOperation(value = "Delete inactive topic policies on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void deleteInactiveTopicPolicies(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -663,7 +689,9 @@ public void getMaxUnackedMessagesOnSubscription(@Suspended final AsyncResponse a @POST @Path("/{tenant}/{namespace}/{topic}/maxUnackedMessagesOnSubscription") @ApiOperation(value = "Set max unacked messages per subscription config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void setMaxUnackedMessagesOnSubscription( @Suspended final AsyncResponse asyncResponse, @@ -691,7 +719,9 @@ public void setMaxUnackedMessagesOnSubscription( @DELETE @Path("/{tenant}/{namespace}/{topic}/maxUnackedMessagesOnSubscription") @ApiOperation(value = "Delete max unacked messages per subscription config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void deleteMaxUnackedMessagesOnSubscription(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -739,7 +769,9 @@ public void getDelayedDeliveryPolicies(@Suspended final AsyncResponse asyncRespo @POST @Path("/{tenant}/{namespace}/{topic}/delayedDelivery") @ApiOperation(value = "Set delayed delivery messages config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void setDelayedDeliveryPolicies( @Suspended final AsyncResponse asyncResponse, @@ -768,7 +800,9 @@ public void setDelayedDeliveryPolicies( @DELETE @Path("/{tenant}/{namespace}/{topic}/delayedDelivery") @ApiOperation(value = "Set delayed delivery messages config on a topic.") - @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), }) public void deleteDelayedDeliveryPolicies(@Suspended final AsyncResponse asyncResponse, @PathParam("tenant") String tenant, @@ -851,6 +885,7 @@ public void updatePartitionedTopic( @Path("/{tenant}/{namespace}/{topic}/createMissedPartitions") @ApiOperation(value = "Create missed partitions of an existing partitioned topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @@ -951,6 +986,7 @@ public void getProperties( @Path("/{tenant}/{namespace}/{topic}/properties") @ApiOperation(value = "Update the properties on the given topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -987,6 +1023,7 @@ public void updateProperties( @Path("/{tenant}/{namespace}/{topic}/properties") @ApiOperation(value = "Remove the key in properties on the given topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -1023,6 +1060,7 @@ public void removeProperties( @ApiOperation(value = "Delete a partitioned topic.", notes = "It will also delete all the partitions of the topic if it exists.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -1063,6 +1101,7 @@ public void deletePartitionedTopic( @Path("/{tenant}/{namespace}/{topic}/unload") @ApiOperation(value = "Unload a topic") @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 = "Namespace or topic does not exist"), @@ -1097,6 +1136,7 @@ public void unloadTopic( + "subscription or producer connected to the it. " + "Force delete ignores connected clients and deletes topic by explicitly closing them.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -1360,6 +1400,7 @@ public void getPartitionedStatsInternal( + " there are any active consumers attached to it. " + "Force delete ignores connected consumers and deletes subscription by explicitly closing them.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -1412,6 +1453,7 @@ public void deleteSubscription( @ApiOperation(value = "Skip all messages on a topic subscription.", notes = "Completely clears the backlog on the subscription.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1447,6 +1489,7 @@ public void skipAllMessages( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/skip/{numMessages}") @ApiOperation(value = "Skipping messages on a topic subscription.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -1483,6 +1526,7 @@ public void skipMessages( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/expireMessages/{expireTimeInSeconds}") @ApiOperation(value = "Expiry messages on a topic subscription.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1520,6 +1564,7 @@ public void expireTopicMessages( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/expireMessages") @ApiOperation(value = "Expiry messages on a topic subscription.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1559,6 +1604,7 @@ public void expireTopicMessages( @Path("/{tenant}/{namespace}/{topic}/all_subscription/expireMessages/{expireTimeInSeconds}") @ApiOperation(value = "Expiry messages on all subscriptions of topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1595,6 +1641,7 @@ public void expireMessagesForAllSubscriptions( @ApiOperation(value = "Create a subscription on the topic.", notes = "Creates a subscription on the topic at the specified message id") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 400, message = "Create subscription on non persistent topic is not supported"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -1650,6 +1697,7 @@ public void createSubscription( @ApiOperation(value = "Reset subscription to message position closest to absolute timestamp (in ms).", notes = "It fence cursor and disconnects all active consumers before resetting cursor.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1700,6 +1748,7 @@ public void resetCursor( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/properties") @ApiOperation(value = "Replace all the properties on the given subscription") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1735,7 +1784,8 @@ public void updateSubscriptionProperties( @GET @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/properties") - @ApiOperation(value = "Return all the properties on the given subscription") + @ApiOperation(value = "Return all the properties on the given subscription", + response = String.class, responseContainer = "Map") @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -1773,6 +1823,7 @@ public void getSubscriptionProperties( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/analyzeBacklog") @ApiOperation(value = "Analyse a subscription, by scanning all the unprocessed messages") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1819,6 +1870,7 @@ public void analyzeSubscriptionBacklog( @ApiOperation(value = "Reset subscription to message position closest to given position.", notes = "It fence cursor and disconnects all active consumers before resetting cursor.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1857,6 +1909,13 @@ public void resetCursorOnPosition( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/position/{messagePosition}") @ApiOperation(value = "Peek nth message on a topic subscription.") @ApiResponses(value = { + @ApiResponse( + code = 200, + message = "Successfully retrieved the message. The response is a binary byte stream " + + "containing the message data. Clients need to parse this binary stream based" + + " on the message metadata provided in the response headers.", + response = byte[].class + ), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1899,6 +1958,13 @@ public void peekNthMessage( @ApiOperation(value = "Examine a specific message on a topic by position relative to the earliest or the latest message.") @ApiResponses(value = { + @ApiResponse( + code = 200, + message = "Successfully retrieved the message. The response is a binary byte stream " + + "containing the message data. Clients need to parse this binary stream based" + + " on the message metadata provided in the response headers.", + response = byte[].class + ), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic, the message position does not exist"), @@ -1941,6 +2007,13 @@ public void examineMessage( @Path("/{tenant}/{namespace}/{topic}/ledger/{ledgerId}/entry/{entryId}") @ApiOperation(value = "Get message by its messageId.") @ApiResponses(value = { + @ApiResponse( + code = 200, + message = "Successfully retrieved the message. The response is a binary byte stream " + + "containing the message data. Clients need to parse this binary stream based" + + " on the message metadata provided in the response headers.", + response = byte[].class + ), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -1981,7 +2054,8 @@ public void getMessageById( @GET @Path("/{tenant}/{namespace}/{topic}/messageid/{timestamp}") - @ApiOperation(value = "Get message ID published at or just after this absolute timestamp (in ms).") + @ApiOperation(value = "Get message ID published at or just after this absolute timestamp (in ms).", + response = MessageIdAdv.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -2111,7 +2185,9 @@ public void getBacklogQuotaMap( @POST @Path("/{tenant}/{namespace}/{topic}/backlogQuota") @ApiOperation(value = "Set a backlog quota for a topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 405, @@ -2140,7 +2216,9 @@ public void setBacklogQuota( @DELETE @Path("/{tenant}/{namespace}/{topic}/backlogQuota") @ApiOperation(value = "Remove a backlog quota policy from a topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2202,7 +2280,9 @@ public void getReplicationClusters(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/replication") @ApiOperation(value = "Set the replication clusters for a topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 409, message = "Concurrent modification"), @ApiResponse(code = 405, @@ -2228,7 +2308,9 @@ public void setReplicationClusters( @DELETE @Path("/{tenant}/{namespace}/{topic}/replication") @ApiOperation(value = "Remove the replication clusters from a topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2287,7 +2369,9 @@ public void getMessageTTL(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/messageTTL") @ApiOperation(value = "Set message TTL in seconds for a topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Not authenticate to perform the request or policy is read only"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = @@ -2317,6 +2401,7 @@ public void setMessageTTL(@Suspended final AsyncResponse asyncResponse, @Path("/{tenant}/{namespace}/{topic}/messageTTL") @ApiOperation(value = "Remove message TTL in seconds for a topic") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Not authenticate to perform the request or policy is read only"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @@ -2371,7 +2456,9 @@ public void getDeduplication(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/deduplicationEnabled") @ApiOperation(value = "Set deduplication enabled on a topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry")}) @@ -2399,7 +2486,9 @@ public void setDeduplication( @DELETE @Path("/{tenant}/{namespace}/{topic}/deduplicationEnabled") @ApiOperation(value = "Remove deduplication configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2452,7 +2541,9 @@ public void getRetention(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/retention") @ApiOperation(value = "Set retention configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2490,7 +2581,9 @@ public void setRetention(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/retention") @ApiOperation(value = "Remove retention configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2552,7 +2645,9 @@ public void getPersistence(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/persistence") @ApiOperation(value = "Set configuration of persistence policies for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2592,7 +2687,9 @@ public void setPersistence(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/persistence") @ApiOperation(value = "Remove configuration of persistence policies for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2651,7 +2748,9 @@ public void getMaxSubscriptionsPerTopic(@Suspended final AsyncResponse asyncResp @POST @Path("/{tenant}/{namespace}/{topic}/maxSubscriptionsPerTopic") @ApiOperation(value = "Set maxSubscriptionsPerTopic config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2684,7 +2783,9 @@ public void setMaxSubscriptionsPerTopic(@Suspended final AsyncResponse asyncResp @DELETE @Path("/{tenant}/{namespace}/{topic}/maxSubscriptionsPerTopic") @ApiOperation(value = "Remove maxSubscriptionsPerTopic config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2741,7 +2842,9 @@ public void getReplicatorDispatchRate(@Suspended final AsyncResponse asyncRespon @POST @Path("/{tenant}/{namespace}/{topic}/replicatorDispatchRate") @ApiOperation(value = "Set replicatorDispatchRate config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2774,7 +2877,9 @@ public void setReplicatorDispatchRate(@Suspended final AsyncResponse asyncRespon @DELETE @Path("/{tenant}/{namespace}/{topic}/replicatorDispatchRate") @ApiOperation(value = "Remove replicatorDispatchRate config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2831,7 +2936,9 @@ public void getMaxProducers(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/maxProducers") @ApiOperation(value = "Set maxProducers config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2866,7 +2973,9 @@ public void setMaxProducers(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/maxProducers") @ApiOperation(value = "Remove maxProducers config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2925,7 +3034,9 @@ public void getMaxConsumers(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/maxConsumers") @ApiOperation(value = "Set maxConsumers config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -2960,7 +3071,9 @@ public void setMaxConsumers(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/maxConsumers") @ApiOperation(value = "Remove maxConsumers config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -3020,7 +3133,9 @@ public void getMaxMessageSize(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/maxMessageSize") @ApiOperation(value = "Set maxMessageSize config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -3057,7 +3172,9 @@ public void setMaxMessageSize(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/maxMessageSize") @ApiOperation(value = "Remove maxMessageSize config for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -3092,6 +3209,12 @@ public void removeMaxMessageSize(@Suspended final AsyncResponse asyncResponse, @ApiOperation(value = "Terminate a topic. A topic that is terminated will not accept any more " + "messages to be published and will let consumer to drain existing messages in backlog") @ApiResponses(value = { + @ApiResponse( + code = 200, + message = "Operation terminated successfully. The response includes the 'lastMessageId'," + + " which is the identifier of the last message processed.", + response = MessageIdAdv.class + ), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -3129,6 +3252,7 @@ public void terminate( @ApiOperation(value = "Terminate all partitioned topic. A topic that is terminated will not accept any more " + "messages to be published and will let consumer to drain existing messages in backlog") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -3155,6 +3279,7 @@ public void terminatePartitionedTopic(@Suspended final AsyncResponse asyncRespon @Path("/{tenant}/{namespace}/{topic}/compaction") @ApiOperation(value = "Trigger a compaction operation on a topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -3187,7 +3312,8 @@ public void compact( @GET @Path("/{tenant}/{namespace}/{topic}/compaction") - @ApiOperation(value = "Get the status of a compaction operation for a topic.") + @ApiOperation(value = "Get the status of a compaction operation for a topic.", + response = LongRunningProcessStatus.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -3225,6 +3351,7 @@ public void compactionStatus( @Path("/{tenant}/{namespace}/{topic}/offload") @ApiOperation(value = "Offload a prefix of a topic to long term storage") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 400, message = "Message ID is null"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -3262,7 +3389,7 @@ public void triggerOffload( @GET @Path("/{tenant}/{namespace}/{topic}/offload") - @ApiOperation(value = "Offload a prefix of a topic to long term storage") + @ApiOperation(value = "Offload a prefix of a topic to long term storage", response = OffloadProcessStatus.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -3295,7 +3422,7 @@ public void offloadStatus( @GET @Path("/{tenant}/{namespace}/{topic}/lastMessageId") - @ApiOperation(value = "Return the last commit message id of topic") + @ApiOperation(value = "Return the last commit message id of topic", response = MessageIdAdv.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" @@ -3328,6 +3455,7 @@ public void getLastMessageId( @Path("/{tenant}/{namespace}/{topic}/trim") @ApiOperation(value = " Trim a topic") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or" + "subscriber is not authorized to access this operation"), @@ -3364,7 +3492,7 @@ public void trimTopic( @GET @Path("/{tenant}/{namespace}/{topic}/dispatchRate") - @ApiOperation(value = "Get dispatch rate configuration for specified topic.") + @ApiOperation(value = "Get dispatch rate configuration for specified topic.", response = DispatchRateImpl.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, @@ -3392,7 +3520,9 @@ public void getDispatchRate(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/dispatchRate") @ApiOperation(value = "Set message dispatch rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3430,7 +3560,9 @@ public void setDispatchRate(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/dispatchRate") @ApiOperation(value = "Remove message dispatch rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3493,7 +3625,9 @@ public void getSubscriptionDispatchRate(@Suspended final AsyncResponse asyncResp @POST @Path("/{tenant}/{namespace}/{topic}/subscriptionDispatchRate") @ApiOperation(value = "Set subscription message dispatch rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3533,7 +3667,9 @@ public void setSubscriptionDispatchRate( @DELETE @Path("/{tenant}/{namespace}/{topic}/subscriptionDispatchRate") @ApiOperation(value = "Remove subscription message dispatch rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3565,7 +3701,8 @@ public void removeSubscriptionDispatchRate(@Suspended final AsyncResponse asyncR @GET @Path("/{tenant}/{namespace}/{topic}/{subName}/dispatchRate") - @ApiOperation(value = "Get message dispatch rate configuration for specified subscription.") + @ApiOperation(value = "Get message dispatch rate configuration for specified subscription.", + response = DispatchRate.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, @@ -3595,7 +3732,9 @@ public void getSubscriptionLevelDispatchRate(@Suspended final AsyncResponse asyn @POST @Path("/{tenant}/{namespace}/{topic}/{subName}/dispatchRate") @ApiOperation(value = "Set message dispatch rate configuration for specified subscription.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3636,7 +3775,9 @@ public void setSubscriptionLevelDispatchRate( @DELETE @Path("/{tenant}/{namespace}/{topic}/{subName}/dispatchRate") @ApiOperation(value = "Remove message dispatch rate configuration for specified subscription.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3697,7 +3838,9 @@ public void getCompactionThreshold(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/compactionThreshold") @ApiOperation(value = "Set compaction threshold configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3735,7 +3878,9 @@ public void setCompactionThreshold(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/compactionThreshold") @ApiOperation(value = "Remove compaction threshold configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3798,7 +3943,9 @@ public void getMaxConsumersPerSubscription(@Suspended final AsyncResponse asyncR @POST @Path("/{tenant}/{namespace}/{topic}/maxConsumersPerSubscription") @ApiOperation(value = "Set max consumers per subscription configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3837,7 +3984,9 @@ public void setMaxConsumersPerSubscription( @DELETE @Path("/{tenant}/{namespace}/{topic}/maxConsumersPerSubscription") @ApiOperation(value = "Remove max consumers per subscription configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3898,7 +4047,9 @@ public void getPublishRate(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/publishRate") @ApiOperation(value = "Set message publish rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -3937,7 +4088,9 @@ public void setPublishRate(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/publishRate") @ApiOperation(value = "Remove message publish rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -4004,7 +4157,9 @@ public void getSubscriptionTypesEnabled(@Suspended final AsyncResponse asyncResp @POST @Path("/{tenant}/{namespace}/{topic}/subscriptionTypesEnabled") @ApiOperation(value = "Set is enable sub types for specified topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -4043,7 +4198,9 @@ public void setSubscriptionTypesEnabled(@Suspended final AsyncResponse asyncResp @DELETE @Path("/{tenant}/{namespace}/{topic}/subscriptionTypesEnabled") @ApiOperation(value = "Remove subscription types enabled for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, to enable the topic level policy and retry"), @@ -4101,7 +4258,9 @@ public void getSubscribeRate(@Suspended final AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/subscribeRate") @ApiOperation(value = "Set subscribe rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -4141,7 +4300,9 @@ public void setSubscribeRate( @DELETE @Path("/{tenant}/{namespace}/{topic}/subscribeRate") @ApiOperation(value = "Remove subscribe rate configuration for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -4180,6 +4341,7 @@ public void removeSubscribeRate(@Suspended final AsyncResponse asyncResponse, notes = "The truncate operation will move all cursors to the end of the topic " + "and delete all inactive ledgers.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -4215,6 +4377,7 @@ public void truncateTopic( @Path("/{tenant}/{namespace}/{topic}/subscription/{subName}/replicatedSubscriptionStatus") @ApiOperation(value = "Enable or disable a replicated subscription on a topic.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 401, message = "Don't have permission to administrate resources on this tenant or " + "subscriber is not authorized to access this operation"), @@ -4311,6 +4474,7 @@ public void getSchemaCompatibilityStrategy( @Path("/{tenant}/{namespace}/{topic}/schemaCompatibilityStrategy") @ApiOperation(value = "Set schema compatibility strategy on a topic") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 405, message = "Operation not allowed on persistent topic"), @@ -4351,6 +4515,7 @@ public void setSchemaCompatibilityStrategy( @Path("/{tenant}/{namespace}/{topic}/schemaCompatibilityStrategy") @ApiOperation(value = "Remove schema compatibility strategy on a topic") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this topic"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 405, message = "Operation not allowed on persistent topic"), @@ -4389,7 +4554,7 @@ public void removeSchemaCompatibilityStrategy( @GET @Path("/{tenant}/{namespace}/{topic}/schemaValidationEnforced") - @ApiOperation(value = "Get schema validation enforced flag for topic.") + @ApiOperation(value = "Get schema validation enforced flag for topic.", response = Boolean.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenants or Namespace doesn't exist") }) public void getSchemaValidationEnforced(@Suspended AsyncResponse asyncResponse, @@ -4417,7 +4582,9 @@ public void getSchemaValidationEnforced(@Suspended AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/schemaValidationEnforced") @ApiOperation(value = "Set schema validation enforced flag on topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or Namespace doesn't exist"), @ApiResponse(code = 412, message = "schemaValidationEnforced value is not valid")}) public void setSchemaValidationEnforced(@Suspended AsyncResponse asyncResponse, @@ -4444,7 +4611,7 @@ public void setSchemaValidationEnforced(@Suspended AsyncResponse asyncResponse, @GET @Path("/{tenant}/{namespace}/{topic}/entryFilters") - @ApiOperation(value = "Get entry filters for a topic.") + @ApiOperation(value = "Get entry filters for a topic.", response = EntryFilters.class) @ApiResponses(value = { @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenants or Namespace doesn't exist") }) public void getEntryFilters(@Suspended AsyncResponse asyncResponse, @@ -4472,7 +4639,9 @@ public void getEntryFilters(@Suspended AsyncResponse asyncResponse, @POST @Path("/{tenant}/{namespace}/{topic}/entryFilters") @ApiOperation(value = "Set entry filters for specified topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -4500,7 +4669,9 @@ public void setEntryFilters(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}/{namespace}/{topic}/entryFilters") @ApiOperation(value = "Remove entry filters for specified topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), @@ -4534,7 +4705,8 @@ public void removeEntryFilters(@Suspended final AsyncResponse asyncResponse, @GET @Path("/{tenant}/{namespace}/{topic}/shadowTopics") - @ApiOperation(value = "Get the shadow topic list for a topic") + @ApiOperation(value = "Get the shadow topic list for a topic", + response = String.class, responseContainer = "List") @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = @@ -4561,7 +4733,9 @@ public void getShadowTopics( @PUT @Path("/{tenant}/{namespace}/{topic}/shadowTopics") @ApiOperation(value = "Set shadow topic list for a topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, enable the topic level policy and retry"), @@ -4588,7 +4762,9 @@ public void setShadowTopics( @DELETE @Path("/{tenant}/{namespace}/{topic}/shadowTopics") @ApiOperation(value = "Delete shadow topics for a topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Namespace or topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, enable the topic level policy and retry"), @@ -4614,7 +4790,9 @@ public void deleteShadowTopics( @POST @Path("/{tenant}/{namespace}/{topic}/autoSubscriptionCreation") @ApiOperation(value = "Override namespace's allowAutoSubscriptionCreation setting for a topic") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Topic doesn't exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, enable the topic level policy and retry"), @@ -4641,7 +4819,8 @@ public void setAutoSubscriptionCreation( @GET @Path("/{tenant}/{namespace}/{topic}/autoSubscriptionCreation") - @ApiOperation(value = "Get autoSubscriptionCreation info in a topic") + @ApiOperation(value = "Get autoSubscriptionCreation info in a topic", + response = AutoSubscriptionCreationOverrideImpl.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Topic does not exist"), @ApiResponse(code = 405, @@ -4668,7 +4847,9 @@ public void getAutoSubscriptionCreation( @DELETE @Path("/{tenant}/{namespace}/{topic}/autoSubscriptionCreation") @ApiOperation(value = "Remove autoSubscriptionCreation ina a topic.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Topic does not exist"), @ApiResponse(code = 405, message = "Topic level policy is disabled, please enable the topic level policy and retry"), From 0ddcd49e29cf9218db824b86d2f500775b37f3ed Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Mon, 27 May 2024 21:45:51 +0800 Subject: [PATCH 522/699] [fix][admin] Clearly define REST API on Open API (#22783) (cherry picked from commit ba20e02f01d75f0d4ec38393841bcf5c417e9363) (cherry picked from commit 9b018f5fe03478d3178a86aceba205546f59e19b) --- .../broker/admin/impl/FunctionsBase.java | 27 +++++++++----- .../pulsar/broker/admin/impl/SinksBase.java | 18 ++++++---- .../pulsar/broker/admin/impl/SourcesBase.java | 20 +++++++---- .../pulsar/broker/admin/impl/TenantsBase.java | 12 +++++-- .../broker/admin/v2/ResourceGroups.java | 5 ++- .../broker/admin/v2/ResourceQuotas.java | 4 ++- .../apache/pulsar/broker/admin/v2/Worker.java | 10 ++++-- .../pulsar/broker/admin/v3/Packages.java | 9 ++--- .../pulsar/broker/admin/v3/Transactions.java | 35 +++++++++++++------ 9 files changed, 96 insertions(+), 44 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/FunctionsBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/FunctionsBase.java index 4350316e2f011..42971ae231c05 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/FunctionsBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/FunctionsBase.java @@ -39,7 +39,6 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.StreamingOutput; import org.apache.pulsar.broker.admin.AdminResource; -import org.apache.pulsar.client.api.Message; import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.functions.FunctionDefinition; import org.apache.pulsar.common.functions.FunctionState; @@ -486,7 +485,7 @@ public List listFunctions( @POST @ApiOperation( value = "Triggers a Pulsar Function with a user-specified value or file data", - response = Message.class + response = String.class ) @ApiResponses(value = { @ApiResponse(code = 400, message = "Invalid request"), @@ -541,6 +540,7 @@ public FunctionState getFunctionState( value = "Put the state associated with a Pulsar Function" ) @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @@ -557,8 +557,9 @@ public void putFunctionState(final @PathParam("tenant") String tenant, } @POST - @ApiOperation(value = "Restart an instance of a Pulsar Function", response = Void.class) + @ApiOperation(value = "Restart an instance of a Pulsar Function") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this function"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @@ -578,8 +579,9 @@ public void restartFunction( } @POST - @ApiOperation(value = "Restart all instances of a Pulsar Function", response = Void.class) + @ApiOperation(value = "Restart all instances of a Pulsar Function") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @ApiResponse(code = 500, message = "Internal server error") @@ -597,8 +599,9 @@ public void restartFunction( } @POST - @ApiOperation(value = "Stop an instance of a Pulsar Function", response = Void.class) + @ApiOperation(value = "Stop an instance of a Pulsar Function") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @ApiResponse(code = 500, message = "Internal server error") @@ -617,8 +620,9 @@ public void stopFunction( } @POST - @ApiOperation(value = "Stop all instances of a Pulsar Function", response = Void.class) + @ApiOperation(value = "Stop all instances of a Pulsar Function") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @ApiResponse(code = 500, message = "Internal server error") @@ -636,8 +640,9 @@ public void stopFunction( } @POST - @ApiOperation(value = "Start an instance of a Pulsar Function", response = Void.class) + @ApiOperation(value = "Start an instance of a Pulsar Function") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @ApiResponse(code = 500, message = "Internal server error") @@ -656,8 +661,9 @@ public void startFunction( } @POST - @ApiOperation(value = "Start all instances of a Pulsar Function", response = Void.class) + @ApiOperation(value = "Start all instances of a Pulsar Function") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 404, message = "The Pulsar Function does not exist"), @ApiResponse(code = 500, message = "Internal server error") @@ -718,7 +724,8 @@ public StreamingOutput downloadFunction( @GET @ApiOperation( value = "Fetches a list of supported Pulsar IO connectors currently running in cluster mode", - response = List.class + response = ConnectorDefinition.class, + responseContainer = "List" ) @ApiResponses(value = { @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @@ -739,6 +746,7 @@ public List getConnectorsList() throws IOException { value = "Reload the built-in Functions" ) @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 401, message = "This operation requires super-user access"), @ApiResponse(code = 503, message = "Function worker service is now initializing. Please try again later."), @ApiResponse(code = 500, message = "Internal server error") @@ -768,6 +776,7 @@ public List getBuiltinFunction() { @PUT @ApiOperation(value = "Updates a Pulsar Function on the worker leader", hidden = true) @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 403, message = "The requester doesn't have super-user permissions"), @ApiResponse(code = 404, message = "The function does not exist"), @ApiResponse(code = 400, message = "Invalid request"), diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SinksBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SinksBase.java index 80ad72d6f9aa9..0a76fe27e0a35 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SinksBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SinksBase.java @@ -389,8 +389,9 @@ public List listSinks(@ApiParam(value = "The tenant of a Pulsar Sink") } @POST - @ApiOperation(value = "Restart an instance of a Pulsar Sink", response = Void.class) + @ApiOperation(value = "Restart an instance of a Pulsar Sink") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this sink"), @ApiResponse(code = 400, message = "Invalid restart request"), @ApiResponse(code = 401, message = "The client is not authorized to perform this operation"), @@ -415,8 +416,9 @@ public void restartSink(@ApiParam(value = "The tenant of a Pulsar Sink") } @POST - @ApiOperation(value = "Restart all instances of a Pulsar Sink", response = Void.class) + @ApiOperation(value = "Restart all instances of a Pulsar Sink") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid restart request"), @ApiResponse(code = 401, message = "The client is not authorized to perform this operation"), @ApiResponse(code = 404, message = "The Pulsar Sink does not exist"), @@ -436,8 +438,9 @@ public void restartSink(@ApiParam(value = "The tenant of a Pulsar Sink") } @POST - @ApiOperation(value = "Stop an instance of a Pulsar Sink", response = Void.class) + @ApiOperation(value = "Stop an instance of a Pulsar Sink") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid stop request"), @ApiResponse(code = 404, message = "The Pulsar Sink instance does not exist"), @ApiResponse(code = 500, message = @@ -460,8 +463,9 @@ public void stopSink(@ApiParam(value = "The tenant of a Pulsar Sink") } @POST - @ApiOperation(value = "Stop all instances of a Pulsar Sink", response = Void.class) + @ApiOperation(value = "Stop all instances of a Pulsar Sink") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid stop request"), @ApiResponse(code = 404, message = "The Pulsar Sink does not exist"), @ApiResponse(code = 500, message = @@ -481,8 +485,9 @@ public void stopSink(@ApiParam(value = "The tenant of a Pulsar Sink") } @POST - @ApiOperation(value = "Start an instance of a Pulsar Sink", response = Void.class) + @ApiOperation(value = "Start an instance of a Pulsar Sink") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid start request"), @ApiResponse(code = 404, message = "The Pulsar Sink does not exist"), @ApiResponse(code = 500, message = @@ -505,8 +510,9 @@ public void startSink(@ApiParam(value = "The tenant of a Pulsar Sink") } @POST - @ApiOperation(value = "Start all instances of a Pulsar Sink", response = Void.class) + @ApiOperation(value = "Start all instances of a Pulsar Sink") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid start request"), @ApiResponse(code = 404, message = "The Pulsar Sink does not exist"), @ApiResponse(code = 500, message = diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SourcesBase.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SourcesBase.java index 4af0afc0d6ec5..0d037dd42362f 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SourcesBase.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/SourcesBase.java @@ -323,7 +323,7 @@ public SourceStatus getSourceStatus( @ApiOperation( value = "Lists all Pulsar Sources currently deployed in a given namespace", response = String.class, - responseContainer = "Collection" + responseContainer = "List" ) @ApiResponses(value = { @ApiResponse(code = 400, message = "Invalid request"), @@ -342,8 +342,9 @@ public List listSources( } @POST - @ApiOperation(value = "Restart an instance of a Pulsar Source", response = Void.class) + @ApiOperation(value = "Restart an instance of a Pulsar Source") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace of this source"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 401, message = "Client is not authorized to perform operation"), @@ -365,8 +366,9 @@ public void restartSource( } @POST - @ApiOperation(value = "Restart all instances of a Pulsar Source", response = Void.class) + @ApiOperation(value = "Restart all instances of a Pulsar Source") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 401, message = "Client is not authorized to perform operation"), @ApiResponse(code = 404, message = "Not Found(The Pulsar Source doesn't exist)"), @@ -386,8 +388,9 @@ public void restartSource( } @POST - @ApiOperation(value = "Stop instance of a Pulsar Source", response = Void.class) + @ApiOperation(value = "Stop instance of a Pulsar Source") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 401, message = "Client is not authorized to perform operation"), @ApiResponse(code = 404, message = "Not Found(The Pulsar Source doesn't exist)"), @@ -407,8 +410,9 @@ public void stopSource( } @POST - @ApiOperation(value = "Stop all instances of a Pulsar Source", response = Void.class) + @ApiOperation(value = "Stop all instances of a Pulsar Source") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 401, message = "Client is not authorized to perform operation"), @ApiResponse(code = 404, message = "Not Found(The Pulsar Source doesn't exist)"), @@ -428,8 +432,9 @@ public void stopSource( } @POST - @ApiOperation(value = "Start an instance of a Pulsar Source", response = Void.class) + @ApiOperation(value = "Start an instance of a Pulsar Source") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 401, message = "Client is not authorized to perform operation"), @ApiResponse(code = 404, message = "Not Found(The Pulsar Source doesn't exist)"), @@ -449,8 +454,9 @@ public void startSource( } @POST - @ApiOperation(value = "Start all instances of a Pulsar Source", response = Void.class) + @ApiOperation(value = "Start all instances of a Pulsar Source") @ApiResponses(value = { + @ApiResponse(code = 200, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 401, message = "Client is not authorized to perform operation"), @ApiResponse(code = 404, message = "Not Found(The Pulsar Source doesn't exist)"), 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 b93f3e3c6ebcc..cb60414edeb48 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 @@ -103,7 +103,9 @@ public void getTenantAdmin(@Suspended final AsyncResponse asyncResponse, @PUT @Path("/{tenant}") @ApiOperation(value = "Create a new tenant.", notes = "This operation requires Pulsar super-user privileges.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 409, message = "Tenant already exists"), @ApiResponse(code = 412, message = "Tenant name is not valid"), @ApiResponse(code = 412, message = "Clusters can not be empty"), @@ -155,7 +157,9 @@ public void createTenant(@Suspended final AsyncResponse asyncResponse, @Path("/{tenant}") @ApiOperation(value = "Update the admins for a tenant.", notes = "This operation requires Pulsar super-user privileges.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 404, message = "Tenant does not exist"), @ApiResponse(code = 409, message = "Tenant already exists"), @ApiResponse(code = 412, message = "Clusters can not be empty"), @@ -190,7 +194,9 @@ public void updateTenant(@Suspended final AsyncResponse asyncResponse, @DELETE @Path("/{tenant}") @ApiOperation(value = "Delete a tenant and all namespaces and topics under it.") - @ApiResponses(value = {@ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 404, message = "Tenant does not exist"), @ApiResponse(code = 405, message = "Broker doesn't allow forced deletion of tenants"), @ApiResponse(code = 409, message = "The tenant still has active namespaces")}) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceGroups.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceGroups.java index 52fd03b18ed0b..58f593e20ce3b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceGroups.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceGroups.java @@ -60,7 +60,9 @@ public ResourceGroup getResourceGroup(@PathParam("resourcegroup") String resourc @PUT @Path("/{resourcegroup}") @ApiOperation(value = "Creates a new resourcegroup with the specified rate limiters") - @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), + @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "cluster doesn't exist")}) public void createOrUpdateResourceGroup(@PathParam("resourcegroup") String name, @ApiParam(value = "Rate limiters for the resourcegroup") @@ -72,6 +74,7 @@ public void createOrUpdateResourceGroup(@PathParam("resourcegroup") String name, @Path("/{resourcegroup}") @ApiOperation(value = "Delete a resourcegroup.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "ResourceGroup doesn't exist"), @ApiResponse(code = 409, message = "ResourceGroup is in use")}) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceQuotas.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceQuotas.java index 58ccc1c10288c..d2884e8ea6f7e 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceQuotas.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/ResourceQuotas.java @@ -75,7 +75,7 @@ public void setDefaultResourceQuota( @GET @Path("/{tenant}/{namespace}/{bundle}") - @ApiOperation(value = "Get resource quota of a namespace bundle.") + @ApiOperation(value = "Get resource quota of a namespace bundle.", response = ResourceQuota.class) @ApiResponses(value = { @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @@ -103,6 +103,7 @@ public void getNamespaceBundleResourceQuota( @Path("/{tenant}/{namespace}/{bundle}") @ApiOperation(value = "Set resource quota on a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 409, message = "Concurrent modification") }) @@ -133,6 +134,7 @@ public void setNamespaceBundleResourceQuota( @Path("/{tenant}/{namespace}/{bundle}") @ApiOperation(value = "Remove resource quota for a namespace.") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 307, message = "Current broker doesn't serve the namespace"), @ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 409, message = "Concurrent modification") }) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Worker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Worker.java index 3813790e4f428..7178b565719ca 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Worker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/Worker.java @@ -87,7 +87,9 @@ public WorkerInfo getClusterLeader() { @GET @ApiOperation( value = "Fetches information about which Pulsar Functions are assigned to which Pulsar clusters", - response = Map.class + response = Map.class, + notes = "Returns a nested map structure which Swagger does not fully support for display." + + "Structure: Map>. Please refer to this structure for details." ) @ApiResponses(value = { @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @@ -102,7 +104,8 @@ public Map> getAssignments() { @GET @ApiOperation( value = "Fetches a list of supported Pulsar IO connectors currently running in cluster mode", - response = List.class + response = ConnectorDefinition.class, + responseContainer = "List" ) @ApiResponses(value = { @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @@ -120,6 +123,7 @@ public List getConnectorsList() throws IOException { value = "Triggers a rebalance of functions to workers" ) @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 408, message = "Request timeout") @@ -134,6 +138,7 @@ public void rebalance() { value = "Drains the specified worker, i.e., moves its work-assignments to other workers" ) @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 408, message = "Request timeout"), @@ -150,6 +155,7 @@ public void drainAtLeader(@QueryParam("workerId") String workerId) { value = "Drains this worker, i.e., moves its work-assignments to other workers" ) @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 400, message = "Invalid request"), @ApiResponse(code = 403, message = "The requester doesn't have admin permissions"), @ApiResponse(code = 408, message = "Request timeout"), diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Packages.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Packages.java index 15e7b69554dc7..4ca7e3948ff5a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Packages.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Packages.java @@ -80,7 +80,7 @@ public void getMeta( ) @ApiResponses( value = { - @ApiResponse(code = 200, message = "Update the metadata of the specified package successfully."), + @ApiResponse(code = 204, message = "Update the metadata of the specified package successfully."), @ApiResponse(code = 404, message = "The specified package is not existent."), @ApiResponse(code = 412, message = "The package name is illegal."), @ApiResponse(code = 500, message = "Internal server error."), @@ -113,7 +113,7 @@ public void updateMeta( ) @ApiResponses( value = { - @ApiResponse(code = 200, message = "Upload the specified package successfully."), + @ApiResponse(code = 204, message = "Upload the specified package successfully."), @ApiResponse(code = 412, message = "The package name is illegal."), @ApiResponse(code = 500, message = "Internal server error."), @ApiResponse(code = 503, message = "Package Management Service is not enabled in the broker.") @@ -169,7 +169,7 @@ public StreamingOutput download( @Path("/{type}/{tenant}/{namespace}/{packageName}/{version}") @ApiResponses( value = { - @ApiResponse(code = 200, message = "Delete the specified package successfully."), + @ApiResponse(code = 204, message = "Delete the specified package successfully."), @ApiResponse(code = 404, message = "The specified package is not existent."), @ApiResponse(code = 412, message = "The package name is illegal."), @ApiResponse(code = 500, message = "Internal server error."), @@ -218,7 +218,8 @@ public void listPackageVersion( @Path("/{type}/{tenant}/{namespace}") @ApiOperation( value = "Get all the specified type packages in a namespace.", - response = PackageMetadata.class + response = PackageMetadata.class, + responseContainer = "List" ) @ApiResponses( value = { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java index b2a3ae8cdb935..fdfbacb65d121 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v3/Transactions.java @@ -43,6 +43,16 @@ import org.apache.pulsar.broker.admin.impl.TransactionsBase; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.common.policies.data.TransactionBufferStats; +import org.apache.pulsar.common.policies.data.TransactionCoordinatorInfo; +import org.apache.pulsar.common.policies.data.TransactionCoordinatorInternalStats; +import org.apache.pulsar.common.policies.data.TransactionCoordinatorStats; +import org.apache.pulsar.common.policies.data.TransactionInBufferStats; +import org.apache.pulsar.common.policies.data.TransactionInPendingAckStats; +import org.apache.pulsar.common.policies.data.TransactionMetadata; +import org.apache.pulsar.common.policies.data.TransactionPendingAckInternalStats; +import org.apache.pulsar.common.policies.data.TransactionPendingAckStats; +import org.apache.pulsar.common.stats.PositionInPendingAckStats; import org.apache.pulsar.common.util.FutureUtil; import org.jetbrains.annotations.Nullable; @@ -55,7 +65,8 @@ public class Transactions extends TransactionsBase { @GET @Path("/coordinators") - @ApiOperation(value = "List transaction coordinators.") + @ApiOperation(value = "List transaction coordinators.", + response = TransactionCoordinatorInfo.class, responseContainer = "List") @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 503, message = "This Broker is not " + "configured with transactionCoordinatorEnabled=true.")}) @@ -66,7 +77,7 @@ public void listCoordinators(@Suspended final AsyncResponse asyncResponse) { @GET @Path("/coordinatorStats") - @ApiOperation(value = "Get transaction coordinator stats.") + @ApiOperation(value = "Get transaction coordinator stats.", response = TransactionCoordinatorStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 503, message = "This Broker is not " + "configured with transactionCoordinatorEnabled=true."), @@ -82,7 +93,7 @@ public void getCoordinatorStats(@Suspended final AsyncResponse asyncResponse, @GET @Path("/transactionInBufferStats/{tenant}/{namespace}/{topic}/{mostSigBits}/{leastSigBits}") - @ApiOperation(value = "Get transaction state in transaction buffer.") + @ApiOperation(value = "Get transaction state in transaction buffer.", response = TransactionInBufferStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), @ApiResponse(code = 503, message = "This Broker is not configured " @@ -119,7 +130,7 @@ public void getTransactionInBufferStats(@Suspended final AsyncResponse asyncResp @GET @Path("/transactionInPendingAckStats/{tenant}/{namespace}/{topic}/{subName}/{mostSigBits}/{leastSigBits}") - @ApiOperation(value = "Get transaction state in pending ack.") + @ApiOperation(value = "Get transaction state in pending ack.", response = TransactionInPendingAckStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), @ApiResponse(code = 503, message = "This Broker is not configured " @@ -157,7 +168,7 @@ public void getTransactionInPendingAckStats(@Suspended final AsyncResponse async @GET @Path("/transactionBufferStats/{tenant}/{namespace}/{topic}") - @ApiOperation(value = "Get transaction buffer stats in topic.") + @ApiOperation(value = "Get transaction buffer stats in topic.", response = TransactionBufferStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic doesn't exist"), @ApiResponse(code = 503, message = "This Broker is not configured " @@ -195,7 +206,7 @@ public void getTransactionBufferStats(@Suspended final AsyncResponse asyncRespon @GET @Path("/pendingAckStats/{tenant}/{namespace}/{topic}/{subName}") - @ApiOperation(value = "Get transaction pending ack stats in topic.") + @ApiOperation(value = "Get transaction pending ack stats in topic.", response = TransactionPendingAckStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic or subName doesn't exist"), @ApiResponse(code = 503, message = "This Broker is not configured " @@ -231,7 +242,7 @@ public void getPendingAckStats(@Suspended final AsyncResponse asyncResponse, @GET @Path("/transactionMetadata/{mostSigBits}/{leastSigBits}") - @ApiOperation(value = "Get transaction metadata") + @ApiOperation(value = "Get transaction metadata", response = TransactionMetadata.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic " + "or coordinator or transaction doesn't exist"), @@ -252,7 +263,7 @@ public void getTransactionMetadata(@Suspended final AsyncResponse asyncResponse, @GET @Path("/slowTransactions/{timeout}") - @ApiOperation(value = "Get slow transactions.") + @ApiOperation(value = "Get slow transactions.", response = TransactionMetadata.class, responseContainer = "Map") @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic " + "or coordinator or transaction doesn't exist"), @@ -272,7 +283,7 @@ public void getSlowTransactions(@Suspended final AsyncResponse asyncResponse, @GET @Path("/coordinatorInternalStats/{coordinatorId}") - @ApiOperation(value = "Get coordinator internal stats.") + @ApiOperation(value = "Get coordinator internal stats.", response = TransactionCoordinatorInternalStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 503, message = "This Broker is not " + "configured with transactionCoordinatorEnabled=true."), @@ -290,7 +301,8 @@ public void getCoordinatorInternalStats(@Suspended final AsyncResponse asyncResp @GET @Path("/pendingAckInternalStats/{tenant}/{namespace}/{topic}/{subName}") - @ApiOperation(value = "Get transaction pending ack internal stats.") + @ApiOperation(value = "Get transaction pending ack internal stats.", + response = TransactionPendingAckInternalStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic " + "or subscription name doesn't exist"), @@ -379,6 +391,7 @@ public void getTransactionBufferInternalStats(@Suspended final AsyncResponse asy @POST @Path("/transactionCoordinator/replicas") @ApiResponses(value = { + @ApiResponse(code = 204, message = "Operation successful"), @ApiResponse(code = 503, message = "This Broker is not configured " + "with transactionCoordinatorEnabled=true."), @ApiResponse(code = 406, message = "The number of replicas should be more than " @@ -401,7 +414,7 @@ public void scaleTransactionCoordinators(@Suspended final AsyncResponse asyncRes @GET @Path("/positionStatsInPendingAck/{tenant}/{namespace}/{topic}/{subName}/{ledgerId}/{entryId}") - @ApiOperation(value = "Get position stats in pending ack.") + @ApiOperation(value = "Get position stats in pending ack.", response = PositionInPendingAckStats.class) @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), @ApiResponse(code = 404, message = "Tenant or cluster or namespace or topic " + "or subscription name doesn't exist"), From f0c5776e9c4360a5bc863bf4ac162fe678ce3315 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 28 May 2024 11:13:12 +0800 Subject: [PATCH 523/699] [fix] [broker] fix topic partitions was expanded even if disabled topic level replication (#22769) (cherry picked from commit 55ad4b22ba2e94029c2e1c01b67b22cb237e5ecc) (cherry picked from commit 96e2bda9c604f997acba97f5aabbd9faef19835c) --- .../admin/impl/PersistentTopicsBase.java | 11 +++- .../broker/service/OneWayReplicatorTest.java | 65 ++++++++++++++++++- .../service/OneWayReplicatorTestBase.java | 13 +++- .../OneWayReplicatorUsingGlobalZKTest.java | 10 +++ 4 files changed, 93 insertions(+), 6 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 d64b6c6e37de0..0ae3ab95176bc 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 @@ -35,6 +35,7 @@ import java.util.Base64; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -463,7 +464,14 @@ protected CompletableFuture internalCreateNonPartitionedTopicAsync(boolean if (!policies.isPresent()) { return CompletableFuture.completedFuture(null); } - final Set replicationClusters = policies.get().replication_clusters; + // Combine namespace level policies and topic level policies. + Set replicationClusters = policies.get().replication_clusters; + TopicPolicies topicPolicies = + pulsarService.getTopicPoliciesService().getTopicPoliciesIfExists(topicName); + if (topicPolicies != null) { + replicationClusters = new HashSet<>(topicPolicies.getReplicationClusters()); + } + // Do check replicated clusters. if (replicationClusters.size() == 0) { return CompletableFuture.completedFuture(null); } @@ -479,6 +487,7 @@ protected CompletableFuture internalCreateNonPartitionedTopicAsync(boolean // The replication clusters just has the current cluster itself. return CompletableFuture.completedFuture(null); } + // Do sync operation to other clusters. List> futures = replicationClusters.stream() .map(replicationCluster -> admin.clusters().getClusterAsync(replicationCluster) .thenCompose(clusterData -> pulsarService.getBrokerService() 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 cc2666af3c13f..e6e45eebf8d24 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 @@ -466,8 +466,17 @@ public void testPartitionedTopicLevelReplicationRemoteTopicExist() throws Except admin2.topics().createPartitionedTopic(topicName, 2); admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1, cluster2)); // Check the partitioned topic has been created at the remote cluster. - PartitionedTopicMetadata topicMetadata2 = admin2.topics().getPartitionedTopicMetadata(topicName); - assertEquals(topicMetadata2.partitions, 2); + Awaitility.await().untilAsserted(() -> { + PartitionedTopicMetadata topicMetadata2 = admin2.topics().getPartitionedTopicMetadata(topicName); + assertEquals(topicMetadata2.partitions, 2); + }); + + // Expand partitions + admin2.topics().updatePartitionedTopic(topicName, 3); + Awaitility.await().untilAsserted(() -> { + PartitionedTopicMetadata topicMetadata2 = admin2.topics().getPartitionedTopicMetadata(topicName); + assertEquals(topicMetadata2.partitions, 3); + }); // cleanup. admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1)); waitReplicatorStopped(partition0); @@ -717,4 +726,56 @@ public void testDeletePartitionedTopic() throws Exception { .persistentTopicExists(TopicName.get(topicName).getPartition(1)).join()); } } + + @Test + public void testNoExpandTopicPartitionsWhenDisableTopicLevelReplication() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); + admin1.topics().createPartitionedTopic(topicName, 2); + + // Verify replicator works. + verifyReplicationWorks(topicName); + + // Disable topic level replication. + setTopicLevelClusters(topicName, Arrays.asList(cluster1), admin1, pulsar1); + setTopicLevelClusters(topicName, Arrays.asList(cluster2), admin2, pulsar2); + + // Expand topic. + admin1.topics().updatePartitionedTopic(topicName, 3); + assertEquals(admin1.topics().getPartitionedTopicMetadata(topicName).partitions, 3); + + // Wait for async tasks that were triggered by expanding topic partitions. + Thread.sleep(3 * 1000); + + + // Verify: the topics on the remote cluster did not been expanded. + assertEquals(admin2.topics().getPartitionedTopicMetadata(topicName).partitions, 2); + + cleanupTopics(() -> { + admin1.topics().deletePartitionedTopic(topicName, false); + admin2.topics().deletePartitionedTopic(topicName, false); + }); + } + + @Test + public void testExpandTopicPartitionsOnNamespaceLevelReplication() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); + admin1.topics().createPartitionedTopic(topicName, 2); + + // Verify replicator works. + verifyReplicationWorks(topicName); + + // Expand topic. + admin1.topics().updatePartitionedTopic(topicName, 3); + assertEquals(admin1.topics().getPartitionedTopicMetadata(topicName).partitions, 3); + + // Verify: the topics on the remote cluster will be expanded. + Awaitility.await().untilAsserted(() -> { + assertEquals(admin2.topics().getPartitionedTopicMetadata(topicName).partitions, 3); + }); + + cleanupTopics(() -> { + admin1.topics().deletePartitionedTopic(topicName, false); + admin2.topics().deletePartitionedTopic(topicName, false); + }); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java index 6a84432890cb5..7372b2e478475 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java @@ -196,9 +196,16 @@ protected void cleanupTopics(String namespace, CleanupTopicAction cleanupTopicAc } protected void waitChangeEventsInit(String namespace) { - PersistentTopic topic = (PersistentTopic) pulsar1.getBrokerService() - .getTopic(namespace + "/" + SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME, false) - .join().get(); + CompletableFuture> future = pulsar1.getBrokerService() + .getTopic(namespace + "/" + SystemTopicNames.NAMESPACE_EVENTS_LOCAL_NAME, false); + if (future == null) { + return; + } + Optional optional = future.join(); + if (!optional.isPresent()) { + return; + } + PersistentTopic topic = (PersistentTopic) optional.get(); Awaitility.await().atMost(Duration.ofSeconds(180)).untilAsserted(() -> { TopicStatsImpl topicStats = topic.getStats(true, false, false); topicStats.getSubscriptions().entrySet().forEach(entry -> { 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 d827235bc326d..b4747a8bd0e47 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 @@ -94,4 +94,14 @@ public void testDeleteNonPartitionedTopic() throws Exception { public void testDeletePartitionedTopic() throws Exception { super.testDeletePartitionedTopic(); } + + @Test(enabled = false) + public void testNoExpandTopicPartitionsWhenDisableTopicLevelReplication() throws Exception { + super.testNoExpandTopicPartitionsWhenDisableTopicLevelReplication(); + } + + @Test(enabled = false) + public void testExpandTopicPartitionsOnNamespaceLevelReplication() throws Exception { + super.testExpandTopicPartitionsOnNamespaceLevelReplication(); + } } From b2ccc5f9f01bb26545e4521b628ceadba88cbfe3 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Tue, 28 May 2024 20:29:35 +0800 Subject: [PATCH 524/699] [improve][cli][branch-3.0] PIP-353: Improve transaction message visibility for peek-message (#22788) (cherry picked from commit 3fd59d28c09a270c1109909a7b6de5e071d12627) --- .../admin/impl/PersistentTopicsBase.java | 125 +++++++++-------- .../admin/v3/AdminApiTransactionTest.java | 126 ++++++++++++++++++ .../apache/pulsar/client/admin/Topics.java | 60 ++++++++- .../client/admin/internal/TopicsImpl.java | 70 ++++++++-- .../client/api/TransactionIsolationLevel.java | 31 +++++ .../pulsar/admin/cli/PulsarAdminToolTest.java | 4 +- .../apache/pulsar/admin/cli/CmdTopics.java | 22 ++- 7 files changed, 364 insertions(+), 74 deletions(-) create mode 100644 pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TransactionIsolationLevel.java 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 0ae3ab95176bc..f20897634d685 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 @@ -98,6 +98,7 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; @@ -2854,7 +2855,7 @@ public void readEntryFailed(ManagedLedgerException exception, @Override public void readEntryComplete(Entry entry, Object ctx) { try { - results.complete(generateResponseWithEntry(entry)); + results.complete(generateResponseWithEntry(entry, (PersistentTopic) topic)); } catch (IOException exception) { throw new RestException(exception); } finally { @@ -2968,57 +2969,59 @@ private CompletableFuture findMessageIdByPublishTime(long timestamp, protected CompletableFuture internalPeekNthMessageAsync(String subName, int messagePosition, boolean authoritative) { - CompletableFuture ret; - // If the topic name is a partition name, no need to get partition topic metadata again - if (!topicName.isPartitioned()) { - ret = getPartitionedTopicMetadataAsync(topicName, authoritative, false) - .thenCompose(topicMetadata -> { - if (topicMetadata.partitions > 0) { - throw new RestException(Status.METHOD_NOT_ALLOWED, - "Peek messages on a partitioned topic is not allowed"); - } - return CompletableFuture.completedFuture(null); - }); - } else { - ret = CompletableFuture.completedFuture(null); - } - return ret.thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) - .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.PEEK_MESSAGES, subName)) - .thenCompose(__ -> getTopicReferenceAsync(topicName)) - .thenCompose(topic -> { - CompletableFuture entry; - if (!(topic instanceof PersistentTopic)) { - log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), - topicName, subName); - throw new RestException(Status.METHOD_NOT_ALLOWED, - "Peek messages on a non-persistent topic is not allowed"); - } else { - if (subName.startsWith(((PersistentTopic) topic).getReplicatorPrefix())) { - PersistentReplicator repl = getReplicatorReference(subName, (PersistentTopic) topic); - entry = repl.peekNthMessage(messagePosition); - } else { - PersistentSubscription sub = - (PersistentSubscription) getSubscriptionReference(subName, (PersistentTopic) topic); - entry = sub.peekNthMessage(messagePosition); - } - } - return entry; - }).thenCompose(entry -> { - try { - Response response = generateResponseWithEntry(entry); - return CompletableFuture.completedFuture(response); - } catch (NullPointerException npe) { - throw new RestException(Status.NOT_FOUND, "Message not found"); - } catch (Exception exception) { - log.error("[{}] Failed to peek message at position {} from {} {}", clientAppId(), - messagePosition, topicName, subName, exception); - throw new RestException(exception); - } finally { - if (entry != null) { - entry.release(); - } - } - }); + CompletableFuture ret = validateTopicOperationAsync(topicName, TopicOperation.PEEK_MESSAGES, subName); + return ret.thenCompose(__ -> { + // If the topic name is a partition name, no need to get partition topic metadata again + if (!topicName.isPartitioned()) { + return getPartitionedTopicMetadataAsync(topicName, authoritative, false) + .thenCompose(topicMetadata -> { + if (topicMetadata.partitions > 0) { + throw new RestException(Status.METHOD_NOT_ALLOWED, + "Peek messages on a partitioned topic is not allowed"); + } + return CompletableFuture.completedFuture(null); + }); + } else { + return CompletableFuture.completedFuture(null); + } + }).thenCompose(__ -> validateTopicOwnershipAsync(topicName, authoritative)) + .thenCompose(__ -> getTopicReferenceAsync(topicName)) + .thenCompose(topic -> { + CompletableFuture entry; + if (!(topic instanceof PersistentTopic)) { + log.error("[{}] Not supported operation of non-persistent topic {} {}", clientAppId(), + topicName, subName); + throw new RestException(Status.METHOD_NOT_ALLOWED, + "Peek messages on a non-persistent topic is not allowed"); + } else { + if (subName.startsWith(((PersistentTopic) topic).getReplicatorPrefix())) { + PersistentReplicator repl = getReplicatorReference(subName, (PersistentTopic) topic); + entry = repl.peekNthMessage(messagePosition); + } else { + PersistentSubscription sub = + (PersistentSubscription) getSubscriptionReference(subName, (PersistentTopic) topic); + entry = sub.peekNthMessage(messagePosition); + } + } + return entry.thenApply(e -> Pair.of(e, (PersistentTopic) topic)); + }).thenCompose(entryTopicPair -> { + Entry entry = entryTopicPair.getLeft(); + PersistentTopic persistentTopic = entryTopicPair.getRight(); + try { + Response response = generateResponseWithEntry(entry, persistentTopic); + return CompletableFuture.completedFuture(response); + } catch (NullPointerException npe) { + throw new RestException(Status.NOT_FOUND, "Message not found"); + } catch (Exception exception) { + log.error("[{}] Failed to peek message at position {} from {} {}", clientAppId(), + messagePosition, topicName, subName, exception); + throw new RestException(exception); + } finally { + if (entry != null) { + entry.release(); + } + } + }); } protected CompletableFuture internalExamineMessageAsync(String initialPosition, long messagePosition, @@ -3082,7 +3085,7 @@ public String toString() { PersistentTopicsBase.this.topicName); } }, null); - return future; + return future.thenApply(entry -> Pair.of(entry, (PersistentTopic) topic)); } catch (ManagedLedgerException exception) { log.error("[{}] Failed to examine message at position {} from {} due to {}", clientAppId(), messagePosition, @@ -3090,9 +3093,11 @@ public String toString() { throw new RestException(exception); } - }).thenApply(entry -> { + }).thenApply(entryTopicPair -> { + Entry entry = entryTopicPair.getLeft(); + PersistentTopic persistentTopic = entryTopicPair.getRight(); try { - return generateResponseWithEntry(entry); + return generateResponseWithEntry(entry, persistentTopic); } catch (IOException exception) { throw new RestException(exception); } finally { @@ -3103,7 +3108,7 @@ public String toString() { }); } - private Response generateResponseWithEntry(Entry entry) throws IOException { + private Response generateResponseWithEntry(Entry entry, PersistentTopic persistentTopic) throws IOException { checkNotNull(entry); PositionImpl pos = (PositionImpl) entry.getPosition(); ByteBuf metadataAndPayload = entry.getDataBuffer(); @@ -3221,6 +3226,14 @@ private Response generateResponseWithEntry(Entry entry) throws IOException { if (metadata.hasNullPartitionKey()) { responseBuilder.header("X-Pulsar-null-partition-key", metadata.isNullPartitionKey()); } + if (metadata.hasTxnidMostBits() && metadata.hasTxnidLeastBits()) { + TxnID txnID = new TxnID(metadata.getTxnidMostBits(), metadata.getTxnidLeastBits()); + boolean isTxnAborted = persistentTopic.isTxnAborted(txnID, (PositionImpl) entry.getPosition()); + responseBuilder.header("X-Pulsar-txn-aborted", isTxnAborted); + } + boolean isTxnUncommitted = ((PositionImpl) entry.getPosition()) + .compareTo(persistentTopic.getMaxReadPosition()) > 0; + responseBuilder.header("X-Pulsar-txn-uncommitted", isTxnUncommitted); // Decode if needed CompressionCodec codec = CompressionCodecProvider.getCompressionCodec(metadata.getCompression()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java index 049fd0f5f4400..f6429251690e7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/v3/AdminApiTransactionTest.java @@ -38,6 +38,7 @@ import lombok.Cleanup; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.http.HttpStatus; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.transaction.buffer.AbortedTxnProcessor; @@ -48,12 +49,16 @@ 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.TransactionIsolationLevel; 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.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.client.impl.transaction.TransactionImpl; +import org.apache.pulsar.common.api.proto.MarkerType; +import org.apache.pulsar.common.api.proto.MessageMetadata; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.SystemTopicNames; import org.apache.pulsar.common.naming.TopicDomain; @@ -894,6 +899,127 @@ public void testGetPositionStatsInPendingAckStatsFroBatch() throws Exception { } + @Test + public void testPeekMessageForSkipTxnMarker() throws Exception { + initTransaction(1); + + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/peek_marker"); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + int n = 10; + for (int i = 0; i < n; i++) { + Transaction txn = pulsarClient.newTransaction().build().get(); + producer.newMessage(txn).value("msg").send(); + txn.commit().get(); + } + + List> peekMsgs = admin.topics().peekMessages(topic, "t-sub", n, + false, TransactionIsolationLevel.READ_UNCOMMITTED); + assertEquals(peekMsgs.size(), n); + for (Message peekMsg : peekMsgs) { + assertEquals(new String(peekMsg.getValue()), "msg"); + } + } + + @Test + public void testPeekMessageFoReadCommittedMessages() throws Exception { + initTransaction(1); + + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/peek_txn"); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + int n = 10; + // Alternately sends `n` committed transactional messages and `n` abort transactional messages. + for (int i = 0; i < 2 * n; i++) { + Transaction txn = pulsarClient.newTransaction().build().get(); + if (i % 2 == 0) { + producer.newMessage(txn).value("msg").send(); + txn.commit().get(); + } else { + producer.newMessage(txn).value("msg-aborted").send(); + txn.abort(); + } + } + // Then sends 1 uncommitted transactional messages. + Transaction txn = pulsarClient.newTransaction().build().get(); + producer.newMessage(txn).value("msg-uncommitted").send(); + // Then sends n-1 no transaction messages. + for (int i = 0; i < n - 1; i++) { + producer.newMessage().value("msg-after-uncommitted").send(); + } + + // peek n message, all messages value should be "msg" + { + List> peekMsgs = admin.topics().peekMessages(topic, "t-sub", n, + false, TransactionIsolationLevel.READ_COMMITTED); + assertEquals(peekMsgs.size(), n); + for (Message peekMsg : peekMsgs) { + assertEquals(new String(peekMsg.getValue()), "msg"); + } + } + + // peek 3 * n message, and still get n message, all messages value should be "msg" + { + List> peekMsgs = admin.topics().peekMessages(topic, "t-sub", 2 * n, + false, TransactionIsolationLevel.READ_COMMITTED); + assertEquals(peekMsgs.size(), n); + for (Message peekMsg : peekMsgs) { + assertEquals(new String(peekMsg.getValue()), "msg"); + } + } + } + + @Test + public void testPeekMessageForShowAllMessages() throws Exception { + initTransaction(1); + + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/peek_all"); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + int n = 10; + // Alternately sends `n` committed transactional messages and `n` abort transactional messages. + for (int i = 0; i < 2 * n; i++) { + Transaction txn = pulsarClient.newTransaction().build().get(); + if (i % 2 == 0) { + producer.newMessage(txn).value("msg").send(); + txn.commit().get(); + } else { + producer.newMessage(txn).value("msg-aborted").send(); + txn.abort(); + } + } + // Then sends `n` uncommitted transactional messages. + Transaction txn = pulsarClient.newTransaction().build().get(); + for (int i = 0; i < n; i++) { + producer.newMessage(txn).value("msg-uncommitted").send(); + } + + // peek 5 * n message, will get 5 * n msg. + List> peekMsgs = admin.topics().peekMessages(topic, "t-sub", 5 * n, + true, TransactionIsolationLevel.READ_UNCOMMITTED); + assertEquals(peekMsgs.size(), 5 * n); + + for (int i = 0; i < 4 * n; i++) { + Message peekMsg = peekMsgs.get(i); + MessageImpl peekMsgImpl = (MessageImpl) peekMsg; + MessageMetadata metadata = peekMsgImpl.getMessageBuilder(); + if (metadata.hasMarkerType()) { + assertTrue(metadata.getMarkerType() == MarkerType.TXN_COMMIT_VALUE || + metadata.getMarkerType() == MarkerType.TXN_ABORT_VALUE); + } else { + String value = new String(peekMsg.getValue()); + assertTrue(value.equals("msg") || value.equals("msg-aborted")); + } + } + for (int i = 4 * n; i < peekMsgs.size(); i++) { + Message peekMsg = peekMsgs.get(i); + assertEquals(new String(peekMsg.getValue()), "msg-uncommitted"); + } + } + private static void verifyCoordinatorStats(String state, long sequenceId, long lowWaterMark) { assertEquals(state, "Ready"); diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java index 156d67e4e58b3..ab7b4ce7111e5 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/Topics.java @@ -31,6 +31,7 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.TransactionIsolationLevel; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.AuthAction; @@ -1642,7 +1643,53 @@ void expireMessagesForAllSubscriptions(String topic, long expireTimeInSeconds) * @throws PulsarAdminException * Unexpected error */ - List> peekMessages(String topic, String subName, int numMessages) throws PulsarAdminException; + default List> peekMessages(String topic, String subName, int numMessages) + throws PulsarAdminException { + return peekMessages(topic, subName, numMessages, false, TransactionIsolationLevel.READ_COMMITTED); + } + + /** + * Peek messages from a topic subscription. + * + * @param topic + * topic name + * @param subName + * Subscription name + * @param numMessages + * Number of messages + * @param showServerMarker + * Enables the display of internal server write markers + * @param transactionIsolationLevel + * Sets the isolation level for peeking messages within transactions. + * - 'READ_COMMITTED' allows peeking only committed transactional messages. + * - 'READ_UNCOMMITTED' allows peeking all messages, + * even transactional messages which have been aborted. + * @return + * @throws NotAuthorizedException + * Don't have admin permission + * @throws NotFoundException + * Topic or subscription does not exist + * @throws PulsarAdminException + * Unexpected error + */ + List> peekMessages(String topic, String subName, int numMessages, + boolean showServerMarker, TransactionIsolationLevel transactionIsolationLevel) + throws PulsarAdminException; + + /** + * Peek messages from a topic subscription asynchronously. + * + * @param topic + * topic name + * @param subName + * Subscription name + * @param numMessages + * Number of messages + * @return a future that can be used to track when the messages are returned + */ + default CompletableFuture>> peekMessagesAsync(String topic, String subName, int numMessages) { + return peekMessagesAsync(topic, subName, numMessages, false, TransactionIsolationLevel.READ_COMMITTED); + } /** * Peek messages from a topic subscription asynchronously. @@ -1653,9 +1700,18 @@ void expireMessagesForAllSubscriptions(String topic, long expireTimeInSeconds) * Subscription name * @param numMessages * Number of messages + * @param showServerMarker + * Enables the display of internal server write markers + @param transactionIsolationLevel + * Sets the isolation level for peeking messages within transactions. + * - 'READ_COMMITTED' allows peeking only committed transactional messages. + * - 'READ_UNCOMMITTED' allows peeking all messages, + * even transactional messages which have been aborted. * @return a future that can be used to track when the messages are returned */ - CompletableFuture>> peekMessagesAsync(String topic, String subName, int numMessages); + CompletableFuture>> peekMessagesAsync( + String topic, String subName, int numMessages, + boolean showServerMarker, TransactionIsolationLevel transactionIsolationLevel); /** * Get a message by its messageId via a topic subscription. diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java index a89da628372bd..42907ca72651f 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/TopicsImpl.java @@ -56,6 +56,7 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.TransactionIsolationLevel; import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.client.impl.MessageImpl; @@ -130,6 +131,8 @@ public class TopicsImpl extends BaseResource implements Topics { private static final String SCHEMA_VERSION = "X-Pulsar-Base64-schema-version-b64encoded"; private static final String ENCRYPTION_PARAM = "X-Pulsar-Base64-encryption-param"; private static final String ENCRYPTION_KEYS = "X-Pulsar-Base64-encryption-keys"; + public static final String TXN_ABORTED = "X-Pulsar-txn-aborted"; + public static final String TXN_UNCOMMITTED = "X-Pulsar-txn-uncommitted"; // CHECKSTYLE.ON: MemberName public static final String PROPERTY_SHADOW_SOURCE_KEY = "PULSAR.SHADOW_SOURCE"; @@ -845,7 +848,9 @@ public CompletableFuture expireMessagesForAllSubscriptionsAsync(String top return asyncPostRequest(path, Entity.entity("", MediaType.APPLICATION_JSON)); } - private CompletableFuture>> peekNthMessage(String topic, String subName, int messagePosition) { + private CompletableFuture>> peekNthMessage( + String topic, String subName, int messagePosition, boolean showServerMarker, + TransactionIsolationLevel transactionIsolationLevel) { TopicName tn = validateTopic(topic); String encodedSubName = Codec.encode(subName); WebTarget path = topicPath(tn, "subscription", encodedSubName, @@ -857,7 +862,8 @@ private CompletableFuture>> peekNthMessage(String topic, St @Override public void completed(Response response) { try { - future.complete(getMessagesFromHttpResponse(tn.toString(), response)); + future.complete(getMessagesFromHttpResponse(tn.toString(), response, + showServerMarker, transactionIsolationLevel)); } catch (Exception e) { future.completeExceptionally(getApiException(e)); } @@ -872,28 +878,35 @@ public void failed(Throwable throwable) { } @Override - public List> peekMessages(String topic, String subName, int numMessages) + public List> peekMessages(String topic, String subName, int numMessages, + boolean showServerMarker, + TransactionIsolationLevel transactionIsolationLevel) throws PulsarAdminException { - return sync(() -> peekMessagesAsync(topic, subName, numMessages)); + return sync(() -> peekMessagesAsync(topic, subName, numMessages, showServerMarker, transactionIsolationLevel)); } @Override - public CompletableFuture>> peekMessagesAsync(String topic, String subName, int numMessages) { + public CompletableFuture>> peekMessagesAsync( + String topic, String subName, int numMessages, + boolean showServerMarker, TransactionIsolationLevel transactionIsolationLevel) { checkArgument(numMessages > 0); CompletableFuture>> future = new CompletableFuture>>(); - peekMessagesAsync(topic, subName, numMessages, new ArrayList<>(), future, 1); + peekMessagesAsync(topic, subName, numMessages, new ArrayList<>(), + future, 1, showServerMarker, transactionIsolationLevel); return future; } private void peekMessagesAsync(String topic, String subName, int numMessages, - List> messages, CompletableFuture>> future, int nthMessage) { + List> messages, CompletableFuture>> future, int nthMessage, + boolean showServerMarker, TransactionIsolationLevel transactionIsolationLevel) { if (numMessages <= 0) { future.complete(messages); return; } // if peeking first message succeeds, we know that the topic and subscription exists - peekNthMessage(topic, subName, nthMessage).handle((r, ex) -> { + peekNthMessage(topic, subName, nthMessage, showServerMarker, transactionIsolationLevel) + .handle((r, ex) -> { if (ex != null) { // if we get a not found exception, it means that the position for the message we are trying to get // does not exist. At this point, we can return the already found messages. @@ -908,7 +921,8 @@ private void peekMessagesAsync(String topic, String subName, int numMessages, for (int i = 0; i < Math.min(r.size(), numMessages); i++) { messages.add(r.get(i)); } - peekMessagesAsync(topic, subName, numMessages - r.size(), messages, future, nthMessage + 1); + peekMessagesAsync(topic, subName, numMessages - r.size(), messages, future, + nthMessage + 1, showServerMarker, transactionIsolationLevel); return null; }); } @@ -1222,6 +1236,13 @@ private TopicName validateTopic(String topic) { } private List> getMessagesFromHttpResponse(String topic, Response response) throws Exception { + return getMessagesFromHttpResponse(topic, response, true, + TransactionIsolationLevel.READ_UNCOMMITTED); + } + + private List> getMessagesFromHttpResponse( + String topic, Response response, boolean showServerMarker, + TransactionIsolationLevel transactionIsolationLevel) throws Exception { if (response.getStatus() != Status.OK.getStatusCode()) { throw getApiException(response); @@ -1253,7 +1274,32 @@ private List> getMessagesFromHttpResponse(String topic, Response Map properties = new TreeMap<>(); MultivaluedMap headers = response.getHeaders(); - Object tmp = headers.getFirst(PUBLISH_TIME); + Object tmp = headers.getFirst(MARKER_TYPE); + if (tmp != null) { + if (!showServerMarker) { + return new ArrayList<>(); + } else { + messageMetadata.setMarkerType(Integer.parseInt(tmp.toString())); + } + } + + tmp = headers.getFirst(TXN_ABORTED); + if (tmp != null && Boolean.parseBoolean(tmp.toString())) { + properties.put(TXN_ABORTED, tmp.toString()); + if (transactionIsolationLevel == TransactionIsolationLevel.READ_COMMITTED) { + return new ArrayList<>(); + } + } + + tmp = headers.getFirst(TXN_UNCOMMITTED); + if (tmp != null && Boolean.parseBoolean(tmp.toString())) { + properties.put(TXN_UNCOMMITTED, tmp.toString()); + if (transactionIsolationLevel == TransactionIsolationLevel.READ_COMMITTED) { + return new ArrayList<>(); + } + } + + tmp = headers.getFirst(PUBLISH_TIME); if (tmp != null) { messageMetadata.setPublishTime(DateFormatter.parse(tmp.toString())); } @@ -1305,10 +1351,6 @@ private List> getMessagesFromHttpResponse(String topic, Response if (tmp != null) { messageMetadata.setPartitionKeyB64Encoded(Boolean.parseBoolean(tmp.toString())); } - tmp = headers.getFirst(MARKER_TYPE); - if (tmp != null) { - messageMetadata.setMarkerType(Integer.parseInt(tmp.toString())); - } tmp = headers.getFirst(TXNID_LEAST_BITS); if (tmp != null) { messageMetadata.setTxnidLeastBits(Long.parseLong(tmp.toString())); diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TransactionIsolationLevel.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TransactionIsolationLevel.java new file mode 100644 index 0000000000000..ae385b20232c7 --- /dev/null +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/TransactionIsolationLevel.java @@ -0,0 +1,31 @@ +/* + * 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 org.apache.pulsar.common.classification.InterfaceAudience; +import org.apache.pulsar.common.classification.InterfaceStability; + +@InterfaceAudience.Public +@InterfaceStability.Stable +public enum TransactionIsolationLevel { + // Consumer can only consume all transactional messages which have been committed. + READ_COMMITTED, + // Consumer can consume all messages, even transactional messages which have been aborted. + READ_UNCOMMITTED; +} diff --git a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java index a722abe19df81..8dc6f752c09ad 100644 --- a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java +++ b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java @@ -79,6 +79,7 @@ import org.apache.pulsar.client.admin.internal.OffloadProcessStatusImpl; import org.apache.pulsar.client.admin.internal.PulsarAdminImpl; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.TransactionIsolationLevel; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.schema.SchemaDefinition; import org.apache.pulsar.client.api.transaction.TxnID; @@ -1717,7 +1718,8 @@ public void topics() throws Exception { verify(mockTopics).deletePartitionedTopic("persistent://myprop/clust/ns1/ds1", true); cmdTopics.run(split("peek-messages persistent://myprop/clust/ns1/ds1 -s sub1 -n 3")); - verify(mockTopics).peekMessages("persistent://myprop/clust/ns1/ds1", "sub1", 3); + verify(mockTopics).peekMessages("persistent://myprop/clust/ns1/ds1", "sub1", 3, + false, TransactionIsolationLevel.READ_COMMITTED); MessageImpl message = mock(MessageImpl.class); when(message.getData()).thenReturn(new byte[]{}); 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 47d5326b002cf..d0b04198e13fc 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 @@ -64,10 +64,13 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.TransactionIsolationLevel; import org.apache.pulsar.client.cli.NoSplitter; import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; 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.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.BacklogQuota; @@ -1226,10 +1229,23 @@ private class PeekMessages extends CliCommand { @Parameter(names = { "-n", "--count" }, description = "Number of messages (default 1)", required = false) private int numMessages = 1; + @Parameter(names = { "-ssm", "--show-server-marker" }, + description = "Enables the display of internal server write markers.", required = false) + private boolean showServerMarker = false; + + @Parameter(names = { "-til", "--transaction-isolation-level" }, + description = "Sets the isolation level for peeking messages within transactions. " + + "'READ_COMMITTED' allows peeking only committed transactional messages. " + + "'READ_UNCOMMITTED' allows peeking all messages, " + + "even transactional messages which have been aborted.", + required = false) + private TransactionIsolationLevel transactionIsolationLevel = TransactionIsolationLevel.READ_COMMITTED; + @Override void run() throws PulsarAdminException { String persistentTopic = validatePersistentTopic(params); - List> messages = getTopics().peekMessages(persistentTopic, subName, numMessages); + List> messages = getTopics().peekMessages(persistentTopic, subName, numMessages, + showServerMarker, transactionIsolationLevel); int position = 0; for (Message msg : messages) { MessageImpl message = (MessageImpl) msg; @@ -1251,6 +1267,10 @@ void run() throws PulsarAdminException { if (message.getDeliverAtTime() != 0) { System.out.println("Deliver at time: " + message.getDeliverAtTime()); } + MessageMetadata msgMetaData = message.getMessageBuilder(); + if (showServerMarker && msgMetaData.hasMarkerType()) { + System.out.println("Marker Type: " + MarkerType.valueOf(msgMetaData.getMarkerType())); + } if (message.getBrokerEntryMetadata() != null) { if (message.getBrokerEntryMetadata().hasBrokerTimestamp()) { From 0f868641482d2af3b554478fa07fa576ac1b6cf8 Mon Sep 17 00:00:00 2001 From: Qiang Zhao Date: Tue, 28 May 2024 22:45:30 +0800 Subject: [PATCH 525/699] [improve][broker] avoid creating new objects when intercepting (#22790) (cherry picked from commit 3e17c6338d3976beb1823b4f1838b38584e3a60a) --- .../BrokerInterceptorWithClassLoader.java | 127 ++++++++++++++---- .../intercept/BrokerInterceptorUtilsTest.java | 2 +- .../BrokerInterceptorWithClassLoaderTest.java | 2 +- 3 files changed, 105 insertions(+), 26 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 faee5799289d0..3997e214f4316 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 @@ -29,7 +29,6 @@ import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.Entry; -import org.apache.pulsar.broker.ClassLoaderSwitcher; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.Consumer; import org.apache.pulsar.broker.service.Producer; @@ -51,16 +50,20 @@ public class BrokerInterceptorWithClassLoader implements BrokerInterceptor { private final BrokerInterceptor interceptor; - private final NarClassLoader classLoader; + private final NarClassLoader narClassLoader; @Override public void beforeSendMessage(Subscription subscription, Entry entry, long[] ackSet, MessageMetadata msgMetadata) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.beforeSendMessage( subscription, entry, ackSet, msgMetadata); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @@ -70,25 +73,37 @@ public void beforeSendMessage(Subscription subscription, long[] ackSet, MessageMetadata msgMetadata, Consumer consumer) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.beforeSendMessage( subscription, entry, ackSet, msgMetadata, consumer); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void onMessagePublish(Producer producer, ByteBuf headersAndPayload, Topic.PublishContext publishContext) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.onMessagePublish(producer, headersAndPayload, publishContext); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void producerCreated(ServerCnx cnx, Producer producer, Map metadata){ - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.producerCreated(cnx, producer, metadata); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @@ -96,8 +111,12 @@ public void producerCreated(ServerCnx cnx, Producer producer, public void producerClosed(ServerCnx cnx, Producer producer, Map metadata) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.producerClosed(cnx, producer, metadata); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @@ -105,9 +124,12 @@ public void producerClosed(ServerCnx cnx, public void consumerCreated(ServerCnx cnx, Consumer consumer, Map metadata) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { - this.interceptor.consumerCreated( - cnx, consumer, metadata); + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); + this.interceptor.consumerCreated(cnx, consumer, metadata); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @@ -115,8 +137,12 @@ public void consumerCreated(ServerCnx cnx, public void consumerClosed(ServerCnx cnx, Consumer consumer, Map metadata) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.consumerClosed(cnx, consumer, metadata); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @@ -124,87 +150,140 @@ public void consumerClosed(ServerCnx cnx, @Override public void messageProduced(ServerCnx cnx, Producer producer, long startTimeNs, long ledgerId, long entryId, Topic.PublishContext publishContext) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.messageProduced(cnx, producer, startTimeNs, ledgerId, entryId, publishContext); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void messageDispatched(ServerCnx cnx, Consumer consumer, long ledgerId, long entryId, ByteBuf headersAndPayload) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.messageDispatched(cnx, consumer, ledgerId, entryId, headersAndPayload); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void messageAcked(ServerCnx cnx, Consumer consumer, CommandAck ackCmd) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.messageAcked(cnx, consumer, ackCmd); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void txnOpened(long tcId, String txnID) { - this.interceptor.txnOpened(tcId, txnID); + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); + this.interceptor.txnOpened(tcId, txnID); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); + } } @Override public void txnEnded(String txnID, long txnAction) { - this.interceptor.txnEnded(txnID, txnAction); + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); + this.interceptor.txnEnded(txnID, txnAction); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); + } } @Override public void onConnectionCreated(ServerCnx cnx) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.onConnectionCreated(cnx); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void onPulsarCommand(BaseCommand command, ServerCnx cnx) throws InterceptException { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.onPulsarCommand(command, cnx); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void onConnectionClosed(ServerCnx cnx) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.onConnectionClosed(cnx); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void onWebserviceRequest(ServletRequest request) throws IOException, ServletException, InterceptException { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.onWebserviceRequest(request); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void onWebserviceResponse(ServletRequest request, ServletResponse response) throws IOException, ServletException { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.onWebserviceResponse(request, response); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void initialize(PulsarService pulsarService) throws Exception { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); this.interceptor.initialize(pulsarService); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } } @Override public void close() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + final ClassLoader previousContext = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(narClassLoader); interceptor.close(); + } finally { + Thread.currentThread().setContextClassLoader(previousContext); } + try { - classLoader.close(); + narClassLoader.close(); } catch (IOException e) { log.warn("Failed to close the broker interceptor class loader", e); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorUtilsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorUtilsTest.java index 5abe8a69ee499..979bf6cd0d5db 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorUtilsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorUtilsTest.java @@ -65,7 +65,7 @@ public void testLoadBrokerEventListener() throws Exception { BrokerInterceptorWithClassLoader returnedPhWithCL = BrokerInterceptorUtils.load(metadata, ""); BrokerInterceptor returnedPh = returnedPhWithCL.getInterceptor(); - assertSame(mockLoader, returnedPhWithCL.getClassLoader()); + assertSame(mockLoader, returnedPhWithCL.getNarClassLoader()); assertTrue(returnedPh instanceof MockBrokerInterceptor); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoaderTest.java index a2f97e16a76ae..64d4b5ee6cca5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/intercept/BrokerInterceptorWithClassLoaderTest.java @@ -135,7 +135,7 @@ public void close() { new BrokerInterceptorWithClassLoader(interceptor, narLoader); ClassLoader curClassLoader = Thread.currentThread().getContextClassLoader(); // test class loader - assertEquals(brokerInterceptorWithClassLoader.getClassLoader(), narLoader); + assertEquals(brokerInterceptorWithClassLoader.getNarClassLoader(), narLoader); // test initialize brokerInterceptorWithClassLoader.initialize(mock(PulsarService.class)); assertEquals(Thread.currentThread().getContextClassLoader(), curClassLoader); From bf9cdeb75365a7e950ea0eeaa4e6c7e1d77663f7 Mon Sep 17 00:00:00 2001 From: Enrico Olivelli Date: Wed, 29 May 2024 17:27:00 +0200 Subject: [PATCH 526/699] [fix][broker] EntryFilters fix NoClassDefFoundError due to closed classloader (#22767) (cherry picked from commit caccd54f2ceabae0b24b8c14df088c94b7a72643) --- .../service/plugin/EntryFilterProvider.java | 3 +- .../plugin/EntryFilterWithClassLoader.java | 29 +++++++++++++++---- .../service/plugin/FilterEntryTest.java | 12 ++++---- .../broker/stats/ConsumerStatsTest.java | 2 +- .../broker/stats/SubscriptionStatsTest.java | 2 +- .../pulsar/common/nar/NarClassLoader.java | 16 ++++++++++ 6 files changed, 49 insertions(+), 15 deletions(-) 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 f93e561542eeb..53418744b5486 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 @@ -197,7 +197,8 @@ protected EntryFilter load(EntryFilterMetaData metadata) + " does not implement entry filter interface"); } EntryFilter pi = (EntryFilter) filter; - return new EntryFilterWithClassLoader(pi, ncl); + // the classloader is shared with the broker, the instance doesn't own it + return new EntryFilterWithClassLoader(pi, ncl, false); } catch (Throwable e) { if (e instanceof IOException) { throw (IOException) e; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterWithClassLoader.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterWithClassLoader.java index c5c5721087788..aab46c62acdb4 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterWithClassLoader.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/plugin/EntryFilterWithClassLoader.java @@ -30,15 +30,23 @@ public class EntryFilterWithClassLoader implements EntryFilter { private final EntryFilter entryFilter; private final NarClassLoader classLoader; + private final boolean classLoaderOwned; - public EntryFilterWithClassLoader(EntryFilter entryFilter, NarClassLoader classLoader) { + public EntryFilterWithClassLoader(EntryFilter entryFilter, NarClassLoader classLoader, boolean classLoaderOwned) { this.entryFilter = entryFilter; this.classLoader = classLoader; + this.classLoaderOwned = classLoaderOwned; } @Override public FilterResult filterEntry(Entry entry, FilterContext context) { - return entryFilter.filterEntry(entry, context); + ClassLoader currentClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); + return entryFilter.filterEntry(entry, context); + } finally { + Thread.currentThread().setContextClassLoader(currentClassLoader); + } } @VisibleForTesting @@ -48,11 +56,20 @@ public EntryFilter getEntryFilter() { @Override public void close() { - entryFilter.close(); + ClassLoader currentClassLoader = Thread.currentThread().getContextClassLoader(); try { - classLoader.close(); - } catch (IOException e) { - log.error("close EntryFilterWithClassLoader failed", e); + Thread.currentThread().setContextClassLoader(classLoader); + entryFilter.close(); + } finally { + Thread.currentThread().setContextClassLoader(currentClassLoader); + } + if (classLoaderOwned) { + log.info("Closing classloader {} for EntryFilter {}", classLoader, entryFilter.getClass().getName()); + try { + classLoader.close(); + } catch (IOException e) { + log.error("close EntryFilterWithClassLoader failed", e); + } } } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java index 1c4f88bc0273c..6c847db05a0e3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/plugin/FilterEntryTest.java @@ -239,9 +239,9 @@ public void testFilter() throws Exception { hasFilterField.setAccessible(true); NarClassLoader narClassLoader = mock(NarClassLoader.class); EntryFilter filter1 = new EntryFilterTest(); - EntryFilterWithClassLoader loader1 = spyWithClassAndConstructorArgsRecordingInvocations(EntryFilterWithClassLoader.class, filter1, narClassLoader); + EntryFilterWithClassLoader loader1 = spyWithClassAndConstructorArgsRecordingInvocations(EntryFilterWithClassLoader.class, filter1, narClassLoader, false); EntryFilter filter2 = new EntryFilter2Test(); - EntryFilterWithClassLoader loader2 = spyWithClassAndConstructorArgsRecordingInvocations(EntryFilterWithClassLoader.class, filter2, narClassLoader); + EntryFilterWithClassLoader loader2 = spyWithClassAndConstructorArgsRecordingInvocations(EntryFilterWithClassLoader.class, filter2, narClassLoader, false); field.set(dispatcher, List.of(loader1, loader2)); hasFilterField.set(dispatcher, true); @@ -371,9 +371,9 @@ public void testFilteredMsgCount(String topic) throws Throwable { hasFilterField.setAccessible(true); NarClassLoader narClassLoader = mock(NarClassLoader.class); EntryFilter filter1 = new EntryFilterTest(); - EntryFilterWithClassLoader loader1 = spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter1, narClassLoader); + EntryFilterWithClassLoader loader1 = spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter1, narClassLoader, false); EntryFilter filter2 = new EntryFilter2Test(); - EntryFilterWithClassLoader loader2 = spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter2, narClassLoader); + EntryFilterWithClassLoader loader2 = spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter2, narClassLoader, false); field.set(dispatcher, List.of(loader1, loader2)); hasFilterField.set(dispatcher, true); @@ -463,10 +463,10 @@ public void testEntryFilterRescheduleMessageDependingOnConsumerSharedSubscriptio NarClassLoader narClassLoader = mock(NarClassLoader.class); EntryFilter filter1 = new EntryFilterTest(); EntryFilterWithClassLoader loader1 = - spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter1, narClassLoader); + spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter1, narClassLoader, false); EntryFilter filter2 = new EntryFilterTest(); EntryFilterWithClassLoader loader2 = - spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter2, narClassLoader); + spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter2, narClassLoader, false); field.set(dispatcher, List.of(loader1, loader2)); hasFilterField.set(dispatcher, true); 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 512a5cfcab661..5aeed40107d5d 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 @@ -409,7 +409,7 @@ public void testAvgMessagesPerEntry() throws Exception { EntryFilter filter = new EntryFilterProducerTest(); EntryFilterWithClassLoader loader = spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter, - narClassLoader); + narClassLoader, false); Pair> entryFilters = Pair.of("filter", List.of(loader)); PersistentTopic topicRef = (PersistentTopic) pulsar.getBrokerService() 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 3e71d8f211101..bc4cb73e5b6fe 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 @@ -208,7 +208,7 @@ public void testSubscriptionStats(final String topic, final String subName, bool NarClassLoader narClassLoader = mock(NarClassLoader.class); EntryFilter filter1 = new EntryFilterTest(); EntryFilterWithClassLoader loader1 = - spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter1, narClassLoader); + spyWithClassAndConstructorArgs(EntryFilterWithClassLoader.class, filter1, narClassLoader, false); field.set(dispatcher, List.of(loader1)); hasFilterField.set(dispatcher, true); } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarClassLoader.java b/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarClassLoader.java index 9736d8b47ef71..44cfc2872ef6b 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarClassLoader.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/nar/NarClassLoader.java @@ -40,6 +40,7 @@ import java.util.Comparator; import java.util.List; import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; @@ -135,6 +136,7 @@ public class NarClassLoader extends URLClassLoader { * The NAR for which this ClassLoader is responsible. */ private final File narWorkingDirectory; + private final AtomicBoolean closed = new AtomicBoolean(); private static final String TMP_DIR_PREFIX = "pulsar-nar"; @@ -292,4 +294,18 @@ protected String findLibrary(final String libname) { public String toString() { return NarClassLoader.class.getName() + "[" + narWorkingDirectory.getPath() + "]"; } + + @Override + protected Class loadClass(String name, boolean resolve) throws ClassNotFoundException { + if (closed.get()) { + log.warn("Loading class {} from a closed classloader ({})", name, this); + } + return super.loadClass(name, resolve); + } + + @Override + public void close() throws IOException { + closed.set(true); + super.close(); + } } From 49902fe25ec5f23e9577b11eeb6b800e39460e20 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 22 May 2024 05:56:14 +0300 Subject: [PATCH 527/699] [improve][broker] Clear thread local BrokerEntryMetadata instance before reuse (#22752) (cherry picked from commit 18db799297d6c6980100d9367231b3aa42db838e) --- .../main/java/org/apache/pulsar/common/protocol/Commands.java | 1 + 1 file changed, 1 insertion(+) 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 3982900041813..faa5fbcd30130 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 @@ -1675,6 +1675,7 @@ public static ByteBuf addBrokerEntryMetadata(ByteBuf headerAndPayload, // | 2 bytes | 4 bytes | BROKER_ENTRY_METADATA_SIZE bytes | BrokerEntryMetadata brokerEntryMetadata = BROKER_ENTRY_METADATA.get(); + brokerEntryMetadata.clear(); for (BrokerEntryMetadataInterceptor interceptor : brokerInterceptors) { interceptor.intercept(brokerEntryMetadata); if (numberOfMessages >= 0) { From 232cde39a13c694ec8edda368c6d7167d0623b93 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Wed, 29 May 2024 22:19:47 +0800 Subject: [PATCH 528/699] [improve][broker] Remove ClassLoaderSwitcher to avoid objects allocations and consistent the codestyle (#22796) (cherry picked from commit 7918ed5805d8175dab2cb4cfa2c1e97b0eb80a09) --- .../pulsar/broker/ClassLoaderSwitcher.java | 37 ---------------- .../AdditionalServletWithClassLoader.java | 25 ++++++++--- .../ProtocolHandlerWithClassLoader.java | 44 +++++++++++++++---- 3 files changed, 55 insertions(+), 51 deletions(-) delete mode 100644 pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ClassLoaderSwitcher.java diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ClassLoaderSwitcher.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ClassLoaderSwitcher.java deleted file mode 100644 index 55cb9198da2bc..0000000000000 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ClassLoaderSwitcher.java +++ /dev/null @@ -1,37 +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.broker; - -/** - * Help to switch the class loader of current thread to the NarClassLoader, and change it back when it's done. - * With the help of try-with-resources statement, the code would be cleaner than using try finally every time. - */ -public class ClassLoaderSwitcher implements AutoCloseable { - private final ClassLoader prevClassLoader; - - public ClassLoaderSwitcher(ClassLoader classLoader) { - prevClassLoader = Thread.currentThread().getContextClassLoader(); - Thread.currentThread().setContextClassLoader(classLoader); - } - - @Override - public void close() { - Thread.currentThread().setContextClassLoader(prevClassLoader); - } -} \ No newline at end of file diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletWithClassLoader.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletWithClassLoader.java index c2b4b90073391..bc1f25c5af933 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletWithClassLoader.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/web/plugin/servlet/AdditionalServletWithClassLoader.java @@ -22,7 +22,6 @@ import lombok.Data; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; -import org.apache.pulsar.broker.ClassLoaderSwitcher; import org.apache.pulsar.common.configuration.PulsarConfiguration; import org.apache.pulsar.common.nar.NarClassLoader; import org.eclipse.jetty.servlet.ServletHolder; @@ -40,29 +39,45 @@ public class AdditionalServletWithClassLoader implements AdditionalServlet { @Override public void loadConfig(PulsarConfiguration pulsarConfiguration) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); servlet.loadConfig(pulsarConfiguration); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public String getBasePath() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); return servlet.getBasePath(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public ServletHolder getServletHolder() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); return servlet.getServletHolder(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public void close() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); servlet.close(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } try { classLoader.close(); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlerWithClassLoader.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlerWithClassLoader.java index d648c261403d4..eb4bcb0a9bf4b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlerWithClassLoader.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/protocol/ProtocolHandlerWithClassLoader.java @@ -26,7 +26,6 @@ import lombok.Data; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; -import org.apache.pulsar.broker.ClassLoaderSwitcher; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.common.nar.NarClassLoader; @@ -44,52 +43,79 @@ class ProtocolHandlerWithClassLoader implements ProtocolHandler { @Override public String protocolName() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); return handler.protocolName(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public boolean accept(String protocol) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); return handler.accept(protocol); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public void initialize(ServiceConfiguration conf) throws Exception { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); handler.initialize(conf); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public String getProtocolDataToAdvertise() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); return handler.getProtocolDataToAdvertise(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public void start(BrokerService service) { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); handler.start(service); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public Map> newChannelInitializers() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); return handler.newChannelInitializers(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } } @Override public void close() { - try (ClassLoaderSwitcher ignored = new ClassLoaderSwitcher(classLoader)) { + ClassLoader prevClassLoader = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(classLoader); handler.close(); + } finally { + Thread.currentThread().setContextClassLoader(prevClassLoader); } - try { classLoader.close(); } catch (IOException e) { From 8f59cfb08ceea1c7a2ce73d1aca5e01351052008 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 6 Jun 2024 00:41:15 +0300 Subject: [PATCH 529/699] [improve][build] Support git worktree working directory while building docker images (#22851) (cherry picked from commit c23e677ae8c53c192053a7a24f73ede9cb913e50) (cherry picked from commit 9d09dafb3389a3dc3bf7c654a1bff2dedbb6c2ff) --- docker/pulsar-all/pom.xml | 2 +- docker/pulsar/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index 81d72e5a107fb..be1a7f00d990c 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -70,7 +70,7 @@ git-commit-id-no-git - ${basedir}/../../.git/index + ${basedir}/../../.git diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index ff1508a6fbac8..2719641c7180c 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -52,7 +52,7 @@ git-commit-id-no-git - ${basedir}/../../.git/index + ${basedir}/../../.git From 0ee36875c5e86229f4d2858932ab08e640b3595c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 5 Jun 2024 22:02:43 +0300 Subject: [PATCH 530/699] [fix][meta] Check if metadata store is closed in RocksdbMetadataStore (#22852) (cherry picked from commit 74192871ed00870e5181a5bd4018ba196fd8f698) (cherry picked from commit 396e3e62be76c71f1b478d47887d3b4b5143f3e6) --- .../metadata/impl/AbstractMetadataStore.java | 25 +++++++++---------- .../metadata/impl/RocksdbMetadataStore.java | 15 +++++++++++ 2 files changed, 27 insertions(+), 13 deletions(-) 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 8949a717863f9..b520818fb28d6 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 @@ -247,8 +247,7 @@ public MetadataCache getMetadataCache(MetadataSerde serde, MetadataCac @Override public CompletableFuture> get(String path) { if (isClosed()) { - return FutureUtil.failedFuture( - new MetadataStoreException.AlreadyClosedException()); + return alreadyClosedFailedFuture(); } long start = System.currentTimeMillis(); if (!isValidPath(path)) { @@ -276,8 +275,7 @@ public CompletableFuture put(String path, byte[] value, Optional exp @Override public final CompletableFuture> getChildren(String path) { if (isClosed()) { - return FutureUtil.failedFuture( - new MetadataStoreException.AlreadyClosedException()); + return alreadyClosedFailedFuture(); } if (!isValidPath(path)) { return FutureUtil.failedFuture(new MetadataStoreException.InvalidPathException(path)); @@ -288,8 +286,7 @@ public final CompletableFuture> getChildren(String path) { @Override public final CompletableFuture exists(String path) { if (isClosed()) { - return FutureUtil.failedFuture( - new MetadataStoreException.AlreadyClosedException()); + return alreadyClosedFailedFuture(); } if (!isValidPath(path)) { return FutureUtil.failedFuture(new MetadataStoreException.InvalidPathException(path)); @@ -352,8 +349,7 @@ public void accept(Notification n) { public final CompletableFuture delete(String path, Optional expectedVersion) { log.info("Deleting path: {} (v. {})", path, expectedVersion); if (isClosed()) { - return FutureUtil.failedFuture( - new MetadataStoreException.AlreadyClosedException()); + return alreadyClosedFailedFuture(); } long start = System.currentTimeMillis(); if (!isValidPath(path)) { @@ -404,8 +400,7 @@ private CompletableFuture deleteInternal(String path, Optional expec public CompletableFuture deleteRecursive(String path) { log.info("Deleting recursively path: {}", path); if (isClosed()) { - return FutureUtil.failedFuture( - new MetadataStoreException.AlreadyClosedException()); + return alreadyClosedFailedFuture(); } return getChildren(path) .thenCompose(children -> FutureUtil.waitForAll( @@ -425,8 +420,7 @@ protected abstract CompletableFuture storePut(String path, byte[] data, Op public final CompletableFuture put(String path, byte[] data, Optional optExpectedVersion, EnumSet options) { if (isClosed()) { - return FutureUtil.failedFuture( - new MetadataStoreException.AlreadyClosedException()); + return alreadyClosedFailedFuture(); } long start = System.currentTimeMillis(); if (!isValidPath(path)) { @@ -506,10 +500,15 @@ protected void receivedSessionEvent(SessionEvent event) { } } - private boolean isClosed() { + protected boolean isClosed() { return isClosed.get(); } + protected static CompletableFuture alreadyClosedFailedFuture() { + return FutureUtil.failedFuture( + new MetadataStoreException.AlreadyClosedException()); + } + @Override public void close() throws Exception { executor.shutdownNow(); diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/RocksdbMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/RocksdbMetadataStore.java index 39f7edd5ceed5..06f7b26053693 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/RocksdbMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/RocksdbMetadataStore.java @@ -375,6 +375,9 @@ public CompletableFuture> storeGet(String path) { } try { dbStateLock.readLock().lock(); + if (isClosed()) { + return alreadyClosedFailedFuture(); + } byte[] value = db.get(optionCache, toBytes(path)); if (value == null) { return CompletableFuture.completedFuture(Optional.empty()); @@ -407,6 +410,9 @@ protected CompletableFuture> getChildrenFromStore(String path) { } try { dbStateLock.readLock().lock(); + if (isClosed()) { + return alreadyClosedFailedFuture(); + } try (RocksIterator iterator = db.newIterator(optionDontCache)) { Set result = new HashSet<>(); String firstKey = path.equals("/") ? path : path + "/"; @@ -449,6 +455,9 @@ protected CompletableFuture existsFromStore(String path) { } try { dbStateLock.readLock().lock(); + if (isClosed()) { + return alreadyClosedFailedFuture(); + } byte[] value = db.get(optionDontCache, toBytes(path)); if (log.isDebugEnabled()) { if (value != null) { @@ -471,6 +480,9 @@ protected CompletableFuture storeDelete(String path, Optional expect } try { dbStateLock.readLock().lock(); + if (isClosed()) { + return alreadyClosedFailedFuture(); + } try (Transaction transaction = db.beginTransaction(writeOptions)) { byte[] pathBytes = toBytes(path); byte[] oldValueData = transaction.getForUpdate(optionDontCache, pathBytes, true); @@ -507,6 +519,9 @@ protected CompletableFuture storePut(String path, byte[] data, Optional Date: Fri, 31 May 2024 03:25:52 +0300 Subject: [PATCH 531/699] [fix][ml] Fix race conditions in RangeCache (#22789) (cherry picked from commit c39f9f82b425c66c899f818583714c9c98d3e213) (cherry picked from commit 9a99e4586067307848179031a3446fa1e0044683) --- .../bookkeeper/mledger/impl/EntryImpl.java | 7 +- .../bookkeeper/mledger/util/RangeCache.java | 278 +++++++++++++----- .../mledger/util/RangeCacheTest.java | 63 ++-- 3 files changed, 254 insertions(+), 94 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java index 6512399173f0a..e53f408ca7563 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java @@ -27,9 +27,10 @@ import org.apache.bookkeeper.client.api.LedgerEntry; import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.util.AbstractCASReferenceCounted; +import org.apache.bookkeeper.mledger.util.RangeCache; public final class EntryImpl extends AbstractCASReferenceCounted implements Entry, Comparable, - ReferenceCounted { + RangeCache.ValueWithKeyValidation { private static final Recycler RECYCLER = new Recycler() { @Override @@ -200,4 +201,8 @@ protected void deallocate() { recyclerHandle.recycle(this); } + @Override + public boolean matchesKey(PositionImpl key) { + return key.compareTo(ledgerId, entryId) == 0; + } } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/util/RangeCache.java index d34857e5e5177..46d03bea1b5ad 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 @@ -19,31 +19,134 @@ package org.apache.bookkeeper.mledger.util; import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.base.Predicate; +import io.netty.util.IllegalReferenceCountException; +import io.netty.util.Recycler; +import io.netty.util.Recycler.Handle; import io.netty.util.ReferenceCounted; import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicLong; +import org.apache.bookkeeper.mledger.util.RangeCache.ValueWithKeyValidation; import org.apache.commons.lang3.tuple.Pair; /** * Special type of cache where get() and delete() operations can be done over a range of keys. + * The implementation avoids locks and synchronization and relies on ConcurrentSkipListMap for storing the entries. + * Since there is no locks, there is a need to have a way to ensure that a single entry in the cache is removed + * exactly once. Removing an entry multiple times would result in the entries of the cache getting released too + * while they could still be in use. * * @param * Cache key. Needs to be Comparable * @param * Cache value */ -public class RangeCache, Value extends ReferenceCounted> { +public class RangeCache, Value extends ValueWithKeyValidation> { + public interface ValueWithKeyValidation extends ReferenceCounted { + boolean matchesKey(T key); + } + // Map from key to nodes inside the linked list - private final ConcurrentNavigableMap entries; + private final ConcurrentNavigableMap> entries; private AtomicLong size; // Total size of values stored in cache private final Weighter weighter; // Weighter object used to extract the size from values private final TimestampExtractor timestampExtractor; // Extract the timestamp associated with a value + /** + * Wrapper around the value to store in Map. This is needed to ensure that a specific instance can be removed from + * the map by calling the {@link Map#remove(Object, Object)} method. Certain race conditions could result in the + * wrong value being removed from the map. The instances of this class are recycled to avoid creating new objects. + */ + private static class IdentityWrapper { + private final Handle recyclerHandle; + private static final Recycler RECYCLER = new Recycler() { + @Override + protected IdentityWrapper newObject(Handle recyclerHandle) { + return new IdentityWrapper(recyclerHandle); + } + }; + private K key; + private V value; + + private IdentityWrapper(Handle recyclerHandle) { + this.recyclerHandle = recyclerHandle; + } + + static IdentityWrapper create(K key, V value) { + IdentityWrapper identityWrapper = RECYCLER.get(); + identityWrapper.key = key; + identityWrapper.value = value; + return identityWrapper; + } + + K getKey() { + return key; + } + + V getValue() { + return value; + } + + void recycle() { + value = null; + recyclerHandle.recycle(this); + } + + @Override + public boolean equals(Object o) { + // only match exact identity of the value + return this == o; + } + + @Override + public int hashCode() { + return Objects.hashCode(key); + } + } + + /** + * Mutable object to store the number of entries and the total size removed from the cache. The instances + * are recycled to avoid creating new instances. + */ + private static class RemovalCounters { + private final Handle recyclerHandle; + private static final Recycler RECYCLER = new Recycler() { + @Override + protected RemovalCounters newObject(Handle recyclerHandle) { + return new RemovalCounters(recyclerHandle); + } + }; + int removedEntries; + long removedSize; + private RemovalCounters(Handle recyclerHandle) { + this.recyclerHandle = recyclerHandle; + } + + static RemovalCounters create() { + RemovalCounters results = RECYCLER.get(); + results.removedEntries = 0; + results.removedSize = 0; + return results; + } + + void recycle() { + removedEntries = 0; + removedSize = 0; + recyclerHandle.recycle(this); + } + + public void entryRemoved(long size) { + removedSize += size; + removedEntries++; + } + } + /** * Construct a new RangeLruCache with default Weighter. */ @@ -68,18 +171,23 @@ public RangeCache(Weighter weighter, TimestampExtractor timestampE * Insert. * * @param key - * @param value - * ref counted value with at least 1 ref to pass on the cache + * @param value ref counted value with at least 1 ref to pass on the cache * @return whether the entry was inserted in the cache */ public boolean put(Key key, Value value) { // retain value so that it's not released before we put it in the cache and calculate the weight value.retain(); try { - if (entries.putIfAbsent(key, value) == null) { + if (!value.matchesKey(key)) { + throw new IllegalArgumentException("Value '" + value + "' does not match key '" + key + "'"); + } + IdentityWrapper newWrapper = IdentityWrapper.create(key, value); + if (entries.putIfAbsent(key, newWrapper) == null) { size.addAndGet(weighter.getSize(value)); return true; } else { + // recycle the new wrapper as it was not used + newWrapper.recycle(); return false; } } finally { @@ -91,16 +199,37 @@ public boolean exists(Key key) { return key != null ? entries.containsKey(key) : true; } + /** + * Get the value associated with the key and increment the reference count of it. + * The caller is responsible for releasing the reference. + */ public Value get(Key key) { - Value value = entries.get(key); - if (value == null) { + return getValue(key, entries.get(key)); + } + + private Value getValue(Key key, IdentityWrapper valueWrapper) { + if (valueWrapper == null) { return null; } else { + if (valueWrapper.getKey() != key) { + // the wrapper has been recycled and contains another key + return null; + } + Value value = valueWrapper.getValue(); 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; - } catch (Throwable t) { - // Value was already destroyed between get() and retain() + } else { + // Value or IdentityWrapper was recycled and already contains another value + // release the reference added in this method + value.release(); return null; } } @@ -118,12 +247,10 @@ public Collection getRange(Key first, Key last) { List values = new ArrayList(); // Return the values of the entries found in cache - for (Value value : entries.subMap(first, true, last, true).values()) { - try { - value.retain(); + for (Map.Entry> entry : entries.subMap(first, true, last, true).entrySet()) { + Value value = getValue(entry.getKey(), entry.getValue()); + if (value != null) { values.add(value); - } catch (Throwable t) { - // Value was already destroyed between get() and retain() } } @@ -138,25 +265,65 @@ 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) { - Map subMap = entries.subMap(first, true, last, lastInclusive); + RemovalCounters counters = RemovalCounters.create(); + Map> subMap = entries.subMap(first, true, last, lastInclusive); + for (Map.Entry> entry : subMap.entrySet()) { + removeEntry(entry, counters); + } + return handleRemovalResult(counters); + } - int removedEntries = 0; - long removedSize = 0; + enum RemoveEntryResult { + ENTRY_REMOVED, + CONTINUE_LOOP, + BREAK_LOOP; + } - for (Key key : subMap.keySet()) { - Value value = entries.remove(key); - if (value == null) { - continue; - } + private RemoveEntryResult removeEntry(Map.Entry> entry, RemovalCounters counters) { + return removeEntry(entry, counters, (x) -> true); + } - removedSize += weighter.getSize(value); + private RemoveEntryResult removeEntry(Map.Entry> entry, RemovalCounters counters, + Predicate removeCondition) { + Key key = entry.getKey(); + IdentityWrapper identityWrapper = entry.getValue(); + if (identityWrapper.getKey() != key) { + // the wrapper has been recycled and contains another key + return RemoveEntryResult.CONTINUE_LOOP; + } + Value value = identityWrapper.getValue(); + try { + // add extra retain to avoid value being released while we are removing it + value.retain(); + } catch (IllegalReferenceCountException e) { + // Value was already released + return RemoveEntryResult.CONTINUE_LOOP; + } + try { + if (!removeCondition.test(value)) { + return RemoveEntryResult.BREAK_LOOP; + } + // 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 + // it is valid that the value contains references even after the key has been removed from the cache + if (value.refCnt() > 1 && value.matchesKey(key) && entries.remove(key, identityWrapper)) { + identityWrapper.recycle(); + counters.entryRemoved(weighter.getSize(value)); + // remove the cache reference + value.release(); + } + } finally { + // remove the extra retain value.release(); - ++removedEntries; } + return RemoveEntryResult.ENTRY_REMOVED; + } - size.addAndGet(-removedSize); - - return Pair.of(removedEntries, removedSize); + private Pair handleRemovalResult(RemovalCounters counters) { + size.addAndGet(-counters.removedSize); + Pair result = Pair.of(counters.removedEntries, counters.removedSize); + counters.recycle(); + return result; } /** @@ -166,24 +333,15 @@ public Pair removeRange(Key first, Key last, boolean lastInclusiv */ public Pair evictLeastAccessedEntries(long minSize) { checkArgument(minSize > 0); - - long removedSize = 0; - int removedEntries = 0; - - while (removedSize < minSize) { - Map.Entry entry = entries.pollFirstEntry(); + RemovalCounters counters = RemovalCounters.create(); + while (counters.removedSize < minSize) { + Map.Entry> entry = entries.firstEntry(); if (entry == null) { break; } - - Value value = entry.getValue(); - ++removedEntries; - removedSize += weighter.getSize(value); - value.release(); + removeEntry(entry, counters); } - - size.addAndGet(-removedSize); - return Pair.of(removedEntries, removedSize); + return handleRemovalResult(counters); } /** @@ -192,27 +350,18 @@ public Pair evictLeastAccessedEntries(long minSize) { * @return the tota */ public Pair evictLEntriesBeforeTimestamp(long maxTimestamp) { - long removedSize = 0; - int removedCount = 0; - + RemovalCounters counters = RemovalCounters.create(); while (true) { - Map.Entry entry = entries.firstEntry(); - if (entry == null || timestampExtractor.getTimestamp(entry.getValue()) > maxTimestamp) { + Map.Entry> entry = entries.firstEntry(); + if (entry == null) { break; } - Value value = entry.getValue(); - boolean removeHits = entries.remove(entry.getKey(), value); - if (!removeHits) { + if (removeEntry(entry, counters, value -> timestampExtractor.getTimestamp(value) <= maxTimestamp) + == RemoveEntryResult.BREAK_LOOP) { break; } - - removedSize += weighter.getSize(value); - removedCount++; - value.release(); } - - size.addAndGet(-removedSize); - return Pair.of(removedCount, removedSize); + return handleRemovalResult(counters); } /** @@ -231,23 +380,16 @@ public long getSize() { * * @return size of removed entries */ - public synchronized Pair clear() { - long removedSize = 0; - int removedCount = 0; - + public Pair clear() { + RemovalCounters counters = RemovalCounters.create(); while (true) { - Map.Entry entry = entries.pollFirstEntry(); + Map.Entry> entry = entries.firstEntry(); if (entry == null) { break; } - Value value = entry.getValue(); - removedSize += weighter.getSize(value); - removedCount++; - value.release(); + removeEntry(entry, counters); } - - size.getAndAdd(-removedSize); - return Pair.of(removedCount, removedSize); + 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 8ce0db4ac4caa..01b3c67bf1113 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 @@ -23,25 +23,30 @@ import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; - import com.google.common.collect.Lists; import io.netty.util.AbstractReferenceCounted; import io.netty.util.ReferenceCounted; -import org.apache.commons.lang3.tuple.Pair; -import org.testng.annotations.Test; -import java.util.UUID; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import lombok.Cleanup; +import org.apache.commons.lang3.tuple.Pair; +import org.testng.annotations.Test; public class RangeCacheTest { - class RefString extends AbstractReferenceCounted implements ReferenceCounted { + class RefString extends AbstractReferenceCounted implements RangeCache.ValueWithKeyValidation { String s; + Integer matchingKey; RefString(String s) { + this(s, null); + } + + RefString(String s, Integer matchingKey) { super(); this.s = s; + this.matchingKey = matchingKey != null ? matchingKey : Integer.parseInt(s); setRefCnt(1); } @@ -65,6 +70,11 @@ public boolean equals(Object obj) { return false; } + + @Override + public boolean matchesKey(Integer key) { + return matchingKey.equals(key); + } } @Test @@ -119,8 +129,8 @@ public void simple() { public void customWeighter() { RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); - cache.put(0, new RefString("zero")); - cache.put(1, new RefString("one")); + cache.put(0, new RefString("zero", 0)); + cache.put(1, new RefString("one", 1)); assertEquals(cache.getSize(), 7); assertEquals(cache.getNumberOfEntries(), 2); @@ -132,9 +142,9 @@ public void customTimeExtraction() { RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> x.s.length()); cache.put(1, new RefString("1")); - cache.put(2, new RefString("22")); - cache.put(3, new RefString("333")); - cache.put(4, new RefString("4444")); + cache.put(22, new RefString("22")); + cache.put(333, new RefString("333")); + cache.put(4444, new RefString("4444")); assertEquals(cache.getSize(), 10); assertEquals(cache.getNumberOfEntries(), 4); @@ -151,12 +161,12 @@ public void customTimeExtraction() { public void doubleInsert() { RangeCache cache = new RangeCache<>(); - RefString s0 = new RefString("zero"); + RefString s0 = new RefString("zero", 0); assertEquals(s0.refCnt(), 1); assertTrue(cache.put(0, s0)); assertEquals(s0.refCnt(), 1); - cache.put(1, new RefString("one")); + cache.put(1, new RefString("one", 1)); assertEquals(cache.getSize(), 2); assertEquals(cache.getNumberOfEntries(), 2); @@ -164,7 +174,7 @@ public void doubleInsert() { assertEquals(s.s, "one"); assertEquals(s.refCnt(), 2); - RefString s1 = new RefString("uno"); + RefString s1 = new RefString("uno", 1); assertEquals(s1.refCnt(), 1); assertFalse(cache.put(1, s1)); assertEquals(s1.refCnt(), 1); @@ -201,10 +211,10 @@ public void getRange() { public void eviction() { RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); - cache.put(0, new RefString("zero")); - cache.put(1, new RefString("one")); - cache.put(2, new RefString("two")); - cache.put(3, new RefString("three")); + cache.put(0, new RefString("zero", 0)); + cache.put(1, new RefString("one", 1)); + cache.put(2, new RefString("two", 2)); + cache.put(3, new RefString("three", 3)); // This should remove the LRU entries: 0, 1 whose combined size is 7 assertEquals(cache.evictLeastAccessedEntries(5), Pair.of(2, (long) 7)); @@ -276,20 +286,23 @@ public void evictions() { } @Test - public void testInParallel() { - RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); - ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); - executor.scheduleWithFixedDelay(cache::clear, 10, 10, TimeUnit.MILLISECONDS); - for (int i = 0; i < 1000; i++) { - cache.put(UUID.randomUUID().toString(), new RefString("zero")); + public void testPutWhileClearIsCalledConcurrently() { + RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); + int numberOfThreads = 4; + @Cleanup("shutdownNow") + ScheduledExecutorService executor = Executors.newScheduledThreadPool(numberOfThreads); + for (int i = 0; i < numberOfThreads; i++) { + executor.scheduleWithFixedDelay(cache::clear, 0, 1, TimeUnit.MILLISECONDS); + } + for (int i = 0; i < 100000; i++) { + cache.put(i, new RefString(String.valueOf(i))); } - executor.shutdown(); } @Test public void testPutSameObj() { RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); - RefString s0 = new RefString("zero"); + RefString s0 = new RefString("zero", 0); assertEquals(s0.refCnt(), 1); assertTrue(cache.put(0, s0)); assertFalse(cache.put(0, s0)); From 040465d01df57a2cc7d05563941e13d688cad4a6 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 31 May 2024 23:14:50 +0300 Subject: [PATCH 532/699] [improve][ml] RangeCache refactoring: test race conditions and prevent endless loops (#22814) (cherry picked from commit e731674f61a973e9b12eab9394f82731c8fc2384) (cherry picked from commit 1f04038026fdb9d91f1247198f63b48da004dc44) --- .../bookkeeper/mledger/util/RangeCache.java | 172 +++++++++++------- .../mledger/impl/EntryCacheManagerTest.java | 2 +- .../mledger/util/RangeCacheTest.java | 35 +++- 3 files changed, 143 insertions(+), 66 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 46d03bea1b5ad..45295d7190654 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 @@ -28,32 +28,36 @@ import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicLong; +import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.util.RangeCache.ValueWithKeyValidation; import org.apache.commons.lang3.tuple.Pair; /** * Special type of cache where get() and delete() operations can be done over a range of keys. - * The implementation avoids locks and synchronization and relies on ConcurrentSkipListMap for storing the entries. - * Since there is no locks, there is a need to have a way to ensure that a single entry in the cache is removed - * exactly once. Removing an entry multiple times would result in the entries of the cache getting released too - * while they could still be in use. + * The implementation avoids locks and synchronization by relying on ConcurrentSkipListMap for storing the entries. + * Since there are no locks, it's necessary to ensure that a single entry in the cache is removed exactly once. + * Removing an entry multiple times could result in the entries of the cache being released multiple times, + * even while they are still in use. This is prevented by using a custom wrapper around the value to store in the map + * that ensures that the value is removed from the map only if the exact same instance is present in the map. + * There's also a check that ensures that the value matches the key. This is used to detect races without impacting + * consistency. * * @param * Cache key. Needs to be Comparable * @param * Cache value */ +@Slf4j public class RangeCache, Value extends ValueWithKeyValidation> { public interface ValueWithKeyValidation extends ReferenceCounted { boolean matchesKey(T key); } // Map from key to nodes inside the linked list - private final ConcurrentNavigableMap> entries; + private final ConcurrentNavigableMap> entries; private AtomicLong size; // Total size of values stored in cache private final Weighter weighter; // Weighter object used to extract the size from values private final TimestampExtractor timestampExtractor; // Extract the timestamp associated with a value @@ -63,51 +67,53 @@ public interface ValueWithKeyValidation extends ReferenceCounted { * the map by calling the {@link Map#remove(Object, Object)} method. Certain race conditions could result in the * wrong value being removed from the map. The instances of this class are recycled to avoid creating new objects. */ - private static class IdentityWrapper { - private final Handle recyclerHandle; - private static final Recycler RECYCLER = new Recycler() { + private static class EntryWrapper { + private final Handle recyclerHandle; + private static final Recycler RECYCLER = new Recycler() { @Override - protected IdentityWrapper newObject(Handle recyclerHandle) { - return new IdentityWrapper(recyclerHandle); + protected EntryWrapper newObject(Handle recyclerHandle) { + return new EntryWrapper(recyclerHandle); } }; private K key; private V value; + long size; - private IdentityWrapper(Handle recyclerHandle) { + private EntryWrapper(Handle recyclerHandle) { this.recyclerHandle = recyclerHandle; } - static IdentityWrapper create(K key, V value) { - IdentityWrapper identityWrapper = RECYCLER.get(); - identityWrapper.key = key; - identityWrapper.value = value; - return identityWrapper; + static EntryWrapper create(K key, V value, long size) { + EntryWrapper entryWrapper = RECYCLER.get(); + synchronized (entryWrapper) { + entryWrapper.key = key; + entryWrapper.value = value; + entryWrapper.size = size; + } + return entryWrapper; } - K getKey() { + synchronized K getKey() { return key; } - V getValue() { + synchronized V getValue(K key) { + if (this.key != key) { + return null; + } return value; } + synchronized long getSize() { + return size; + } + void recycle() { + key = null; value = null; + size = 0; recyclerHandle.recycle(this); } - - @Override - public boolean equals(Object o) { - // only match exact identity of the value - return this == o; - } - - @Override - public int hashCode() { - return Objects.hashCode(key); - } } /** @@ -181,9 +187,10 @@ public boolean put(Key key, Value value) { if (!value.matchesKey(key)) { throw new IllegalArgumentException("Value '" + value + "' does not match key '" + key + "'"); } - IdentityWrapper newWrapper = IdentityWrapper.create(key, value); + long entrySize = weighter.getSize(value); + EntryWrapper newWrapper = EntryWrapper.create(key, value, entrySize); if (entries.putIfAbsent(key, newWrapper) == null) { - size.addAndGet(weighter.getSize(value)); + this.size.addAndGet(entrySize); return true; } else { // recycle the new wrapper as it was not used @@ -207,15 +214,15 @@ public Value get(Key key) { return getValue(key, entries.get(key)); } - private Value getValue(Key key, IdentityWrapper valueWrapper) { + private Value getValue(Key key, EntryWrapper valueWrapper) { if (valueWrapper == null) { return null; } else { - if (valueWrapper.getKey() != key) { + Value value = valueWrapper.getValue(key); + if (value == null) { // the wrapper has been recycled and contains another key return null; } - Value value = valueWrapper.getValue(); try { value.retain(); } catch (IllegalReferenceCountException e) { @@ -247,7 +254,7 @@ public Collection getRange(Key first, Key last) { List values = new ArrayList(); // Return the values of the entries found in cache - for (Map.Entry> entry : entries.subMap(first, true, last, true).entrySet()) { + for (Map.Entry> entry : entries.subMap(first, true, last, true).entrySet()) { Value value = getValue(entry.getKey(), entry.getValue()); if (value != null) { values.add(value); @@ -266,9 +273,9 @@ public Collection getRange(Key first, Key last) { */ public Pair removeRange(Key first, Key last, boolean lastInclusive) { RemovalCounters counters = RemovalCounters.create(); - Map> subMap = entries.subMap(first, true, last, lastInclusive); - for (Map.Entry> entry : subMap.entrySet()) { - removeEntry(entry, counters); + Map> subMap = entries.subMap(first, true, last, lastInclusive); + for (Map.Entry> entry : subMap.entrySet()) { + removeEntry(entry, counters, true); } return handleRemovalResult(counters); } @@ -279,36 +286,76 @@ enum RemoveEntryResult { BREAK_LOOP; } - private RemoveEntryResult removeEntry(Map.Entry> entry, RemovalCounters counters) { - return removeEntry(entry, counters, (x) -> true); + 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, - Predicate removeCondition) { + private RemoveEntryResult removeEntry(Map.Entry> entry, RemovalCounters counters, + boolean skipInvalid, Predicate removeCondition) { Key key = entry.getKey(); - IdentityWrapper identityWrapper = entry.getValue(); - if (identityWrapper.getKey() != key) { - // the wrapper has been recycled and contains another key + EntryWrapper entryWrapper = entry.getValue(); + Value value = entryWrapper.getValue(key); + if (value == null) { + // the wrapper has already been recycled and contains another key + if (!skipInvalid) { + EntryWrapper removed = entries.remove(key); + if (removed != null) { + // log and remove the entry without releasing the value + log.info("Key {} does not match the entry's value wrapper's key {}, removed entry by key without " + + "releasing the value", key, entryWrapper.getKey()); + counters.entryRemoved(removed.getSize()); + return RemoveEntryResult.ENTRY_REMOVED; + } + } return RemoveEntryResult.CONTINUE_LOOP; } - Value value = identityWrapper.getValue(); try { // add extra retain to avoid value being released while we are removing it value.retain(); } catch (IllegalReferenceCountException e) { // Value was already released + if (!skipInvalid) { + // remove the specific entry without releasing the value + if (entries.remove(key, entryWrapper)) { + log.info("Value was already released for key {}, removed entry without releasing the value", key); + counters.entryRemoved(entryWrapper.getSize()); + return RemoveEntryResult.ENTRY_REMOVED; + } + } return RemoveEntryResult.CONTINUE_LOOP; } + if (!value.matchesKey(key)) { + // this is unexpected since the IdentityWrapper.getValue(key) already checked that the value matches the key + log.warn("Unexpected race condition. Value {} does not match the key {}. Removing entry.", value, key); + } try { if (!removeCondition.test(value)) { return RemoveEntryResult.BREAK_LOOP; } - // 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 - // it is valid that the value contains references even after the key has been removed from the cache - if (value.refCnt() > 1 && value.matchesKey(key) && entries.remove(key, identityWrapper)) { - identityWrapper.recycle(); - counters.entryRemoved(weighter.getSize(value)); + if (!skipInvalid) { + // remove the specific entry + boolean entryRemoved = entries.remove(key, entryWrapper); + if (entryRemoved) { + counters.entryRemoved(entryWrapper.getSize()); + // check that the value hasn't been recycled in between + // there should be at least 2 references since this method adds one and the cache should have + // one reference. it is valid that the value contains references even after the key has been + // removed from the cache + if (value.refCnt() > 1) { + entryWrapper.recycle(); + // remove the cache reference + value.release(); + } else { + log.info("Unexpected refCnt {} for key {}, removed entry without releasing the value", + value.refCnt(), key); + } + } + } else if (skipInvalid && value.refCnt() > 1 && entries.remove(key, entryWrapper)) { + // when skipInvalid is true, we don't remove the entry if it doesn't match matches the key + // or the refCnt is invalid + counters.entryRemoved(entryWrapper.getSize()); + entryWrapper.recycle(); // remove the cache reference value.release(); } @@ -334,12 +381,12 @@ private Pair handleRemovalResult(RemovalCounters counters) { public Pair evictLeastAccessedEntries(long minSize) { checkArgument(minSize > 0); RemovalCounters counters = RemovalCounters.create(); - while (counters.removedSize < minSize) { - Map.Entry> entry = entries.firstEntry(); + while (counters.removedSize < minSize && !Thread.currentThread().isInterrupted()) { + Map.Entry> entry = entries.firstEntry(); if (entry == null) { break; } - removeEntry(entry, counters); + removeEntry(entry, counters, false); } return handleRemovalResult(counters); } @@ -351,12 +398,12 @@ public Pair evictLeastAccessedEntries(long minSize) { */ public Pair evictLEntriesBeforeTimestamp(long maxTimestamp) { RemovalCounters counters = RemovalCounters.create(); - while (true) { - Map.Entry> entry = entries.firstEntry(); + while (!Thread.currentThread().isInterrupted()) { + Map.Entry> entry = entries.firstEntry(); if (entry == null) { break; } - if (removeEntry(entry, counters, value -> timestampExtractor.getTimestamp(value) <= maxTimestamp) + if (removeEntry(entry, counters, false, value -> timestampExtractor.getTimestamp(value) <= maxTimestamp) == RemoveEntryResult.BREAK_LOOP) { break; } @@ -382,12 +429,12 @@ public long getSize() { */ public Pair clear() { RemovalCounters counters = RemovalCounters.create(); - while (true) { - Map.Entry> entry = entries.firstEntry(); + while (!Thread.currentThread().isInterrupted()) { + Map.Entry> entry = entries.firstEntry(); if (entry == null) { break; } - removeEntry(entry, counters); + removeEntry(entry, counters, false); } return handleRemovalResult(counters); } @@ -421,5 +468,4 @@ public long getSize(Value value) { return 1; } } - } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java index 1b02cd674c567..1ab3198498ac3 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java @@ -193,9 +193,9 @@ public void cacheSizeUpdate() throws Exception { } cacheManager.removeEntryCache(ml1.getName()); - assertTrue(cacheManager.getSize() > 0); assertEquals(factory2.getMbean().getCacheInsertedEntriesCount(), 20); assertEquals(factory2.getMbean().getCacheEntriesCount(), 0); + assertEquals(0, cacheManager.getSize()); assertEquals(factory2.getMbean().getCacheEvictedEntriesCount(), 20); } 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 01b3c67bf1113..4bcf2cc6c4e35 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 @@ -30,11 +30,14 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import lombok.Cleanup; +import lombok.Data; import org.apache.commons.lang3.tuple.Pair; +import org.awaitility.Awaitility; import org.testng.annotations.Test; public class RangeCacheTest { + @Data class RefString extends AbstractReferenceCounted implements RangeCache.ValueWithKeyValidation { String s; Integer matchingKey; @@ -288,15 +291,21 @@ public void evictions() { @Test public void testPutWhileClearIsCalledConcurrently() { RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); - int numberOfThreads = 4; + int numberOfThreads = 8; @Cleanup("shutdownNow") ScheduledExecutorService executor = Executors.newScheduledThreadPool(numberOfThreads); for (int i = 0; i < numberOfThreads; i++) { executor.scheduleWithFixedDelay(cache::clear, 0, 1, TimeUnit.MILLISECONDS); } - for (int i = 0; i < 100000; i++) { + for (int i = 0; i < 200000; i++) { cache.put(i, new RefString(String.valueOf(i))); } + executor.shutdown(); + // ensure that no clear operation got into endless loop + Awaitility.await().untilAsserted(() -> assertTrue(executor.isTerminated())); + // ensure that clear can be called and all entries are removed + cache.clear(); + assertEquals(cache.getNumberOfEntries(), 0); } @Test @@ -307,4 +316,26 @@ public void testPutSameObj() { assertTrue(cache.put(0, s0)); assertFalse(cache.put(0, s0)); } + + @Test + public void testRemoveEntryWithInvalidRefCount() { + RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); + RefString value = new RefString("1"); + cache.put(1, value); + // release the value to make the reference count invalid + value.release(); + cache.clear(); + assertEquals(cache.getNumberOfEntries(), 0); + } + + @Test + public void testRemoveEntryWithInvalidMatchingKey() { + RangeCache cache = new RangeCache<>(value -> value.s.length(), x -> 0); + RefString value = new RefString("1"); + cache.put(1, value); + // change the matching key to make it invalid + value.setMatchingKey(123); + cache.clear(); + assertEquals(cache.getNumberOfEntries(), 0); + } } From 12971bcc13afa55b402184e62d50d505a4aed0f4 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 3 Jun 2024 19:40:50 +0300 Subject: [PATCH 533/699] [improve][ml] RangeCache refactoring follow-up: use StampedLock instead of synchronized (#22818) (cherry picked from commit 208d8a53be20a95613703f0a11527ca9735e7bf8) (cherry picked from commit 62f674a7ea2c20111ff93fc6949eb3a9bad736f4) --- .../bookkeeper/mledger/util/RangeCache.java | 49 ++++++++++++++----- 1 file changed, 37 insertions(+), 12 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 45295d7190654..2f2b161a30684 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 @@ -31,6 +31,7 @@ import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.StampedLock; import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.mledger.util.RangeCache.ValueWithKeyValidation; import org.apache.commons.lang3.tuple.Pair; @@ -75,6 +76,7 @@ protected EntryWrapper newObject(Handle recyclerHandle) { return new EntryWrapper(recyclerHandle); } }; + private final StampedLock lock = new StampedLock(); private K key; private V value; long size; @@ -85,27 +87,50 @@ private EntryWrapper(Handle recyclerHandle) { static EntryWrapper create(K key, V value, long size) { EntryWrapper entryWrapper = RECYCLER.get(); - synchronized (entryWrapper) { - entryWrapper.key = key; - entryWrapper.value = value; - entryWrapper.size = size; - } + long stamp = entryWrapper.lock.writeLock(); + entryWrapper.key = key; + entryWrapper.value = value; + entryWrapper.size = size; + entryWrapper.lock.unlockWrite(stamp); return entryWrapper; } - synchronized K getKey() { - return key; + K getKey() { + long stamp = lock.tryOptimisticRead(); + K localKey = key; + if (!lock.validate(stamp)) { + stamp = lock.readLock(); + localKey = key; + lock.unlockRead(stamp); + } + return localKey; } - synchronized V getValue(K key) { - if (this.key != key) { + V getValue(K key) { + long stamp = lock.tryOptimisticRead(); + K localKey = this.key; + V localValue = this.value; + if (!lock.validate(stamp)) { + stamp = lock.readLock(); + localKey = this.key; + localValue = this.value; + lock.unlockRead(stamp); + } + if (localKey != key) { return null; } - return value; + return localValue; } - synchronized long getSize() { - return size; + long getSize() { + long stamp = lock.tryOptimisticRead(); + long localSize = size; + if (!lock.validate(stamp)) { + stamp = lock.readLock(); + localSize = size; + lock.unlockRead(stamp); + } + return localSize; } void recycle() { From 7b426c611341c9535fd5a00bc96524886714e295 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 3 Jun 2024 19:49:02 +0300 Subject: [PATCH 534/699] [fix][sec] Upgrade Bouncycastle libraries to address CVEs (#22826) (cherry picked from commit 05d98f7b07b6e3ac249845f042bfa937d1744f42) (cherry picked from commit fd4df2ccd55be9be76e8e6497e6e0597b51a31d8) --- bouncy-castle/bc/LICENSE | 5 ++--- distribution/server/src/assemble/LICENSE.bin.txt | 7 +++---- distribution/shell/src/assemble/LICENSE.bin.txt | 7 +++---- pom.xml | 6 +++--- pulsar-sql/presto-distribution/LICENSE | 7 +++---- 5 files changed, 14 insertions(+), 18 deletions(-) diff --git a/bouncy-castle/bc/LICENSE b/bouncy-castle/bc/LICENSE index 14f4e76e921d3..c95d33d3d1ffb 100644 --- a/bouncy-castle/bc/LICENSE +++ b/bouncy-castle/bc/LICENSE @@ -205,6 +205,5 @@ This projects includes binary packages with the following licenses: Bouncy Castle License * Bouncy Castle -- licenses/LICENSE-bouncycastle.txt - - org.bouncycastle-bcpkix-jdk18on-1.78.jar - - org.bouncycastle-bcprov-jdk18on-1.78.jar - - org.bouncycastle-bcprov-ext-jdk18on-1.78.jar + - org.bouncycastle-bcpkix-jdk18on-1.78.1.jar + - org.bouncycastle-bcprov-jdk18on-1.78.1.jar diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index d3dc693f9131e..377f3a509a30d 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -579,10 +579,9 @@ Creative Commons Attribution License Bouncy Castle License * Bouncy Castle -- ../licenses/LICENSE-bouncycastle.txt - - org.bouncycastle-bcpkix-jdk18on-1.78.jar - - org.bouncycastle-bcprov-ext-jdk18on-1.78.jar - - org.bouncycastle-bcprov-jdk18on-1.78.jar - - org.bouncycastle-bcutil-jdk18on-1.78.jar + - org.bouncycastle-bcpkix-jdk18on-1.78.1.jar + - org.bouncycastle-bcprov-jdk18on-1.78.1.jar + - org.bouncycastle-bcutil-jdk18on-1.78.1.jar Datastax - com.datastax.oss-managed-ledger-3.1.4.2-SNAPSHOT.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 96d515c2da165..7c9b0d2d5de43 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -466,10 +466,9 @@ Creative Commons Attribution License Bouncy Castle License * Bouncy Castle -- ../licenses/LICENSE-bouncycastle.txt - - bcpkix-jdk18on-1.78.jar - - bcprov-ext-jdk18on-1.78.jar - - bcprov-jdk18on-1.78.jar - - bcutil-jdk18on-1.78.jar + - bcpkix-jdk18on-1.78.1.jar + - bcprov-jdk18on-1.78.1.jar + - bcutil-jdk18on-1.78.1.jar ------------------------ diff --git a/pom.xml b/pom.xml index 536238a04e9bb..ce39a16c94184 100644 --- a/pom.xml +++ b/pom.xml @@ -155,9 +155,9 @@ flexible messaging model and an intuitive client API. 1.7.32 4.4 2.18.0 - 1.78 - 1.0.6 - 1.0.2.4 + 1.78.1 + 1.0.7 + 1.0.2.5 2.14.2 0.10.2 1.6.2 diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 03187ff390064..81a295608eb5f 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -588,7 +588,6 @@ Creative Commons Attribution License Bouncy Castle License * Bouncy Castle -- licenses/LICENSE-bouncycastle.txt - - bcpkix-jdk18on-1.78.jar - - bcprov-ext-jdk18on-1.78.jar - - bcprov-jdk18on-1.78.jar - - bcutil-jdk18on-1.78.jar + - bcpkix-jdk18on-1.78.1.jar + - bcprov-jdk18on-1.78.1.jar + - bcutil-jdk18on-1.78.1.jar From 4b6e6b59cc92bbda7b03ad3ba592f06ce31ae723 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 3 Jun 2024 11:07:24 -0700 Subject: [PATCH 535/699] [fix] Bump io.airlift:aircompressor from 0.20 to 0.27 (#22819) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Matteo Merli (cherry picked from commit 1a7b72fc21961ffefa3c56af41709e6afbf13bc5) --- 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 377f3a509a30d..b11b4c0c44667 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -377,7 +377,7 @@ The Apache Software License, Version 2.0 - org.apache.httpcomponents-httpclient-4.5.13.jar - org.apache.httpcomponents-httpcore-4.4.15.jar * AirCompressor - - io.airlift-aircompressor-0.20.jar + - 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 diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 7c9b0d2d5de43..ff69efeda4644 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -390,7 +390,7 @@ The Apache Software License, Version 2.0 - cpu-affinity-4.16.5.jar - circe-checksum-4.16.5.jar * AirCompressor - - aircompressor-0.20.jar + - aircompressor-0.27.jar * AsyncHttpClient - async-http-client-2.12.1.jar - async-http-client-netty-utils-2.12.1.jar diff --git a/pom.xml b/pom.xml index ce39a16c94184..589eeba115198 100644 --- a/pom.xml +++ b/pom.xml @@ -211,7 +211,7 @@ flexible messaging model and an intuitive client API. 1.0 0.16.1 6.2.8 - 0.20 + 0.27 2.12.1 1.82 3.11 From ec6af443d546774c5770c9e3eee5bf6ad404bcb4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Tue, 12 Mar 2024 23:36:59 +0800 Subject: [PATCH 536/699] [cleanup][ml] ManagedCursor clean up. (#22246) (cherry picked from commit 532b0d9063474bd1c7ae8ac7cf5bd2d56b002164) (cherry picked from commit d9d67daa40442d156a22ff119520e9c5832887b7) --- .../org/apache/bookkeeper/mledger/impl/EntryImpl.java | 7 ++++++- .../bookkeeper/mledger/impl/ManagedCursorImpl.java | 11 +++-------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java index e53f408ca7563..48a79a4ac529c 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java @@ -43,6 +43,7 @@ protected EntryImpl newObject(Handle handle) { private long timestamp; private long ledgerId; private long entryId; + private PositionImpl position; ByteBuf data; private Runnable onDeallocate; @@ -152,7 +153,10 @@ public int getLength() { @Override public PositionImpl getPosition() { - return new PositionImpl(ledgerId, entryId); + if (position == null) { + position = PositionImpl.get(ledgerId, entryId); + } + return position; } @Override @@ -198,6 +202,7 @@ protected void deallocate() { timestamp = -1; ledgerId = -1; entryId = -1; + position = null; recyclerHandle.recycle(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 3451167affbb6..2df37a0b6ce9f 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 @@ -1506,10 +1506,7 @@ public Set asyncReplayEntries(Set positi Set alreadyAcknowledgedPositions = new HashSet<>(); lock.readLock().lock(); try { - positions.stream() - .filter(position -> ((PositionImpl) position).compareTo(markDeletePosition) <= 0 - || individualDeletedMessages.contains(position.getLedgerId(), position.getEntryId())) - .forEach(alreadyAcknowledgedPositions::add); + positions.stream().filter(this::isMessageDeleted).forEach(alreadyAcknowledgedPositions::add); } finally { lock.readLock().unlock(); } @@ -2281,8 +2278,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb return; } - if (position.compareTo(markDeletePosition) <= 0 - || individualDeletedMessages.contains(position.getLedgerId(), position.getEntryId())) { + if (isMessageDeleted(position)) { if (config.isDeletionAtBatchIndexLevelEnabled()) { BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.remove(position); if (bitSetRecyclable != null) { @@ -3541,8 +3537,7 @@ public Range getLastIndividualDeletedRange() { @Override public void trimDeletedEntries(List entries) { entries.removeIf(entry -> { - boolean isDeleted = markDeletePosition.compareTo(entry.getLedgerId(), entry.getEntryId()) >= 0 - || individualDeletedMessages.contains(entry.getLedgerId(), entry.getEntryId()); + boolean isDeleted = isMessageDeleted(entry.getPosition()); if (isDeleted) { entry.release(); } From 90bd00d24a7ac7041eab345d275bc899d5ecae20 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Fri, 10 May 2024 10:37:44 +0800 Subject: [PATCH 537/699] [fix][broker] Fix cursor should use latest ledger config (#22644) Signed-off-by: Zixuan Liu (cherry picked from commit 774a5d42e8342ee50395cf3626b9e7af27da849e) (cherry picked from commit 919cfebaf20b841b756d600a7418d5265e09a861) --- .../mledger/impl/ManagedCursorImpl.java | 59 +++++++++---------- .../mledger/impl/ManagedCursorMXBeanImpl.java | 3 +- .../mledger/impl/ManagedLedgerImpl.java | 8 +-- .../mledger/impl/NonDurableCursorImpl.java | 5 +- .../bookkeeper/mledger/impl/OpReadEntry.java | 3 +- .../mledger/impl/RangeSetWrapper.java | 2 +- .../mledger/impl/ReadOnlyCursorImpl.java | 5 +- .../impl/ReadOnlyManagedLedgerImpl.java | 2 +- ...edCursorIndividualDeletedMessagesTest.java | 3 +- .../mledger/impl/ManagedCursorTest.java | 7 +-- .../mledger/impl/ManagedLedgerTest.java | 2 +- .../service/BrokerBkEnsemblesTests.java | 8 +-- .../persistent/PersistentTopicTest.java | 25 ++++++++ 13 files changed, 76 insertions(+), 56 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 2df37a0b6ce9f..52d4e803f0bb9 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 @@ -119,7 +119,6 @@ public class ManagedCursorImpl implements ManagedCursor { return 0; }; protected final BookKeeper bookkeeper; - protected final ManagedLedgerConfig config; protected final ManagedLedgerImpl ledger; private final String name; @@ -299,31 +298,30 @@ public interface VoidCallback { void operationFailed(ManagedLedgerException exception); } - ManagedCursorImpl(BookKeeper bookkeeper, ManagedLedgerConfig config, ManagedLedgerImpl ledger, String cursorName) { + ManagedCursorImpl(BookKeeper bookkeeper, ManagedLedgerImpl ledger, String cursorName) { this.bookkeeper = bookkeeper; this.cursorProperties = Collections.emptyMap(); - this.config = config; this.ledger = ledger; this.name = cursorName; this.individualDeletedMessages = new RangeSetWrapper<>(positionRangeConverter, positionRangeReverseConverter, this); - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { this.batchDeletedIndexes = new ConcurrentSkipListMap<>(); } else { this.batchDeletedIndexes = null; } - this.digestType = BookKeeper.DigestType.fromApiDigestType(config.getDigestType()); + this.digestType = BookKeeper.DigestType.fromApiDigestType(getConfig().getDigestType()); STATE_UPDATER.set(this, State.Uninitialized); PENDING_MARK_DELETED_SUBMITTED_COUNT_UPDATER.set(this, 0); PENDING_READ_OPS_UPDATER.set(this, 0); RESET_CURSOR_IN_PROGRESS_UPDATER.set(this, FALSE); WAITING_READ_OP_UPDATER.set(this, null); - this.clock = config.getClock(); + this.clock = getConfig().getClock(); this.lastActive = this.clock.millis(); this.lastLedgerSwitchTimestamp = this.clock.millis(); - if (config.getThrottleMarkDelete() > 0.0) { - markDeleteLimiter = RateLimiter.create(config.getThrottleMarkDelete()); + if (getConfig().getThrottleMarkDelete() > 0.0) { + markDeleteLimiter = RateLimiter.create(getConfig().getThrottleMarkDelete()); } else { // Disable mark-delete rate limiter markDeleteLimiter = null; @@ -602,7 +600,7 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac if (positionInfo.getIndividualDeletedMessagesCount() > 0) { recoverIndividualDeletedMessages(positionInfo.getIndividualDeletedMessagesList()); } - if (config.isDeletionAtBatchIndexLevelEnabled() + if (getConfig().isDeletionAtBatchIndexLevelEnabled() && positionInfo.getBatchedEntryDeletionIndexInfoCount() > 0) { recoverBatchDeletedIndexes(positionInfo.getBatchedEntryDeletionIndexInfoList()); } @@ -611,7 +609,8 @@ protected void recoverFromLedger(final ManagedCursorInfo info, final VoidCallbac }, null); }; try { - bookkeeper.asyncOpenLedger(ledgerId, digestType, config.getPassword(), openCallback, null); + bookkeeper.asyncOpenLedger(ledgerId, digestType, getConfig().getPassword(), openCallback, + null); } catch (Throwable t) { log.error("[{}] Encountered error on opening cursor ledger {} for cursor {}", ledger.getName(), ledgerId, name, t); @@ -968,10 +967,10 @@ public void asyncReadEntriesWithSkipOrWait(int maxEntries, long maxSizeBytes, Re // Check again for new entries after the configured time, then if still no entries are available register // to be notified - if (config.getNewEntriesCheckDelayInMillis() > 0) { + if (getConfig().getNewEntriesCheckDelayInMillis() > 0) { ledger.getScheduledExecutor() .schedule(() -> checkForNewEntries(op, callback, ctx), - config.getNewEntriesCheckDelayInMillis(), TimeUnit.MILLISECONDS); + getConfig().getNewEntriesCheckDelayInMillis(), TimeUnit.MILLISECONDS); } else { // If there's no delay, check directly from the same thread checkForNewEntries(op, callback, ctx); @@ -1319,7 +1318,7 @@ public void operationComplete() { lastMarkDeleteEntry = new MarkDeleteEntry(newMarkDeletePosition, isCompactionCursor() ? getProperties() : Collections.emptyMap(), null, null); individualDeletedMessages.clear(); - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { batchDeletedIndexes.values().forEach(BitSetRecyclable::recycle); batchDeletedIndexes.clear(); long[] resetWords = newReadPosition.ackSet; @@ -1578,7 +1577,7 @@ protected long getNumberOfEntries(Range range) { lock.readLock().lock(); try { - if (config.isUnackedRangesOpenCacheSetEnabled()) { + if (getConfig().isUnackedRangesOpenCacheSetEnabled()) { int cardinality = individualDeletedMessages.cardinality( range.lowerEndpoint().ledgerId, range.lowerEndpoint().entryId, range.upperEndpoint().ledgerId, range.upperEndpoint().entryId); @@ -1958,7 +1957,7 @@ public void asyncMarkDelete(final Position position, Map propertie PositionImpl newPosition = (PositionImpl) position; - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { if (newPosition.ackSet != null) { AtomicReference bitSetRecyclable = new AtomicReference<>(); BitSetRecyclable givenBitSet = BitSetRecyclable.create().resetWords(newPosition.ackSet); @@ -2141,7 +2140,7 @@ public void operationComplete() { try { individualDeletedMessages.removeAtMost(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()); - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { Map subMap = batchDeletedIndexes.subMap(PositionImpl.EARLIEST, false, PositionImpl.get(mdEntry.newPosition.getLedgerId(), mdEntry.newPosition.getEntryId()), true); @@ -2279,7 +2278,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb } if (isMessageDeleted(position)) { - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.remove(position); if (bitSetRecyclable != null) { bitSetRecyclable.recycle(); @@ -2291,7 +2290,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb continue; } if (position.ackSet == null) { - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { BitSetRecyclable bitSetRecyclable = batchDeletedIndexes.remove(position); if (bitSetRecyclable != null) { bitSetRecyclable.recycle(); @@ -2308,7 +2307,7 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb log.debug("[{}] [{}] Individually deleted messages: {}", ledger.getName(), name, individualDeletedMessages); } - } else if (config.isDeletionAtBatchIndexLevelEnabled()) { + } else if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { BitSetRecyclable givenBitSet = BitSetRecyclable.create().resetWords(position.ackSet); BitSetRecyclable bitSet = batchDeletedIndexes.computeIfAbsent(position, (v) -> givenBitSet); if (givenBitSet != bitSet) { @@ -2655,8 +2654,8 @@ public void operationFailed(MetaStoreException e) { private boolean shouldPersistUnackRangesToLedger() { return cursorLedger != null && !isCursorLedgerReadOnly - && config.getMaxUnackedRangesToPersist() > 0 - && individualDeletedMessages.size() > config.getMaxUnackedRangesToPersistInMetadataStore(); + && getConfig().getMaxUnackedRangesToPersist() > 0 + && individualDeletedMessages.size() > getConfig().getMaxUnackedRangesToPersistInMetadataStore(); } private void persistPositionMetaStore(long cursorsLedgerId, PositionImpl position, Map properties, @@ -2681,7 +2680,7 @@ private void persistPositionMetaStore(long cursorsLedgerId, PositionImpl positio info.addAllCursorProperties(buildStringPropertiesMap(cursorProperties)); if (persistIndividualDeletedMessageRanges) { info.addAllIndividualDeletedMessages(buildIndividualDeletedMessageRanges()); - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { info.addAllBatchedEntryDeletionIndexInfo(buildBatchEntryDeletionIndexInfoList()); } } @@ -2946,7 +2945,7 @@ public void operationFailed(ManagedLedgerException exception) { private CompletableFuture doCreateNewMetadataLedger() { CompletableFuture future = new CompletableFuture<>(); - ledger.asyncCreateLedger(bookkeeper, config, digestType, (rc, lh, ctx) -> { + ledger.asyncCreateLedger(bookkeeper, getConfig(), digestType, (rc, lh, ctx) -> { if (ledger.checkAndCompleteLedgerOpTask(rc, lh, ctx)) { future.complete(null); @@ -3051,7 +3050,7 @@ private List buildIndividualDeletedMessageRanges() { acksSerializedSize.addAndGet(messageRange.getSerializedSize()); rangeList.add(messageRange); - return rangeList.size() <= config.getMaxUnackedRangesToPersist(); + return rangeList.size() <= getConfig().getMaxUnackedRangesToPersist(); }); this.individualDeletedMessagesSerializedSize = acksSerializedSize.get(); @@ -3065,7 +3064,7 @@ private List buildIndividualDeletedMessageRanges() { private List buildBatchEntryDeletionIndexInfoList() { lock.readLock().lock(); try { - if (!config.isDeletionAtBatchIndexLevelEnabled() || batchDeletedIndexes.isEmpty()) { + if (!getConfig().isDeletionAtBatchIndexLevelEnabled() || batchDeletedIndexes.isEmpty()) { return Collections.emptyList(); } MLDataFormats.NestedPositionInfo.Builder nestedPositionBuilder = MLDataFormats.NestedPositionInfo @@ -3074,7 +3073,7 @@ private List buildBatchEntryDeletio .BatchedEntryDeletionIndexInfo.newBuilder(); List result = new ArrayList<>(); Iterator> iterator = batchDeletedIndexes.entrySet().iterator(); - while (iterator.hasNext() && result.size() < config.getMaxBatchDeletedIndexToPersist()) { + while (iterator.hasNext() && result.size() < getConfig().getMaxBatchDeletedIndexToPersist()) { Map.Entry entry = iterator.next(); nestedPositionBuilder.setLedgerId(entry.getKey().getLedgerId()); nestedPositionBuilder.setEntryId(entry.getKey().getEntryId()); @@ -3194,8 +3193,8 @@ public void operationFailed(MetaStoreException e) { boolean shouldCloseLedger(LedgerHandle lh) { long now = clock.millis(); if (ledger.getFactory().isMetadataServiceAvailable() - && (lh.getLastAddConfirmed() >= config.getMetadataMaxEntriesPerLedger() - || lastLedgerSwitchTimestamp < (now - config.getLedgerRolloverTimeout() * 1000)) + && (lh.getLastAddConfirmed() >= getConfig().getMetadataMaxEntriesPerLedger() + || lastLedgerSwitchTimestamp < (now - getConfig().getLedgerRolloverTimeout() * 1000)) && (STATE_UPDATER.get(this) != State.Closed && STATE_UPDATER.get(this) != State.Closing)) { // It's safe to modify the timestamp since this method will be only called from a callback, implying that // calls will be serialized on one single thread @@ -3551,7 +3550,7 @@ private ManagedCursorImpl cursorImpl() { @Override public long[] getDeletedBatchIndexesAsLongArray(PositionImpl position) { - if (config.isDeletionAtBatchIndexLevelEnabled()) { + if (getConfig().isDeletionAtBatchIndexLevelEnabled()) { BitSetRecyclable bitSet = batchDeletedIndexes.get(position); return bitSet == null ? null : bitSet.toLongArray(); } else { @@ -3652,7 +3651,7 @@ public boolean isCacheReadEntry() { private static final Logger log = LoggerFactory.getLogger(ManagedCursorImpl.class); public ManagedLedgerConfig getConfig() { - return config; + return getManagedLedger().getConfig(); } /*** diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorMXBeanImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorMXBeanImpl.java index 48465e6294b0e..a183c0d61ce16 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorMXBeanImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorMXBeanImpl.java @@ -90,7 +90,8 @@ public long getPersistZookeeperErrors() { @Override public void addWriteCursorLedgerSize(final long size) { - writeCursorLedgerSize.add(size * ((ManagedCursorImpl) managedCursor).config.getWriteQuorumSize()); + writeCursorLedgerSize.add( + size * managedCursor.getManagedLedger().getConfig().getWriteQuorumSize()); writeCursorLedgerLogicalSize.add(size); } 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 4cc883f140a46..58d20b2e62e81 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 @@ -585,7 +585,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, config, ManagedLedgerImpl.this, cursorName); + cursor = new ManagedCursorImpl(bookKeeper, ManagedLedgerImpl.this, cursorName); cursor.recover(new VoidCallback() { @Override @@ -616,7 +616,7 @@ public void operationFailed(ManagedLedgerException exception) { log.debug("[{}] Recovering cursor {} lazily", name, cursorName); } final ManagedCursorImpl cursor; - cursor = new ManagedCursorImpl(bookKeeper, config, ManagedLedgerImpl.this, cursorName); + cursor = new ManagedCursorImpl(bookKeeper, ManagedLedgerImpl.this, cursorName); CompletableFuture cursorRecoveryFuture = new CompletableFuture<>(); uninitializedCursors.put(cursorName, cursorRecoveryFuture); @@ -1001,7 +1001,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, config, this, cursorName); + final ManagedCursorImpl cursor = new ManagedCursorImpl(bookKeeper, this, cursorName); CompletableFuture cursorFuture = new CompletableFuture<>(); uninitializedCursors.put(cursorName, cursorFuture); PositionImpl position = InitialPosition.Earliest == initialPosition ? getFirstPosition() : getLastPosition(); @@ -1134,7 +1134,7 @@ public ManagedCursor newNonDurableCursor(Position startCursorPosition, String cu return cachedCursor; } - NonDurableCursorImpl cursor = new NonDurableCursorImpl(bookKeeper, config, this, cursorName, + NonDurableCursorImpl cursor = new NonDurableCursorImpl(bookKeeper, this, cursorName, (PositionImpl) startCursorPosition, initialPosition, isReadCompacted); cursor.setActive(); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java index 77216ce2e4588..734eab20bc58e 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorImpl.java @@ -25,7 +25,6 @@ import org.apache.bookkeeper.mledger.AsyncCallbacks.CloseCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCursorCallback; import org.apache.bookkeeper.mledger.AsyncCallbacks.MarkDeleteCallback; -import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.common.api.proto.CommandSubscribe; import org.slf4j.Logger; @@ -35,10 +34,10 @@ public class NonDurableCursorImpl extends ManagedCursorImpl { private final boolean readCompacted; - NonDurableCursorImpl(BookKeeper bookkeeper, ManagedLedgerConfig config, ManagedLedgerImpl ledger, String cursorName, + NonDurableCursorImpl(BookKeeper bookkeeper, ManagedLedgerImpl ledger, String cursorName, PositionImpl startCursorPosition, CommandSubscribe.InitialPosition initialPosition, boolean isReadCompacted) { - super(bookkeeper, config, ledger, cursorName); + super(bookkeeper, ledger, cursorName); this.readCompacted = isReadCompacted; // Compare with "latest" position marker by using only the ledger id. Since the C++ client is using 48bits to 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 a79ba3fb5e23b..534ef3d76cb0d 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 @@ -111,7 +111,8 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { callback.readEntriesComplete(entries, ctx); recycle(); }); - } else if (cursor.config.isAutoSkipNonRecoverableData() && exception instanceof NonRecoverableLedgerException) { + } else if (cursor.getConfig().isAutoSkipNonRecoverableData() + && exception instanceof NonRecoverableLedgerException) { log.warn("[{}][{}] read failed from ledger at position:{} : {}", cursor.ledger.getName(), cursor.getName(), readPosition, exception.getMessage()); final ManagedLedgerImpl ledger = (ManagedLedgerImpl) cursor.getManagedLedger(); 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 02e43504482d8..f235ffc63ace5 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 @@ -52,7 +52,7 @@ public RangeSetWrapper(LongPairConsumer rangeConverter, RangeBoundConsumer rangeBoundConsumer, ManagedCursorImpl managedCursor) { requireNonNull(managedCursor); - this.config = managedCursor.getConfig(); + this.config = managedCursor.getManagedLedger().getConfig(); this.rangeConverter = rangeConverter; this.rangeSet = config.isUnackedRangesOpenCacheSetEnabled() ? new ConcurrentOpenLongPairRangeSet<>(4096, rangeConverter) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java index 1661613f07d7d..2461bcf780e99 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyCursorImpl.java @@ -22,7 +22,6 @@ import lombok.extern.slf4j.Slf4j; import org.apache.bookkeeper.client.BookKeeper; import org.apache.bookkeeper.mledger.AsyncCallbacks; -import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.ReadOnlyCursor; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.PositionBound; @@ -31,9 +30,9 @@ @Slf4j public class ReadOnlyCursorImpl extends ManagedCursorImpl implements ReadOnlyCursor { - public ReadOnlyCursorImpl(BookKeeper bookkeeper, ManagedLedgerConfig config, ManagedLedgerImpl ledger, + public ReadOnlyCursorImpl(BookKeeper bookkeeper, ManagedLedgerImpl ledger, PositionImpl startPosition, String cursorName) { - super(bookkeeper, config, ledger, cursorName); + super(bookkeeper, ledger, cursorName); if (startPosition.equals(PositionImpl.EARLIEST)) { readPosition = ledger.getFirstPosition().getNext(); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java index 707b71c9d9f09..d844963599995 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ReadOnlyManagedLedgerImpl.java @@ -143,7 +143,7 @@ ReadOnlyCursor createReadOnlyCursor(PositionImpl startPosition) { } } - return new ReadOnlyCursorImpl(bookKeeper, config, this, startPosition, "read-only-cursor"); + return new ReadOnlyCursorImpl(bookKeeper, this, startPosition, "read-only-cursor"); } @Override diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorIndividualDeletedMessagesTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorIndividualDeletedMessagesTest.java index aa0d04783d991..864c25c6c434b 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorIndividualDeletedMessagesTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedCursorIndividualDeletedMessagesTest.java @@ -56,8 +56,9 @@ void testRecoverIndividualDeletedMessages() throws Exception { ManagedLedgerImpl ledger = mock(ManagedLedgerImpl.class); doReturn(ledgersInfo).when(ledger).getLedgersInfo(); + doReturn(config).when(ledger).getConfig(); - ManagedCursorImpl cursor = spy(new ManagedCursorImpl(bookkeeper, config, ledger, "test-cursor")); + ManagedCursorImpl cursor = spy(new ManagedCursorImpl(bookkeeper, ledger, "test-cursor")); LongPairRangeSet deletedMessages = cursor.getIndividuallyDeletedMessagesSet(); Method recoverMethod = ManagedCursorImpl.class.getDeclaredMethod("recoverIndividualDeletedMessages", 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 5c10533e2476b..4c95454e33a92 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 @@ -3465,10 +3465,10 @@ public Object answer(InvocationOnMock invocation) { when(ml.getNextValidLedger(markDeleteLedgerId)).thenReturn(3L); when(ml.getNextValidPosition(lastPosition)).thenReturn(nextPosition); when(ml.ledgerExists(markDeleteLedgerId)).thenReturn(false); + when(ml.getConfig()).thenReturn(new ManagedLedgerConfig()); BookKeeper mockBookKeeper = mock(BookKeeper.class); - final ManagedCursorImpl cursor = new ManagedCursorImpl(mockBookKeeper, new ManagedLedgerConfig(), ml, - cursorName); + final ManagedCursorImpl cursor = new ManagedCursorImpl(mockBookKeeper, ml, cursorName); cursor.recover(new VoidCallback() { @Override @@ -4772,8 +4772,7 @@ public void testRecoverCursorWithTerminateManagedLedger() throws Exception { // Reopen the ledger. ledger = (ManagedLedgerImpl) factory.open(mlName, config); BookKeeper mockBookKeeper = mock(BookKeeper.class); - final ManagedCursorImpl cursor = new ManagedCursorImpl(mockBookKeeper, new ManagedLedgerConfig(), ledger, - cursorName); + final ManagedCursorImpl cursor = new ManagedCursorImpl(mockBookKeeper, ledger, cursorName); CompletableFuture recoverFuture = new CompletableFuture<>(); // Recover the cursor. 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 eca9c28a5ef6b..797dbe88a0196 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 @@ -3159,7 +3159,7 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { // (2) test read-timeout for: ManagedLedger.asyncReadEntry(..) AtomicReference responseException2 = new AtomicReference<>(); PositionImpl readPositionRef = PositionImpl.EARLIEST; - ManagedCursorImpl cursor = new ManagedCursorImpl(bk, config, ledger, "cursor1"); + ManagedCursorImpl cursor = new ManagedCursorImpl(bk, ledger, "cursor1"); OpReadEntry opReadEntry = OpReadEntry.create(cursor, readPositionRef, 1, new ReadEntriesCallback() { @Override 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 42b9358911a69..82892ad353aa1 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 @@ -210,10 +210,8 @@ public void testSkipCorruptDataLedger() throws Exception { PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topic1).get(); ManagedLedgerImpl ml = (ManagedLedgerImpl) topic.getManagedLedger(); ManagedCursorImpl cursor = (ManagedCursorImpl) ml.getCursors().iterator().next(); - Field configField = ManagedCursorImpl.class.getDeclaredField("config"); - configField.setAccessible(true); // Create multiple data-ledger - ManagedLedgerConfig config = (ManagedLedgerConfig) configField.get(cursor); + ManagedLedgerConfig config = ml.getConfig(); config.setMaxEntriesPerLedger(entriesPerLedger); config.setMinimumRolloverTime(1, TimeUnit.MILLISECONDS); // bookkeeper client @@ -323,10 +321,8 @@ public void testTruncateCorruptDataLedger() throws Exception { PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topic1).get(); ManagedLedgerImpl ml = (ManagedLedgerImpl) topic.getManagedLedger(); ManagedCursorImpl cursor = (ManagedCursorImpl) ml.getCursors().iterator().next(); - Field configField = ManagedCursorImpl.class.getDeclaredField("config"); - configField.setAccessible(true); // Create multiple data-ledger - ManagedLedgerConfig config = (ManagedLedgerConfig) configField.get(cursor); + ManagedLedgerConfig config = ml.getConfig(); config.setMaxEntriesPerLedger(entriesPerLedger); config.setMinimumRolloverTime(1, TimeUnit.MILLISECONDS); // bookkeeper client 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 d523586c2e2d3..5b750a0b9c2e5 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 @@ -66,6 +66,7 @@ import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.PrometheusMetricsTestUtil; @@ -754,6 +755,30 @@ public void testDynamicConfigurationAutoSkipNonRecoverableData() throws Exceptio admin.topics().delete(topicName); } + @Test + public void testCursorGetConfigAfterTopicPoliciesChanged() throws Exception { + final String topicName = "persistent://prop/ns-abc/" + UUID.randomUUID(); + final String subName = "test_sub"; + + @Cleanup + Consumer subscribe = pulsarClient.newConsumer().topic(topicName).subscriptionName(subName).subscribe(); + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).join().get(); + PersistentSubscription subscription = persistentTopic.getSubscription(subName); + + int maxConsumers = 100; + admin.topicPolicies().setMaxConsumers(topicName, 100); + Awaitility.await().untilAsserted(() -> { + assertEquals(admin.topicPolicies().getMaxConsumers(topicName, false), maxConsumers); + }); + + ManagedCursorImpl cursor = (ManagedCursorImpl) subscription.getCursor(); + assertEquals(cursor.getConfig(), persistentTopic.getManagedLedger().getConfig()); + + subscribe.close(); + admin.topics().delete(topicName); + } + @Test public void testAddWaitingCursorsForNonDurable() throws Exception { final String ns = "prop/ns-test"; From 2c34c2c4d2092e8bda7d0a67a5c034a0a3b073d3 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 30 May 2024 16:42:26 +0800 Subject: [PATCH 538/699] [improve] [client] improve the class GetTopicsResult (#22766) (cherry picked from commit 87a33399873ff1e9723a6ca3812cbf914d8c8eef) (cherry picked from commit 250dacb911a3ad490a6374912a7700db2ff68d50) --- .../pulsar/client/impl/LookupServiceTest.java | 128 ++++++++++++++++++ .../client/impl/BinaryProtoLookupService.java | 14 +- .../pulsar/client/impl/HttpLookupService.java | 13 +- .../pulsar/common/lookup/GetTopicsResult.java | 106 +++++++++++++-- 4 files changed, 225 insertions(+), 36 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/impl/LookupServiceTest.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/LookupServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/LookupServiceTest.java new file mode 100644 index 0000000000000..59cb7ae03d0e3 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/LookupServiceTest.java @@ -0,0 +1,128 @@ +/* + * 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.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace.Mode; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; +import java.util.Collection; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicName; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Test(groups = "broker-admin") +@Slf4j +public class LookupServiceTest extends ProducerConsumerBase { + + private PulsarClientImpl clientWithHttpLookup; + private PulsarClientImpl clientWitBinaryLookup; + + private boolean enableBrokerSideSubscriptionPatternEvaluation = true; + private int subscriptionPatternMaxLength = 10_000; + + @Override + @BeforeClass + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + clientWithHttpLookup = + (PulsarClientImpl) PulsarClient.builder().serviceUrl(pulsar.getWebServiceAddress()).build(); + clientWitBinaryLookup = + (PulsarClientImpl) PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); + } + + @Override + @AfterClass(alwaysRun = true) + protected void cleanup() throws Exception { + super.internalCleanup(); + if (clientWithHttpLookup != null) { + clientWithHttpLookup.close(); + } + if (clientWitBinaryLookup != null) { + clientWitBinaryLookup.close(); + } + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + conf.setEnableBrokerSideSubscriptionPatternEvaluation(enableBrokerSideSubscriptionPatternEvaluation); + conf.setSubscriptionPatternMaxLength(subscriptionPatternMaxLength); + } + + private LookupService getLookupService(boolean isUsingHttpLookup) { + if (isUsingHttpLookup) { + return clientWithHttpLookup.getLookup(); + } else { + return clientWitBinaryLookup.getLookup(); + } + } + + @DataProvider(name = "isUsingHttpLookup") + public Object[][] isUsingHttpLookup() { + return new Object[][]{ + {true}, + {false} + }; + } + + @Test(dataProvider = "isUsingHttpLookup") + public void testGetTopicsOfGetTopicsResult(boolean isUsingHttpLookup) throws Exception { + LookupService lookupService = getLookupService(isUsingHttpLookup); + String nonPartitionedTopic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(nonPartitionedTopic); + String partitionedTopic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createPartitionedTopic(partitionedTopic, 3); + String nonPersistentTopic = BrokerTestUtil.newUniqueName("non-persistent://public/default/tp"); + + // Verify the new method "GetTopicsResult.getTopics" works as expected. + Collection topics = lookupService.getTopicsUnderNamespace(NamespaceName.get("public/default"), + Mode.PERSISTENT, "public/default/.*", null).join().getTopics(); + assertTrue(topics.contains(nonPartitionedTopic)); + assertTrue(topics.contains(partitionedTopic)); + assertFalse(topics.contains(nonPersistentTopic)); + assertFalse(topics.contains(TopicName.get(partitionedTopic).getPartition(0).toString())); + // Verify the new method "GetTopicsResult.nonPartitionedOrPartitionTopics" works as expected. + Collection nonPartitionedOrPartitionTopics = + lookupService.getTopicsUnderNamespace(NamespaceName.get("public/default"), + Mode.PERSISTENT, "public/default/.*", null).join() + .getNonPartitionedOrPartitionTopics(); + assertTrue(nonPartitionedOrPartitionTopics.contains(nonPartitionedTopic)); + assertFalse(nonPartitionedOrPartitionTopics.contains(partitionedTopic)); + assertFalse(nonPartitionedOrPartitionTopics.contains(nonPersistentTopic)); + assertTrue(nonPartitionedOrPartitionTopics.contains(TopicName.get(partitionedTopic).getPartition(0) + .toString())); + assertTrue(nonPartitionedOrPartitionTopics.contains(TopicName.get(partitionedTopic).getPartition(1) + .toString())); + assertTrue(nonPartitionedOrPartitionTopics.contains(TopicName.get(partitionedTopic).getPartition(2) + .toString())); + + // Cleanup. + admin.topics().deletePartitionedTopic(partitionedTopic, false); + admin.topics().delete(nonPartitionedTopic, false); + } + +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index 9634c60f09a27..9d01d863143e2 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -22,8 +22,6 @@ import io.netty.buffer.ByteBuf; import java.net.InetSocketAddress; import java.net.URI; -import java.util.ArrayList; -import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -341,17 +339,7 @@ private void getTopicsUnderNamespace(InetSocketAddress socketAddress, log.debug("[namespace: {}] Success get topics list in request: {}", namespace, requestId); } - // do not keep partition part of topic name - List result = new ArrayList<>(); - r.getTopics().forEach(topic -> { - String filtered = TopicName.get(topic).getPartitionedTopicName(); - if (!result.contains(filtered)) { - result.add(filtered); - } - }); - - getTopicsResultFuture.complete(new GetTopicsResult(result, r.getTopicsHash(), - r.isFiltered(), r.isChanged())); + getTopicsResultFuture.complete(r); } client.getCnxPool().releaseConnection(clientCnx); }); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java index e33efabcc9e0e..6e8c2b4314e17 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java @@ -23,10 +23,7 @@ import java.net.URI; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Arrays; import java.util.Base64; -import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.commons.lang3.StringUtils; @@ -137,15 +134,7 @@ public CompletableFuture getTopicsUnderNamespace(NamespaceName httpClient .get(String.format(format, namespace, mode.toString()), String[].class) .thenAccept(topics -> { - List result = new ArrayList<>(); - // do not keep partition part of topic name - Arrays.asList(topics).forEach(topic -> { - String filtered = TopicName.get(topic).getPartitionedTopicName(); - if (!result.contains(filtered)) { - result.add(filtered); - } - }); - future.complete(new GetTopicsResult(result, topicsHash, false, true)); + future.complete(new GetTopicsResult(topics)); }).exceptionally(ex -> { Throwable cause = FutureUtil.unwrapCompletionException(ex); log.warn("Failed to getTopicsUnderNamespace namespace {} {}.", namespace, cause.getMessage()); diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/lookup/GetTopicsResult.java b/pulsar-common/src/main/java/org/apache/pulsar/common/lookup/GetTopicsResult.java index 55fe6253ff971..80f16e6c36717 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/lookup/GetTopicsResult.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/lookup/GetTopicsResult.java @@ -18,21 +18,105 @@ */ package org.apache.pulsar.common.lookup; +import java.util.ArrayList; +import java.util.Arrays; import java.util.List; -import lombok.AllArgsConstructor; import lombok.Getter; -import lombok.NoArgsConstructor; -import lombok.Setter; import lombok.ToString; +import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; +import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespaceResponse; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.topics.TopicList; -@Getter -@Setter -@AllArgsConstructor -@NoArgsConstructor +/*** + * A value object. + * - The response of HTTP API "admin/v2/namespaces/{domain}/topics" is a topic(non-partitioned topic or partitions) + * array. It will be wrapped to "topics: {topic array}, topicsHash: null, filtered: false, changed: true". + * - The response of binary API {@link CommandGetTopicsOfNamespace} is a {@link CommandGetTopicsOfNamespaceResponse}, + * it will be transferred to a {@link GetTopicsResult}. + * See more details https://github.com/apache/pulsar/pull/14804. + */ @ToString public class GetTopicsResult { - private List topics; - private String topicsHash; - private boolean filtered; - private boolean changed; + + /** + * Non-partitioned topics, and topic partitions of partitioned topics. + */ + @Getter + private final List nonPartitionedOrPartitionTopics; + + /** + * The topics have been filtered by Broker using a regexp. Otherwise, the client should do a client-side filter. + * There are three cases that brokers will not filter the topics: + * 1. the lookup service is typed HTTP lookup service, the HTTP API has not implemented this feature yet. + * 2. the broker does not support this feature(in other words, its version is lower than "2.11.0"). + * 3. the input param "topicPattern" is too long than the broker config "subscriptionPatternMaxLength". + */ + @Getter + private final boolean filtered; + + /** + * The topics hash that was calculated by {@link TopicList#calculateHash(List)}. The param topics that will be used + * to calculate the hash code is only contains the topics that has been filtered. + * Note: It is always "null" if broker did not filter the topics when calling the API + * "LookupService.getTopicsUnderNamespace"(in other words, {@link #filtered} is false). + */ + @Getter + private final String topicsHash; + + /** + * The topics hash has changed after compare with the input param "topicsHash" when calling + * "LookupService.getTopicsUnderNamespace". + * Note: It is always set "true" if the input param "topicsHash" that used to call + * "LookupService.getTopicsUnderNamespace" is null or the "LookupService" is "HttpLookupService". + */ + @Getter + private final boolean changed; + + /** + * Partitioned topics and non-partitioned topics. + * In other words, there is no topic partitions of partitioned topics in this list. + * Note: it is not a field of the response of "LookupService.getTopicsUnderNamespace", it is generated in + * client-side memory. + */ + private volatile List topics; + + /** + * This constructor is used for binary API. + */ + public GetTopicsResult(List nonPartitionedOrPartitionTopics, String topicsHash, boolean filtered, + boolean changed) { + this.nonPartitionedOrPartitionTopics = nonPartitionedOrPartitionTopics; + this.topicsHash = topicsHash; + this.filtered = filtered; + this.changed = changed; + } + + /** + * This constructor is used for HTTP API. + */ + public GetTopicsResult(String[] nonPartitionedOrPartitionTopics) { + this(Arrays.asList(nonPartitionedOrPartitionTopics), null, false, true); + } + + public List getTopics() { + if (topics != null) { + return topics; + } + synchronized (this) { + if (topics != null) { + return topics; + } + // Group partitioned topics. + List grouped = new ArrayList<>(); + for (String topic : nonPartitionedOrPartitionTopics) { + String partitionedTopic = TopicName.get(topic).getPartitionedTopicName(); + if (!grouped.contains(partitionedTopic)) { + grouped.add(partitionedTopic); + } + } + topics = grouped; + return topics; + } + } } From 22a0603e97cbf00df8bb02fcfd538c7f67b7abde Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Tue, 4 Jun 2024 03:18:39 -0700 Subject: [PATCH 539/699] [improve] Upgrade Jetcd to 0.7.7 and VertX to 4.5.8 (#22835) (cherry picked from commit be5eb919f8c9fb4612fea74054eee8c1412b954b) (cherry picked from commit 46b5419224c1a821ec0b9a839d0d7da57ac5f181) --- .../server/src/assemble/LICENSE.bin.txt | 23 ++++++++++--------- pom.xml | 23 +++++++++++++++++-- pulsar-sql/presto-distribution/LICENSE | 17 +++++++------- tests/integration/pom.xml | 6 +++++ .../integration/io/PulsarIOTestRunner.java | 7 +++--- .../io/sinks/PulsarIOSinkRunner.java | 2 +- .../io/sources/PulsarIOSourceRunner.java | 2 +- .../PulsarIODebeziumSourceRunner.java | 2 +- 8 files changed, 55 insertions(+), 27 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index b11b4c0c44667..c831228e5fb28 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -436,6 +436,7 @@ The Apache Software License, Version 2.0 - io.grpc-grpc-rls-1.55.3.jar - io.grpc-grpc-servlet-1.55.3.jar - io.grpc-grpc-servlet-jakarta-1.55.3.jar + - io.grpc-grpc-util-1.60.0.jar * Perfmark - io.perfmark-perfmark-api-0.26.0.jar * OpenCensus @@ -444,7 +445,7 @@ The Apache Software License, Version 2.0 - io.opencensus-opencensus-proto-0.2.0.jar * Jodah - net.jodah-typetools-0.5.0.jar - - net.jodah-failsafe-2.4.4.jar + - dev.failsafe-failsafe-3.3.2.jar * Byte Buddy - net.bytebuddy-byte-buddy-1.14.12.jar * zt-zip @@ -476,12 +477,12 @@ 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.3.8.jar - - io.vertx-vertx-bridge-common-4.3.8.jar - - io.vertx-vertx-core-4.3.8.jar - - io.vertx-vertx-web-4.3.8.jar - - io.vertx-vertx-web-common-4.3.8.jar - - io.vertx-vertx-grpc-4.3.5.jar + - 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-grpc-4.5.8.jar * Apache ZooKeeper - org.apache.zookeeper-zookeeper-3.9.2.jar - org.apache.zookeeper-zookeeper-jute-3.9.2.jar @@ -494,10 +495,10 @@ The Apache Software License, Version 2.0 - com.google.auto.value-auto-value-annotations-1.9.jar - com.google.re2j-re2j-1.6.jar * Jetcd - - io.etcd-jetcd-api-0.7.5.jar - - io.etcd-jetcd-common-0.7.5.jar - - io.etcd-jetcd-core-0.7.5.jar - - io.etcd-jetcd-grpc-0.7.5.jar + - io.etcd-jetcd-api-0.7.7.jar + - io.etcd-jetcd-common-0.7.7.jar + - io.etcd-jetcd-core-0.7.7.jar + - io.etcd-jetcd-grpc-0.7.7.jar * IPAddress - com.github.seancfoley-ipaddress-5.3.3.jar * RxJava diff --git a/pom.xml b/pom.xml index 589eeba115198..0ae3322035978 100644 --- a/pom.xml +++ b/pom.xml @@ -150,7 +150,7 @@ flexible messaging model and an intuitive client API. 2.34 1.10.50 0.16.0 - 4.3.8 + 4.5.8 7.9.2 1.7.32 4.4 @@ -248,13 +248,14 @@ flexible messaging model and an intuitive client API. 5.3.27 4.5.13 4.4.15 - 0.7.5 + 0.7.7 2.0 1.10.12 5.3.3 3.4.3 1.5.2-3 2.0.6 + 3.3.2 1.18.3 @@ -376,6 +377,12 @@ flexible messaging model and an intuitive client API. ${mockito.version} + + dev.failsafe + failsafe + ${failsafe.version} + + org.apache.zookeeper zookeeper @@ -497,6 +504,11 @@ flexible messaging model and an intuitive client API. vertx-web ${vertx.version} + + io.vertx + vertx-grpc + ${vertx.version} + org.apache.curator @@ -595,6 +607,13 @@ flexible messaging model and an intuitive client API. + + io.grpc + grpc-util + + 1.60.0 + + org.apache.bookkeeper bookkeeper-common diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 81a295608eb5f..9242877df015f 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -271,17 +271,18 @@ The Apache Software License, Version 2.0 - grpc-protobuf-1.55.3.jar - grpc-protobuf-lite-1.55.3.jar - grpc-stub-1.55.3.jar + - grpc-util-1.60.0.jar * JEtcd - - jetcd-api-0.7.5.jar - - jetcd-common-0.7.5.jar - - jetcd-core-0.7.5.jar - - jetcd-grpc-0.7.5.jar + - jetcd-api-0.7.7.jar + - jetcd-common-0.7.7.jar + - jetcd-core-0.7.7.jar + - jetcd-grpc-0.7.7.jar * Vertx - - vertx-core-4.3.8.jar - - vertx-grpc-4.3.5.jar + - vertx-core-4.5.8.jar + - vertx-grpc-4.5.8.jar * Joda Time - joda-time-2.10.10.jar - - failsafe-2.4.4.jar + - failsafe-3.3.2.jar * Jetty - http2-client-9.4.54.v20240208.jar - http2-common-9.4.54.v20240208.jar @@ -306,7 +307,7 @@ The Apache Software License, Version 2.0 * Bytecode - bytecode-1.2.jar * Airlift - - aircompressor-0.20.jar + - aircompressor-0.27.jar - bootstrap-213.jar - concurrent-213.jar - configuration-213.jar diff --git a/tests/integration/pom.xml b/tests/integration/pom.xml index 9020b0dc0f94e..cf245b428f046 100644 --- a/tests/integration/pom.xml +++ b/tests/integration/pom.xml @@ -102,6 +102,12 @@ test + + dev.failsafe + failsafe + test + + org.testcontainers mysql diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/PulsarIOTestRunner.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/PulsarIOTestRunner.java index 4492f6a407520..7c47a0dcff89b 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/PulsarIOTestRunner.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/PulsarIOTestRunner.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.tests.integration.io; +import dev.failsafe.RetryPolicy; import java.time.Duration; import java.util.LinkedHashMap; import java.util.Map; @@ -33,7 +34,6 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; -import net.jodah.failsafe.RetryPolicy; @Slf4j public abstract class PulsarIOTestRunner { @@ -42,10 +42,11 @@ public abstract class PulsarIOTestRunner { final Duration ONE_MINUTE = Duration.ofMinutes(1); final Duration TEN_SECONDS = Duration.ofSeconds(10); - protected final RetryPolicy statusRetryPolicy = new RetryPolicy() + protected final RetryPolicy statusRetryPolicy = RetryPolicy.builder() .withMaxDuration(ONE_MINUTE) .withDelay(TEN_SECONDS) - .onRetry(e -> log.error("Retry ... ")); + .onRetry(e -> log.error("Retry ... ")) + .build(); protected PulsarCluster pulsarCluster; protected String functionRuntimeType; diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarIOSinkRunner.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarIOSinkRunner.java index e5b524ebbef8b..3736bd0155343 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarIOSinkRunner.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sinks/PulsarIOSinkRunner.java @@ -22,6 +22,7 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import dev.failsafe.Failsafe; import java.util.LinkedHashMap; import java.util.Map; @@ -46,7 +47,6 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; -import net.jodah.failsafe.Failsafe; @Slf4j public class PulsarIOSinkRunner extends PulsarIOTestRunner { diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/PulsarIOSourceRunner.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/PulsarIOSourceRunner.java index b843e146e2985..daf645020ce5a 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/PulsarIOSourceRunner.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/PulsarIOSourceRunner.java @@ -22,6 +22,7 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import dev.failsafe.Failsafe; import java.util.Map; import org.apache.commons.lang3.StringUtils; @@ -45,7 +46,6 @@ import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; -import net.jodah.failsafe.Failsafe; @Slf4j public class PulsarIOSourceRunner extends PulsarIOTestRunner { diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/PulsarIODebeziumSourceRunner.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/PulsarIODebeziumSourceRunner.java index 762dd34e17c91..8f45f0604e378 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/PulsarIODebeziumSourceRunner.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/PulsarIODebeziumSourceRunner.java @@ -19,9 +19,9 @@ package org.apache.pulsar.tests.integration.io.sources.debezium; import com.google.common.base.Preconditions; +import dev.failsafe.Failsafe; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; -import net.jodah.failsafe.Failsafe; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.SubscriptionInitialPosition; From 7caf98859f216f94401f36d90b25ad00d648371a Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Mon, 10 Jun 2024 10:58:32 +0530 Subject: [PATCH 540/699] Updated license version for Datastax --- .../server/src/assemble/LICENSE.bin.txt | 44 +++++++++---------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index c831228e5fb28..e98e03104af49 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -585,28 +585,28 @@ Bouncy Castle License - org.bouncycastle-bcutil-jdk18on-1.78.1.jar Datastax - - com.datastax.oss-managed-ledger-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-auth-oidc-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-auth-sasl-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-common-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-config-validation-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-docs-tools-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-api-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-instance-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-local-runner-original-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-proto-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-runtime-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-secrets-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-utils-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-worker-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-io-core-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-metadata-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-proxy-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-testclient-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-transaction-common-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-pulsar-transaction-coordinator-3.1.4.2-SNAPSHOT.jar - - com.datastax.oss-testmocks-3.1.4.2-SNAPSHOT.jar + - com.datastax.oss-managed-ledger-3.1.4.2.jar + - com.datastax.oss-pulsar-broker-3.1.4.2.jar + - com.datastax.oss-pulsar-broker-auth-oidc-3.1.4.2.jar + - com.datastax.oss-pulsar-broker-auth-sasl-3.1.4.2.jar + - com.datastax.oss-pulsar-broker-common-3.1.4.2.jar + - com.datastax.oss-pulsar-config-validation-3.1.4.2.jar + - com.datastax.oss-pulsar-docs-tools-3.1.4.2.jar + - com.datastax.oss-pulsar-functions-api-3.1.4.2.jar + - com.datastax.oss-pulsar-functions-instance-3.1.4.2.jar + - com.datastax.oss-pulsar-functions-local-runner-original-3.1.4.2.jar + - com.datastax.oss-pulsar-functions-proto-3.1.4.2.jar + - com.datastax.oss-pulsar-functions-runtime-3.1.4.2.jar + - com.datastax.oss-pulsar-functions-secrets-3.1.4.2.jar + - com.datastax.oss-pulsar-functions-utils-3.1.4.2.jar + - com.datastax.oss-pulsar-functions-worker-3.1.4.2.jar + - com.datastax.oss-pulsar-io-core-3.1.4.2.jar + - com.datastax.oss-pulsar-metadata-3.1.4.2.jar + - com.datastax.oss-pulsar-proxy-3.1.4.2.jar + - com.datastax.oss-pulsar-testclient-3.1.4.2.jar + - com.datastax.oss-pulsar-transaction-common-3.1.4.2.jar + - com.datastax.oss-pulsar-transaction-coordinator-3.1.4.2.jar + - com.datastax.oss-testmocks-3.1.4.2.jar ------------------------ From 458f0592bd96a76036dbf9de6109bb561d7e70b6 Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Mon, 10 Jun 2024 11:12:45 +0530 Subject: [PATCH 541/699] Release 3.1.4.2 --- 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-experimental/pom.xml | 2 +- docker/pulsar/pom.xml | 2 +- jclouds-shaded/pom.xml | 2 +- managed-ledger/pom.xml | 2 +- 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-client-1x-base/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-1x/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-2x-shaded/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/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/hdfs2/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-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-sql/pom.xml | 2 +- pulsar-sql/presto-distribution/pom.xml | 2 +- pulsar-sql/presto-pulsar-plugin/pom.xml | 2 +- pulsar-sql/presto-pulsar/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 +- 134 files changed, 136 insertions(+), 136 deletions(-) diff --git a/bouncy-castle/bc/pom.xml b/bouncy-castle/bc/pom.xml index 1c267926e3b7a..03b72dcd6a004 100644 --- a/bouncy-castle/bc/pom.xml +++ b/bouncy-castle/bc/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.4.2-SNAPSHOT + 3.1.4.2 .. diff --git a/bouncy-castle/bcfips-include-test/pom.xml b/bouncy-castle/bcfips-include-test/pom.xml index 277e607dc4f55..4110d7962c0ad 100644 --- a/bouncy-castle/bcfips-include-test/pom.xml +++ b/bouncy-castle/bcfips-include-test/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.4.2-SNAPSHOT + 3.1.4.2 .. diff --git a/bouncy-castle/bcfips/pom.xml b/bouncy-castle/bcfips/pom.xml index 26a4ec0e5f86f..242782199f321 100644 --- a/bouncy-castle/bcfips/pom.xml +++ b/bouncy-castle/bcfips/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.4.2-SNAPSHOT + 3.1.4.2 .. diff --git a/bouncy-castle/pom.xml b/bouncy-castle/pom.xml index 1835082990829..ea14616fc8039 100644 --- a/bouncy-castle/pom.xml +++ b/bouncy-castle/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.4.2-SNAPSHOT + 3.1.4.2 .. diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 5a10364c88b6b..141d22eee5ebb 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -31,12 +31,12 @@ com.datastax.oss buildtools - 3.1.4.2-SNAPSHOT + 3.1.4.2 jar Pulsar Build Tools - 2024-05-20T13:27:28Z + 2024-06-10T05:42:45Z 1.8 1.8 3.1.0 diff --git a/distribution/io/pom.xml b/distribution/io/pom.xml index aa77b1f022db9..fe5fc54633b9d 100644 --- a/distribution/io/pom.xml +++ b/distribution/io/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.4.2-SNAPSHOT + 3.1.4.2 .. diff --git a/distribution/offloaders/pom.xml b/distribution/offloaders/pom.xml index 86380e06ced43..024e0e6ca06dd 100644 --- a/distribution/offloaders/pom.xml +++ b/distribution/offloaders/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.4.2-SNAPSHOT + 3.1.4.2 .. diff --git a/distribution/pom.xml b/distribution/pom.xml index b7e789a75295e..042ad279dbae6 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.4.2-SNAPSHOT + 3.1.4.2 .. diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index c3d8561c93c20..ef44d73f3829c 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.4.2-SNAPSHOT + 3.1.4.2 .. diff --git a/distribution/shell/pom.xml b/distribution/shell/pom.xml index 326aa171d7c83..5354321430405 100644 --- a/distribution/shell/pom.xml +++ b/distribution/shell/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.4.2-SNAPSHOT + 3.1.4.2 .. diff --git a/docker/pom.xml b/docker/pom.xml index cb50d60231f4d..6baa9ac242f74 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.4.2-SNAPSHOT + 3.1.4.2 docker-images Apache Pulsar :: Docker Images diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index be1a7f00d990c..f8ee989b31830 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.4.2-SNAPSHOT + 3.1.4.2 4.0.0 pulsar-all-docker-image diff --git a/docker/pulsar-experimental/pom.xml b/docker/pulsar-experimental/pom.xml index f146b3db6d057..0b5bba3e6e9a7 100644 --- a/docker/pulsar-experimental/pom.xml +++ b/docker/pulsar-experimental/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.4.2-SNAPSHOT + 3.1.4.2 4.0.0 pulsar-experimental-docker-image diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 2719641c7180c..dcb23ab2e9209 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.4.2-SNAPSHOT + 3.1.4.2 4.0.0 pulsar-docker-image diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml index fdfc3cb928c78..279067022285d 100644 --- a/jclouds-shaded/pom.xml +++ b/jclouds-shaded/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.4.2-SNAPSHOT + 3.1.4.2 .. diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index f4600b85cbd2f..bc9844af99f30 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.4.2-SNAPSHOT + 3.1.4.2 .. diff --git a/pom.xml b/pom.xml index 0ae3322035978..3541f07a444b4 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ com.datastax.oss pulsar - 3.1.4.2-SNAPSHOT + 3.1.4.2 Pulsar Pulsar is a distributed pub-sub messaging platform with a very @@ -95,7 +95,7 @@ flexible messaging model and an intuitive client API. UTF-8 UTF-8 - 2024-05-20T13:27:28Z + 2024-06-10T05:42:45Z true - + + + #{(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).*'))} https://ge.apache.org false - true true true #{isFalse(env['GITHUB_ACTIONS'])} - ALWAYS - true #{{'0.0.0.0'}} @@ -47,4 +46,4 @@ false - + \ No newline at end of file diff --git a/.mvn/extensions.xml b/.mvn/extensions.xml index 872764f899827..4a2117925f163 100644 --- a/.mvn/extensions.xml +++ b/.mvn/extensions.xml @@ -23,12 +23,12 @@ xsi:schemaLocation="http://maven.apache.org/EXTENSIONS/1.0.0 http://maven.apache.org/xsd/core-extensions-1.0.0.xsd"> com.gradle - gradle-enterprise-maven-extension - 1.17.1 + develocity-maven-extension + 1.21.4 com.gradle common-custom-user-data-maven-extension - 1.11.1 + 2.0 From 35876cc30d068c9acc51d0bed0aafa019f2f250c Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 31 Jan 2024 10:01:44 -0800 Subject: [PATCH 545/699] [improve][broker] Include runtime dependencies in server distribution (#22001) (cherry picked from commit 57025bc11913680f7aac26ab42399ea8a6fccc05) (cherry picked from commit cdd40e188522bd0f152d69006111891171a3b252) --- distribution/server/src/assemble/LICENSE.bin.txt | 2 +- distribution/server/src/assemble/bin.xml | 9 ++++++--- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 0374b97b6836f..e06d69e45b38a 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -505,6 +505,7 @@ The Apache Software License, Version 2.0 - io.reactivex.rxjava3-rxjava-3.0.1.jar * RoaringBitmap - org.roaringbitmap-RoaringBitmap-0.9.44.jar + - org.roaringbitmap-shims-0.9.44.jar BSD 3-clause "New" or "Revised" License * Google auth library @@ -537,7 +538,6 @@ Protocol Buffers License CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt * Java Annotations API - - javax.annotation-javax.annotation-api-1.3.2.jar - 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 diff --git a/distribution/server/src/assemble/bin.xml b/distribution/server/src/assemble/bin.xml index aafb559d67fb2..c00e020ff3394 100644 --- a/distribution/server/src/assemble/bin.xml +++ b/distribution/server/src/assemble/bin.xml @@ -126,7 +126,7 @@ lib false - compile + runtime false @@ -135,12 +135,15 @@ com.datastax.oss:pulsar-functions-runtime-all - org.projectlombok:lombok - com.datastax.oss:pulsar-functions-api-examples *:tar.gz + + org.codehaus.mojo:animal-sniffer-annotations + com.google.android:annotations + + net.java.dev.jna:jna From aa3915246bdb7579b6431bc2984a538b4d9533ac Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 21 Mar 2024 13:23:21 -0700 Subject: [PATCH 546/699] [improve][misc] Include native epoll library for Netty for arm64 (#22319) (cherry picked from commit 24e9437ce065613fd924a74f61b620d9fdc0058b) (cherry picked from commit 22b724fd1c3eac463834a58102d667617451d453) # Conflicts: # distribution/server/src/assemble/LICENSE.bin.txt # distribution/shell/src/assemble/LICENSE.bin.txt # pulsar-sql/presto-distribution/LICENSE (cherry picked from commit 69ecbcd9d3c89d4dd4aaf8a7e14390829b91ee93) --- distribution/server/src/assemble/LICENSE.bin.txt | 1 + distribution/shell/src/assemble/LICENSE.bin.txt | 1 + pulsar-common/pom.xml | 6 ++++++ pulsar-sql/presto-distribution/LICENSE | 1 + 4 files changed, 9 insertions(+) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index e06d69e45b38a..dda81b53a4663 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -306,6 +306,7 @@ The Apache Software License, Version 2.0 - io.netty-netty-resolver-dns-native-macos-4.1.108.Final-osx-x86_64.jar - io.netty-netty-transport-4.1.108.Final.jar - io.netty-netty-transport-classes-epoll-4.1.108.Final.jar + - io.netty-netty-transport-native-epoll-4.1.108.Final-linux-aarch_64.jar - io.netty-netty-transport-native-epoll-4.1.108.Final-linux-x86_64.jar - io.netty-netty-transport-native-unix-common-4.1.108.Final.jar - io.netty-netty-transport-native-unix-common-4.1.108.Final-linux-x86_64.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index ff69efeda4644..71811b83f65c2 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -357,6 +357,7 @@ The Apache Software License, Version 2.0 - netty-resolver-dns-4.1.108.Final.jar - netty-transport-4.1.108.Final.jar - netty-transport-classes-epoll-4.1.108.Final.jar + - netty-transport-native-epoll-4.1.108.Final-linux-aarch_64.jar - netty-transport-native-epoll-4.1.108.Final-linux-x86_64.jar - netty-transport-native-unix-common-4.1.108.Final.jar - netty-transport-native-unix-common-4.1.108.Final-linux-x86_64.jar diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 2332a91a469d4..04ace01ceb5e5 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -99,6 +99,12 @@ linux-x86_64 + + io.netty + netty-transport-native-epoll + linux-aarch_64 + + io.netty netty-transport-native-unix-common diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 9242877df015f..90723dafab95f 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -255,6 +255,7 @@ The Apache Software License, Version 2.0 - netty-tcnative-classes-2.0.65.Final.jar - netty-transport-4.1.108.Final.jar - netty-transport-classes-epoll-4.1.108.Final.jar + - netty-transport-native-epoll-4.1.108.Final-linux-aarch_64.jar - netty-transport-native-epoll-4.1.108.Final-linux-x86_64.jar - netty-transport-native-unix-common-4.1.108.Final.jar - netty-transport-native-unix-common-4.1.108.Final-linux-x86_64.jar From ed9e77ab16d7d733ad3f182abce2eae8318ed8f3 Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Wed, 5 Jun 2024 10:49:00 -0700 Subject: [PATCH 547/699] [fix] Remove blocking calls from BookieRackAffinityMapping (#22846) (cherry picked from commit aece67e35ecec4a9d90a951b78cfc89ca6395054) (cherry picked from commit efa5e8b04018356447ec1744c6e083430e8e1f05) --- .../BookieRackAffinityMapping.java | 44 ++++++++++++------- ...IsolatedBookieEnsemblePlacementPolicy.java | 2 +- 2 files changed, 28 insertions(+), 18 deletions(-) diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java index 983822f22941b..4a5ff746f4039 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java @@ -70,7 +70,7 @@ public class BookieRackAffinityMapping extends AbstractDNSToSwitchMapping private BookiesRackConfiguration racksWithHost = new BookiesRackConfiguration(); private Map bookieInfoMap = new HashMap<>(); - public static MetadataStore createMetadataStore(Configuration conf) throws MetadataException { + static MetadataStore getMetadataStore(Configuration conf) throws MetadataException { MetadataStore store; Object storeProperty = conf.getProperty(METADATA_STORE_INSTANCE); if (storeProperty != null) { @@ -116,12 +116,20 @@ public synchronized void setConf(Configuration conf) { super.setConf(conf); MetadataStore store; try { - store = createMetadataStore(conf); - bookieMappingCache = store.getMetadataCache(BookiesRackConfiguration.class); - store.registerListener(this::handleUpdates); - racksWithHost = bookieMappingCache.get(BOOKIE_INFO_ROOT_PATH).get() - .orElseGet(BookiesRackConfiguration::new); - for (Map bookieMapping : racksWithHost.values()) { + store = getMetadataStore(conf); + } catch (MetadataException e) { + throw new RuntimeException(METADATA_STORE_INSTANCE + " failed to init BookieId list"); + } + + bookieMappingCache = store.getMetadataCache(BookiesRackConfiguration.class); + store.registerListener(this::handleUpdates); + + try { + var racksWithHost = bookieMappingCache.get(BOOKIE_INFO_ROOT_PATH) + .thenApply(optRes -> optRes.orElseGet(BookiesRackConfiguration::new)) + .get(); + + for (var bookieMapping : racksWithHost.values()) { for (String address : bookieMapping.keySet()) { bookieAddressListLastTime.add(BookieId.parse(address)); } @@ -131,10 +139,12 @@ public synchronized void setConf(Configuration conf) { } } updateRacksWithHost(racksWithHost); - watchAvailableBookies(); - } catch (InterruptedException | ExecutionException | MetadataException e) { - throw new RuntimeException(METADATA_STORE_INSTANCE + " failed to init BookieId list"); + } catch (ExecutionException | InterruptedException e) { + LOG.error("Failed to update rack info. ", e); + throw new RuntimeException(e); } + + watchAvailableBookies(); } private void watchAvailableBookies() { @@ -145,13 +155,13 @@ private void watchAvailableBookies() { field.setAccessible(true); RegistrationClient registrationClient = (RegistrationClient) field.get(bookieAddressResolver); registrationClient.watchWritableBookies(versioned -> { - try { - racksWithHost = bookieMappingCache.get(BOOKIE_INFO_ROOT_PATH).get() - .orElseGet(BookiesRackConfiguration::new); - updateRacksWithHost(racksWithHost); - } catch (InterruptedException | ExecutionException e) { - LOG.error("Failed to update rack info. ", e); - } + bookieMappingCache.get(BOOKIE_INFO_ROOT_PATH) + .thenApply(optRes -> optRes.orElseGet(BookiesRackConfiguration::new)) + .thenAccept(this::updateRacksWithHost) + .exceptionally(ex -> { + LOG.error("Failed to update rack info. ", ex); + return null; + }); }); } catch (NoSuchFieldException | IllegalAccessException e) { LOG.error("Failed watch available bookies.", e); diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/IsolatedBookieEnsemblePlacementPolicy.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/IsolatedBookieEnsemblePlacementPolicy.java index 02ddea9487469..8b1ef16cb6dae 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/IsolatedBookieEnsemblePlacementPolicy.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/IsolatedBookieEnsemblePlacementPolicy.java @@ -73,7 +73,7 @@ public RackawareEnsemblePlacementPolicyImpl initialize(ClientConfiguration conf, StatsLogger statsLogger, BookieAddressResolver bookieAddressResolver) { MetadataStore store; try { - store = BookieRackAffinityMapping.createMetadataStore(conf); + store = BookieRackAffinityMapping.getMetadataStore(conf); } catch (MetadataException e) { throw new RuntimeException(METADATA_STORE_INSTANCE + " failed initialized"); } From 338f52d645802c3cb7773b92842c198ff9f9c7ad Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 7 Jun 2024 18:36:52 +0300 Subject: [PATCH 548/699] [fix][cli] Fix Pulsar standalone shutdown - bkCluster wasn't closed (#22868) (cherry picked from commit c5cc25ebdc3a32d002b944e77fb59c9ccd1f14c1) (cherry picked from commit ebb4282949e53ecf2229525d840143830beec89b) --- .../org/apache/pulsar/PulsarStandalone.java | 10 ++++ .../pulsar/PulsarStandaloneStarter.java | 58 ++++++++++++++----- .../apache/pulsar/PulsarStandaloneTest.java | 48 +++++++++++++-- .../configurations/pulsar_broker_test.conf | 26 ++++----- .../pulsar_broker_test_standalone.conf | 26 ++++----- ...r_broker_test_standalone_with_rocksdb.conf | 26 ++++----- .../standalone_no_client_auth.conf | 4 +- .../pulsar/metadata/bookkeeper/BKCluster.java | 43 +++++++++----- 8 files changed, 167 insertions(+), 74 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java index ba136e7c91058..a2a101fe394cc 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java @@ -417,18 +417,22 @@ public void close() { try { if (fnWorkerService != null) { fnWorkerService.stop(); + fnWorkerService = null; } if (broker != null) { broker.close(); + broker = null; } if (bkCluster != null) { bkCluster.close(); + bkCluster = null; } if (bkEnsemble != null) { bkEnsemble.stop(); + bkEnsemble = null; } } catch (Exception e) { log.error("Shutdown failed: {}", e.getMessage(), e); @@ -493,5 +497,11 @@ private static void processTerminator(int exitCode) { ShutdownUtil.triggerImmediateForcefulShutdown(exitCode); } + public String getBrokerServiceUrl() { + return broker.getBrokerServiceUrl(); + } + public String getWebServiceUrl() { + return broker.getWebServiceAddress(); + } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandaloneStarter.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandaloneStarter.java index 25320964fd62b..63ee992de91ca 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandaloneStarter.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandaloneStarter.java @@ -21,9 +21,12 @@ import static org.apache.commons.lang3.StringUtils.isBlank; import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; import java.io.FileInputStream; import java.util.Arrays; +import lombok.AccessLevel; +import lombok.Setter; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; import org.apache.logging.log4j.LogManager; @@ -38,6 +41,9 @@ public class PulsarStandaloneStarter extends PulsarStandalone { @Parameter(names = {"-g", "--generate-docs"}, description = "Generate docs") private boolean generateDocs = false; + private Thread shutdownThread; + @Setter(AccessLevel.PACKAGE) + private boolean testMode; public PulsarStandaloneStarter(String[] args) throws Exception { @@ -108,26 +114,50 @@ public PulsarStandaloneStarter(String[] args) throws Exception { } } } + } - Runtime.getRuntime().addShutdownHook(new Thread(() -> { + @Override + public synchronized void start() throws Exception { + super.start(); + if (shutdownThread != null) { + throw new IllegalStateException("Shutdown hook already registered"); + } + shutdownThread = new Thread(() -> { try { - if (fnWorkerService != null) { - fnWorkerService.stop(); - } - - if (broker != null) { - broker.close(); - } - - if (bkEnsemble != null) { - bkEnsemble.stop(); - } + doClose(false); } catch (Exception e) { log.error("Shutdown failed: {}", e.getMessage(), e); } finally { - LogManager.shutdown(); + if (!testMode) { + LogManager.shutdown(); + } } - })); + }); + Runtime.getRuntime().addShutdownHook(shutdownThread); + } + + // simulate running the shutdown hook, for testing + @VisibleForTesting + void runShutdownHook() { + if (!testMode) { + throw new IllegalStateException("Not in test mode"); + } + Runtime.getRuntime().removeShutdownHook(shutdownThread); + shutdownThread.run(); + shutdownThread = null; + } + + @Override + public void close() { + doClose(true); + } + + private synchronized void doClose(boolean removeShutdownHook) { + super.close(); + if (shutdownThread != null && removeShutdownHook) { + Runtime.getRuntime().removeShutdownHook(shutdownThread); + shutdownThread = null; + } } private static boolean argsContains(String[] args, String arg) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/PulsarStandaloneTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/PulsarStandaloneTest.java index 6ed93a75a3fb5..3d22feb822e32 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/PulsarStandaloneTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/PulsarStandaloneTest.java @@ -31,6 +31,7 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.metadata.bookkeeper.BKCluster; import org.testng.Assert; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -46,12 +47,15 @@ public Object[][] enableBrokerClientAuth() { @Test public void testStandaloneWithRocksDB() throws Exception { String[] args = new String[]{"--config", - "./src/test/resources/configurations/pulsar_broker_test_standalone_with_rocksdb.conf"}; + "./src/test/resources/configurations/pulsar_broker_test_standalone_with_rocksdb.conf", + "-nss", + "-nfw"}; final int bookieNum = 3; final File tempDir = IOUtils.createTempDir("standalone", "test"); PulsarStandaloneStarter standalone = new PulsarStandaloneStarter(args); standalone.setBkDir(tempDir.getAbsolutePath()); + standalone.setBkPort(0); standalone.setNumOfBk(bookieNum); standalone.startBookieWithMetadataStore(); @@ -90,11 +94,12 @@ public void testMetadataInitialization(boolean enableBrokerClientAuth) throws Ex } final File bkDir = IOUtils.createTempDir("standalone", "bk"); standalone.setNumOfBk(1); + standalone.setBkPort(0); standalone.setBkDir(bkDir.getAbsolutePath()); standalone.start(); @Cleanup PulsarAdmin admin = PulsarAdmin.builder() - .serviceHttpUrl("http://localhost:8080") + .serviceHttpUrl(standalone.getWebServiceUrl()) .authentication(new MockTokenAuthenticationProvider.MockAuthentication()) .build(); if (enableBrokerClientAuth) { @@ -104,8 +109,8 @@ public void testMetadataInitialization(boolean enableBrokerClientAuth) throws Ex } else { assertTrue(admin.clusters().getClusters().isEmpty()); admin.clusters().createCluster("test_cluster", ClusterData.builder() - .serviceUrl("http://localhost:8080/") - .brokerServiceUrl("pulsar://localhost:6650/") + .serviceUrl(standalone.getWebServiceUrl()) + .brokerServiceUrl(standalone.getBrokerServiceUrl()) .build()); assertTrue(admin.tenants().getTenants().isEmpty()); admin.tenants().createTenant("public", TenantInfo.builder() @@ -125,4 +130,39 @@ public void testMetadataInitialization(boolean enableBrokerClientAuth) throws Ex cleanDirectory(bkDir); cleanDirectory(metadataDir); } + + + @Test + public void testShutdownHookClosesBkCluster() throws Exception { + File dataDir = IOUtils.createTempDir("data", ""); + File metadataDir = new File(dataDir, "metadata"); + File bkDir = new File(dataDir, "bookkeeper"); + @Cleanup + PulsarStandaloneStarter standalone = new PulsarStandaloneStarter(new String[] { + "--config", + "./src/test/resources/configurations/pulsar_broker_test_standalone_with_rocksdb.conf", + "-nss", + "-nfw", + "--metadata-dir", + metadataDir.getAbsolutePath(), + "--bookkeeper-dir", + bkDir.getAbsolutePath() + }); + standalone.setTestMode(true); + standalone.setBkPort(0); + standalone.start(); + BKCluster bkCluster = standalone.bkCluster; + standalone.runShutdownHook(); + assertTrue(bkCluster.isClosed()); + } + + @Test + public void testWipeData() throws Exception { + PulsarStandaloneStarter standalone = new PulsarStandaloneStarter(new String[] { + "--config", + "./src/test/resources/configurations/standalone_no_client_auth.conf", + "--wipe-data" + }); + assertTrue(standalone.isWipeData()); + } } diff --git a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf index bfbbfb7487c42..36f5869d73de6 100644 --- a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf +++ b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf @@ -17,17 +17,17 @@ # under the License. # -applicationName="pulsar_broker" -zookeeperServers="localhost" -configurationStoreServers="localhost" +applicationName=pulsar_broker +zookeeperServers=localhost +configurationStoreServers=localhost brokerServicePort=6650 -brokerServicePortTls=6651 +brokerServicePortTls= webServicePort=8080 -webServicePortTls=4443 +webServicePortTls= httpMaxRequestHeaderSize=1234 bindAddress=0.0.0.0 advertisedAddress= -clusterName="test_cluster" +clusterName=test_cluster brokerShutdownTimeoutMs=3000 backlogQuotaCheckEnabled=true backlogQuotaCheckIntervalInSeconds=60 @@ -42,17 +42,17 @@ clientLibraryVersionCheckEnabled=false clientLibraryVersionCheckAllowUnversioned=true statusFilePath=/tmp/status.html tlsEnabled=false -tlsCertificateFilePath=/usr/local/conf/pulsar/server.crt -tlsKeyFilePath=/home/local/conf/pulsar/server.key +tlsCertificateFilePath= +tlsKeyFilePath= tlsTrustCertsFilePath= tlsAllowInsecureConnection=false authenticationEnabled=false authorizationEnabled=false -superUserRoles="test_user" -brokerClientAuthenticationPlugin="org.apache.pulsar.client.impl.auth.AuthenticationDisabled" +superUserRoles=test_user +brokerClientAuthenticationPlugin=org.apache.pulsar.client.impl.auth.AuthenticationDisabled brokerClientAuthenticationParameters= -bookkeeperClientAuthenticationPlugin="test_auth_plugin" -bookkeeperClientAuthenticationAppId="test_auth_id" +bookkeeperClientAuthenticationPlugin= +bookkeeperClientAuthenticationAppId=test_auth_id bookkeeperClientTimeoutInSeconds=30 bookkeeperClientSpeculativeReadTimeoutInMillis=0 bookkeeperClientHealthCheckEnabled=true @@ -64,7 +64,7 @@ bookkeeperClientRegionawarePolicyEnabled=false bookkeeperClientMinNumRacksPerWriteQuorum=2 bookkeeperClientEnforceMinNumRacksPerWriteQuorum=false bookkeeperClientReorderReadSequenceEnabled=false -bookkeeperClientIsolationGroups="test_group" +bookkeeperClientIsolationGroups=test_group managedLedgerDefaultEnsembleSize=3 managedLedgerDefaultWriteQuorum=2 managedLedgerDefaultAckQuorum=2 diff --git a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf index c733409fc0043..0748418be6390 100644 --- a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf +++ b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf @@ -17,18 +17,18 @@ # under the License. # -applicationName="pulsar_broker" -metadataStoreUrl="zk:localhost:2181/ledger" -configurationMetadataStoreUrl="zk:localhost:2181" -brokerServicePort=6650 -brokerServicePortTls=6651 -webServicePort=8080 -webServicePortTls=4443 +applicationName=pulsar_broker +metadataStoreUrl=zk:localhost:2181/ledger +configurationMetadataStoreUrl=zk:localhost:2181 +brokerServicePort=0 +brokerServicePortTls= +webServicePort=0 +webServicePortTls= bindAddress=0.0.0.0 advertisedAddress= advertisedListeners=internal:pulsar://192.168.1.11:6660,internal:pulsar+ssl://192.168.1.11:6651 internalListenerName=internal -clusterName="test_cluster" +clusterName=test_cluster brokerShutdownTimeoutMs=3000 backlogQuotaCheckEnabled=true backlogQuotaCheckIntervalInSeconds=60 @@ -49,11 +49,11 @@ tlsTrustCertsFilePath= tlsAllowInsecureConnection=false authenticationEnabled=false authorizationEnabled=false -superUserRoles="test_user" -brokerClientAuthenticationPlugin="org.apache.pulsar.client.impl.auth.AuthenticationDisabled" +superUserRoles=test_user +brokerClientAuthenticationPlugin=org.apache.pulsar.client.impl.auth.AuthenticationDisabled brokerClientAuthenticationParameters= -bookkeeperClientAuthenticationPlugin="test_auth_plugin" -bookkeeperClientAuthenticationAppId="test_auth_id" +bookkeeperClientAuthenticationPlugin= +bookkeeperClientAuthenticationAppId= bookkeeperClientTimeoutInSeconds=30 bookkeeperClientSpeculativeReadTimeoutInMillis=0 bookkeeperClientHealthCheckEnabled=true @@ -65,7 +65,7 @@ bookkeeperClientRegionawarePolicyEnabled=false bookkeeperClientMinNumRacksPerWriteQuorum=2 bookkeeperClientEnforceMinNumRacksPerWriteQuorum=false bookkeeperClientReorderReadSequenceEnabled=false -bookkeeperClientIsolationGroups="test_group" +bookkeeperClientIsolationGroups= managedLedgerDefaultEnsembleSize=3 managedLedgerDefaultWriteQuorum=2 managedLedgerDefaultAckQuorum=2 diff --git a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone_with_rocksdb.conf b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone_with_rocksdb.conf index d8b26bbbfa99d..46c876686b05b 100644 --- a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone_with_rocksdb.conf +++ b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone_with_rocksdb.conf @@ -17,19 +17,19 @@ # under the License. # -applicationName="pulsar_broker" +applicationName=pulsar_broker metadataStoreUrl= configurationMetadataStoreUrl= -brokerServicePort=6650 -brokerServicePortTls=6651 -webServicePort=8080 +brokerServicePort=0 +brokerServicePortTls= +webServicePort=0 allowLoopback=true -webServicePortTls=4443 +webServicePortTls= bindAddress=0.0.0.0 advertisedAddress= advertisedListeners= internalListenerName=internal -clusterName="test_cluster" +clusterName=test_cluster brokerShutdownTimeoutMs=3000 backlogQuotaCheckEnabled=true backlogQuotaCheckIntervalInSeconds=60 @@ -44,17 +44,17 @@ clientLibraryVersionCheckEnabled=false clientLibraryVersionCheckAllowUnversioned=true statusFilePath=/tmp/status.html tlsEnabled=false -tlsCertificateFilePath=/usr/local/conf/pulsar/server.crt -tlsKeyFilePath=/home/local/conf/pulsar/server.key +tlsCertificateFilePath= +tlsKeyFilePath= tlsTrustCertsFilePath= tlsAllowInsecureConnection=false authenticationEnabled=false authorizationEnabled=false -superUserRoles="test_user" -brokerClientAuthenticationPlugin="org.apache.pulsar.client.impl.auth.AuthenticationDisabled" +superUserRoles=test_user +brokerClientAuthenticationPlugin=org.apache.pulsar.client.impl.auth.AuthenticationDisabled brokerClientAuthenticationParameters= -bookkeeperClientAuthenticationPlugin="test_auth_plugin" -bookkeeperClientAuthenticationAppId="test_auth_id" +bookkeeperClientAuthenticationPlugin= +bookkeeperClientAuthenticationAppId=test_auth_id bookkeeperClientTimeoutInSeconds=30 bookkeeperClientSpeculativeReadTimeoutInMillis=0 bookkeeperClientHealthCheckEnabled=true @@ -66,7 +66,7 @@ bookkeeperClientRegionawarePolicyEnabled=false bookkeeperClientMinNumRacksPerWriteQuorum=2 bookkeeperClientEnforceMinNumRacksPerWriteQuorum=false bookkeeperClientReorderReadSequenceEnabled=false -bookkeeperClientIsolationGroups="test_group" +bookkeeperClientIsolationGroups=test_group managedLedgerDefaultEnsembleSize=3 managedLedgerDefaultWriteQuorum=2 managedLedgerDefaultAckQuorum=2 diff --git a/pulsar-broker/src/test/resources/configurations/standalone_no_client_auth.conf b/pulsar-broker/src/test/resources/configurations/standalone_no_client_auth.conf index 4e2fd40298354..6f0d82cef17bc 100644 --- a/pulsar-broker/src/test/resources/configurations/standalone_no_client_auth.conf +++ b/pulsar-broker/src/test/resources/configurations/standalone_no_client_auth.conf @@ -17,8 +17,8 @@ # under the License. # -brokerServicePort=6650 -webServicePort=8080 +brokerServicePort=0 +webServicePort=0 allowLoopback=true clusterName=test_cluster superUserRoles=admin diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/BKCluster.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/BKCluster.java index c2f3f72ec21c0..8d3a90239efd3 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/BKCluster.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/BKCluster.java @@ -30,6 +30,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Matcher; import java.util.regex.Pattern; import lombok.Getter; @@ -49,8 +50,8 @@ import org.apache.bookkeeper.replication.AutoRecoveryMain; import org.apache.bookkeeper.server.conf.BookieConfiguration; import org.apache.bookkeeper.util.IOUtils; -import org.apache.bookkeeper.util.PortManager; import org.apache.commons.io.FileUtils; +import org.apache.pulsar.common.util.PortManager; import org.apache.pulsar.metadata.api.MetadataStoreConfig; import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; @@ -74,6 +75,9 @@ public class BKCluster implements AutoCloseable { protected final ServerConfiguration baseConf; protected final ClientConfiguration baseClientConf; + private final List lockedPorts = new ArrayList<>(); + private final AtomicBoolean closed = new AtomicBoolean(false); + public static class BKClusterConf { private ServerConfiguration baseServerConfiguration; @@ -148,20 +152,24 @@ private BKCluster(BKClusterConf bkClusterConf) throws Exception { @Override public void close() throws Exception { - // stop bookkeeper service - try { - stopBKCluster(); - } catch (Exception e) { - log.error("Got Exception while trying to stop BKCluster", e); - } - // cleanup temp dirs - try { - cleanupTempDirs(); - } catch (Exception e) { - log.error("Got Exception while trying to cleanupTempDirs", e); - } + if (closed.compareAndSet(false, true)) { + // stop bookkeeper service + try { + stopBKCluster(); + } catch (Exception e) { + log.error("Got Exception while trying to stop BKCluster", e); + } + lockedPorts.forEach(PortManager::releaseLockedPort); + lockedPorts.clear(); + // cleanup temp dirs + try { + cleanupTempDirs(); + } catch (Exception e) { + log.error("Got Exception while trying to cleanupTempDirs", e); + } - this.store.close(); + this.store.close(); + } } private File createTempDir(String prefix, String suffix) throws IOException { @@ -229,7 +237,8 @@ private ServerConfiguration newServerConfiguration(int index) throws Exception { int port; if (baseConf.isEnableLocalTransport() || !baseConf.getAllowEphemeralPorts() || clusterConf.bkPort == 0) { - port = PortManager.nextFreePort(); + port = PortManager.nextLockedFreePort(); + lockedPorts.add(port); } else { // bk 4.15 cookie validation finds the same ip:port in case of port 0 // and 2nd bookie's cookie validation fails @@ -399,4 +408,8 @@ private static ServerConfiguration setLoopbackInterfaceAndAllowLoopback(ServerCo serverConf.setAllowLoopback(true); return serverConf; } + + public boolean isClosed() { + return closed.get(); + } } From cf967630262c2ea3d1aa17aa5604295b6ce8b37c Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 10 Jun 2024 19:30:24 +0300 Subject: [PATCH 549/699] [fix][cli] Fix Pulsar standalone "--wipe-data" (#22885) (cherry picked from commit f6eceedbded53cded4dd751206ebb51d2867e978) (cherry picked from commit d9928ef944f5b2ac8eb2e817704a7d3b69e11d5e) --- .../main/java/org/apache/pulsar/PulsarStandalone.java | 9 ++++++++- .../apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java | 2 ++ .../org/apache/pulsar/metadata/bookkeeper/BKCluster.java | 1 + 3 files changed, 11 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java index a2a101fe394cc..360f8caeba6ec 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarStandalone.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar; +import static org.apache.commons.io.FileUtils.cleanDirectory; import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; import static org.apache.pulsar.common.naming.SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN; import com.beust.jcommander.Parameter; @@ -25,6 +26,7 @@ import com.google.common.collect.Sets; import io.netty.util.internal.PlatformDependent; import java.io.File; +import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; import java.util.List; @@ -443,7 +445,12 @@ public void close() { void startBookieWithMetadataStore() throws Exception { if (StringUtils.isBlank(metadataStoreUrl)){ log.info("Starting BK with RocksDb metadata store"); - metadataStoreUrl = "rocksdb://" + Paths.get(metadataDir).toAbsolutePath(); + Path metadataDirPath = Paths.get(metadataDir); + metadataStoreUrl = "rocksdb://" + metadataDirPath.toAbsolutePath(); + if (wipeData && Files.exists(metadataDirPath)) { + log.info("Wiping RocksDb metadata store at {}", metadataStoreUrl); + cleanDirectory(metadataDirPath.toFile()); + } } else { log.info("Starting BK with metadata store: {}", metadataStoreUrl); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java b/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java index 63d146a3a1521..4c8d2dbbfa7d3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java @@ -194,6 +194,7 @@ private void runZookeeper(int maxCC) throws IOException { : createTempDirectory("zktest"); if (this.clearOldData) { + LOG.info("Wiping Zookeeper data directory at {}", zkDataDir.getAbsolutePath()); cleanDirectory(zkDataDir); } @@ -291,6 +292,7 @@ private void runBookies(ServerConfiguration baseConf) throws Exception { : createTempDirectory("bk" + i + "test"); if (this.clearOldData) { + LOG.info("Wiping Bookie data directory at {}", bkDataDir.getAbsolutePath()); cleanDirectory(bkDataDir); } diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/BKCluster.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/BKCluster.java index 8d3a90239efd3..fe2b981ffe995 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/BKCluster.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/bookkeeper/BKCluster.java @@ -232,6 +232,7 @@ private ServerConfiguration newServerConfiguration(int index) throws Exception { } if (clusterConf.clearOldData && dataDir.exists()) { + log.info("Wiping Bookie data directory at {}", dataDir.getAbsolutePath()); cleanDirectory(dataDir); } From 78a8e3069fd82efb6d6bf5b7a5fbaae29648707f Mon Sep 17 00:00:00 2001 From: Matteo Merli Date: Mon, 10 Jun 2024 12:39:49 -0700 Subject: [PATCH 550/699] [improve] Upgrade IPAddress to 5.5.0 (#22886) (cherry picked from commit f17d90e528687fc796cc7e9c5c5b7487a3e3723e) (cherry picked from commit caf08c26f586eb1e09a14755916a769b60474008) --- distribution/server/src/assemble/LICENSE.bin.txt | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index dda81b53a4663..3f830e16c5895 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -501,7 +501,7 @@ The Apache Software License, Version 2.0 - io.etcd-jetcd-core-0.7.7.jar - io.etcd-jetcd-grpc-0.7.7.jar * IPAddress - - com.github.seancfoley-ipaddress-5.3.3.jar + - com.github.seancfoley-ipaddress-5.5.0.jar * RxJava - io.reactivex.rxjava3-rxjava-3.0.1.jar * RoaringBitmap diff --git a/pom.xml b/pom.xml index 5aea0b7f98c53..31f0d67094ba8 100644 --- a/pom.xml +++ b/pom.xml @@ -251,7 +251,7 @@ flexible messaging model and an intuitive client API. 0.7.7 2.0 1.10.12 - 5.3.3 + 5.5.0 3.4.3 1.5.2-3 2.0.6 From 48c2b7d6d3d5b27d20e4936d9c6e696848ccc065 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=90=A7=E6=98=93=E5=AE=A2?= Date: Tue, 11 Jun 2024 12:46:04 +0800 Subject: [PATCH 551/699] [fix][misc] Topic name from persistence name should decode local name (#22879) (cherry picked from commit c326d8e2203b6e9be37f4f2066fd7e90a9b9fb54) (cherry picked from commit dae7d8bcf3ed701ba0c77ffb0be221757418d278) --- .../java/org/apache/pulsar/common/naming/TopicName.java | 5 ++--- .../java/org/apache/pulsar/common/naming/TopicNameTest.java | 6 ++++++ 2 files changed, 8 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 eebca0e0d7214..e051e01495dbe 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 @@ -358,17 +358,16 @@ public static String fromPersistenceNamingEncoding(String mlName) { String localName; if (parts.size() == 4) { tenant = parts.get(0); - cluster = null; namespacePortion = parts.get(1); domain = parts.get(2); - localName = parts.get(3); + localName = Codec.decode(parts.get(3)); return String.format("%s://%s/%s/%s", domain, tenant, namespacePortion, localName); } else if (parts.size() == 5) { tenant = parts.get(0); cluster = parts.get(1); namespacePortion = parts.get(2); domain = parts.get(3); - localName = parts.get(4); + localName = Codec.decode(parts.get(4)); return String.format("%s://%s/%s/%s/%s", domain, tenant, cluster, namespacePortion, localName); } else { throw new IllegalArgumentException("Invalid managedLedger name: " + mlName); 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 835045f9167dd..485bea3f1addb 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 @@ -267,6 +267,12 @@ public void testFromPersistenceNamingEncoding() { } catch (IllegalArgumentException e) { // Exception is expected. } + + // case5: local name with special characters e.g. a:b:c + String topicName = "persistent://tenant/namespace/a:b:c"; + String persistentNamingEncoding = "tenant/namespace/persistent/a%3Ab%3Ac"; + assertEquals(TopicName.get(topicName).getPersistenceNamingEncoding(), persistentNamingEncoding); + assertEquals(TopicName.fromPersistenceNamingEncoding(persistentNamingEncoding), topicName); } From 0aa3d4393e8cc8836abc7affc8bbbd43fa7e96eb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Tue, 11 Jun 2024 23:45:12 +0800 Subject: [PATCH 552/699] [improve][broker] Optimize PersistentTopic.getLastDispatchablePosition (#22707) [PersistentTopic#getLastDispatchablePosition](https://github.com/apache/pulsar/blob/master/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java#L3776-L3788) is using by [Reader#hasMessageAvailable](https://github.com/apache/pulsar/blob/master/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Reader.java#L116) , [ConsumerImpl#hasMessageAvailable](https://github.com/apache/pulsar/blob/master/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java#L2440-L2448), [Consumer#getLastMessageIdAsync](https://github.com/apache/pulsar/blob/master/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/Consumer.java#L591-L615). The current implementation is read entries from Bookkeeper(or sth else), which leads to low throughput, high latency and heavy load, this PR is for the purpose of optimization. (cherry picked from commit 266243cae246a6fa52b4b6c626932885ad44cbf4) (cherry picked from commit 912ae3c5590b9ec143b2c6e52daf3ecc1a89d07d) --- .../service/persistent/PersistentTopic.java | 66 +++++++++++++++---- .../buffer/impl/InMemTransactionBuffer.java | 14 +++- .../buffer/impl/TopicTransactionBuffer.java | 11 ++++ .../buffer/impl/TransactionBufferDisable.java | 14 +++- 4 files changed, 89 insertions(+), 16 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 b83a8206b4ab4..b566a6b776908 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 @@ -271,6 +271,9 @@ protected TopicStatsHelper initialValue() { private volatile CloseFutures closeFutures; + // The last position that can be dispatched to consumers + private volatile Position lastDispatchablePosition; + /*** * We use 2 futures to prevent a new closing if there is an in-progress deletion or closing. We make Pulsar return * the in-progress one when it is called the second time. @@ -3476,18 +3479,57 @@ public Position getLastPosition() { @Override public CompletableFuture getLastDispatchablePosition() { - return ManagedLedgerImplUtils.asyncGetLastValidPosition((ManagedLedgerImpl) ledger, entry -> { - MessageMetadata md = Commands.parseMessageMetadata(entry.getDataBuffer()); - // If a messages has marker will filter by AbstractBaseDispatcher.filterEntriesForConsumer - if (Markers.isServerOnlyMarker(md)) { - return false; - } else if (md.hasTxnidMostBits() && md.hasTxnidLeastBits()) { - // Filter-out transaction aborted messages. - TxnID txnID = new TxnID(md.getTxnidMostBits(), md.getTxnidLeastBits()); - return !isTxnAborted(txnID, (PositionImpl) entry.getPosition()); - } - return true; - }, getMaxReadPosition()); + if (lastDispatchablePosition != null) { + return CompletableFuture.completedFuture(lastDispatchablePosition); + } + return ManagedLedgerImplUtils + .asyncGetLastValidPosition((ManagedLedgerImpl) ledger, entry -> { + MessageMetadata md = Commands.parseMessageMetadata(entry.getDataBuffer()); + // If a messages has marker will filter by AbstractBaseDispatcher.filterEntriesForConsumer + if (Markers.isServerOnlyMarker(md)) { + return false; + } else if (md.hasTxnidMostBits() && md.hasTxnidLeastBits()) { + // Filter-out transaction aborted messages. + TxnID txnID = new TxnID(md.getTxnidMostBits(), md.getTxnidLeastBits()); + return !isTxnAborted(txnID, (PositionImpl) entry.getPosition()); + } + return true; + }, getMaxReadPosition()) + .thenApply(position -> { + // Update lastDispatchablePosition to the given position + updateLastDispatchablePosition(position); + return position; + }); + } + + /** + * Update lastDispatchablePosition if the given position is greater than the lastDispatchablePosition. + * + * @param position + */ + public synchronized void updateLastDispatchablePosition(Position position) { + // Update lastDispatchablePosition to null if the position is null, fallback to + // ManagedLedgerImplUtils#asyncGetLastValidPosition + if (position == null) { + lastDispatchablePosition = null; + return; + } + + PositionImpl position0 = (PositionImpl) position; + // If the position is greater than the maxReadPosition, ignore + if (position0.compareTo(getMaxReadPosition()) > 0) { + return; + } + // If the lastDispatchablePosition is null, set it to the position + if (lastDispatchablePosition == null) { + lastDispatchablePosition = position; + return; + } + // If the position is greater than the lastDispatchablePosition, update it + PositionImpl lastDispatchablePosition0 = (PositionImpl) lastDispatchablePosition; + if (position0.compareTo(lastDispatchablePosition0) > 0) { + lastDispatchablePosition = position; + } } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java index bab7b64c608c4..533d0716d413c 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/InMemTransactionBuffer.java @@ -377,8 +377,11 @@ public boolean isTxnAborted(TxnID txnID, PositionImpl readPosition) { @Override public void syncMaxReadPositionForNormalPublish(PositionImpl position, boolean isMarkerMessage) { - if (!isMarkerMessage && maxReadPositionCallBack != null) { - maxReadPositionCallBack.maxReadPositionMovedForward(null, position); + if (!isMarkerMessage) { + updateLastDispatchablePosition(position); + if (maxReadPositionCallBack != null) { + maxReadPositionCallBack.maxReadPositionMovedForward(null, position); + } } } @@ -436,4 +439,11 @@ public long getCommittedTxnCount() { .filter(txnBuffer -> txnBuffer.status.equals(TxnStatus.COMMITTED)) .count(); } + + // ThreadSafe + private void updateLastDispatchablePosition(Position position) { + if (topic instanceof PersistentTopic t) { + t.updateLastDispatchablePosition(position); + } + } } 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 cbf9630658f19..2eb9e0f2ba0d1 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 @@ -297,6 +297,11 @@ private void handleTransactionMessage(TxnID txnId, Position position) { } } + // ThreadSafe + private void updateLastDispatchablePosition(Position position) { + topic.updateLastDispatchablePosition(position); + } + @Override public CompletableFuture openTransactionBufferReader(TxnID txnID, long startSequenceId) { return null; @@ -459,6 +464,8 @@ void removeTxnAndUpdateMaxReadPosition(TxnID txnID) { } else { updateMaxReadPosition((PositionImpl) topic.getManagedLedger().getLastConfirmedEntry(), false); } + // Update the last dispatchable position to null if there is a TXN finished. + updateLastDispatchablePosition(null); } /** @@ -523,6 +530,10 @@ public void syncMaxReadPositionForNormalPublish(PositionImpl position, boolean i } } } + // If the message is a normal message, update the last dispatchable position. + if (!isMarkerMessage) { + updateLastDispatchablePosition(position); + } } @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java index ebd61dbaa82ec..6f5dc0cd4d0dd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/buffer/impl/TransactionBufferDisable.java @@ -99,8 +99,11 @@ public boolean isTxnAborted(TxnID txnID, PositionImpl readPosition) { @Override public void syncMaxReadPositionForNormalPublish(PositionImpl position, boolean isMarkerMessage) { - if (!isMarkerMessage && maxReadPositionCallBack != null) { - maxReadPositionCallBack.maxReadPositionMovedForward(null, position); + if (!isMarkerMessage) { + updateLastDispatchablePosition(position); + if (maxReadPositionCallBack != null) { + maxReadPositionCallBack.maxReadPositionMovedForward(null, position); + } } } @@ -148,4 +151,11 @@ public long getAbortedTxnCount() { public long getCommittedTxnCount() { return 0; } + + // ThreadSafe + private void updateLastDispatchablePosition(Position position) { + if (topic instanceof PersistentTopic t) { + t.updateLastDispatchablePosition(position); + } + } } From 8ba2f710ebe08ea708c5cf670a6b20d4e41a5124 Mon Sep 17 00:00:00 2001 From: Dragos Misca Date: Wed, 31 Jan 2024 10:01:44 -0800 Subject: [PATCH 553/699] [improve][broker] Include runtime dependencies in server distribution (#22001) (cherry picked from commit 57025bc11913680f7aac26ab42399ea8a6fccc05) (cherry picked from commit 3670515b08593b0da5dc56e52e7c32782ad7f618) --- distribution/server/src/assemble/bin.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/distribution/server/src/assemble/bin.xml b/distribution/server/src/assemble/bin.xml index c00e020ff3394..ab3199079b9eb 100644 --- a/distribution/server/src/assemble/bin.xml +++ b/distribution/server/src/assemble/bin.xml @@ -134,7 +134,6 @@ com.datastax.oss:pulsar-functions-runtime-all - com.datastax.oss:pulsar-functions-api-examples From f7ee78dc75c100bd16cde0c3306fd2a6fffae35c Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Thu, 13 Jun 2024 19:18:02 +0800 Subject: [PATCH 554/699] [fix][broker][branch-3.0] The topic might reference a closed ledger (#22860) (#22900) (cherry picked from commit 8be3e8ab7b14451ea31e2a979c1b5fa2d1f993d6) --- .../apache/pulsar/broker/PulsarService.java | 5 + .../pulsar/broker/service/BrokerService.java | 154 +++++++++--------- .../pulsar/broker/service/ReplicatorTest.java | 10 +- .../client/api/OrphanPersistentTopicTest.java | 68 ++++++++ 4 files changed, 150 insertions(+), 87 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 6d3cd98738762..58018be20b642 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 @@ -1940,6 +1940,11 @@ protected BrokerService newBrokerService(PulsarService pulsar) throws Exception return new BrokerService(pulsar, ioEventLoopGroup); } + @VisibleForTesting + public void setTransactionExecutorProvider(TransactionBufferProvider transactionBufferProvider) { + this.transactionBufferProvider = transactionBufferProvider; + } + public void initConfigMetadataSynchronizerIfNeeded() { mutex.lock(); try { 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 45f9bdd7b0f7e..9da21c35c31aa 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 @@ -1044,38 +1044,38 @@ public CompletableFuture> getTopic(final String topic, boolean c return getTopic(TopicName.get(topic), createIfMissing, properties); } + /** + * Retrieves or creates a topic based on the specified parameters. + * 0. If disable PersistentTopics or NonPersistentTopics, it will return a failed future with NotAllowedException. + * 1. If topic future exists in the cache returned directly regardless of whether it fails or timeout. + * 2. If the topic metadata exists, the topic is created regardless of {@code createIfMissing}. + * 3. If the topic metadata not exists, and {@code createIfMissing} is false, + * returns an empty Optional in a CompletableFuture. And this empty future not be added to the map. + * 4. Otherwise, use computeIfAbsent. It returns the existing topic or creates and adds a new topicFuture. + * Any exceptions will remove the topicFuture from the map. + * + * @param topicName The name of the topic, potentially including partition information. + * @param createIfMissing If true, creates the topic if it does not exist. + * @param properties Topic configuration properties used during creation. + * @return CompletableFuture with an Optional of the topic if found or created, otherwise empty. + */ public CompletableFuture> getTopic(final TopicName topicName, boolean createIfMissing, Map properties) { try { - CompletableFuture> topicFuture = topics.get(topicName.toString()); - if (topicFuture != null) { - if (topicFuture.isCompletedExceptionally() - || (topicFuture.isDone() && !topicFuture.getNow(Optional.empty()).isPresent())) { - // Exceptional topics should be recreated. - topics.remove(topicName.toString(), topicFuture); - } else { - // a non-existing topic in the cache shouldn't prevent creating a topic - if (createIfMissing) { - if (topicFuture.isDone() && topicFuture.getNow(Optional.empty()).isPresent()) { - return topicFuture; - } else { - return topicFuture.thenCompose(value -> { - if (!value.isPresent()) { - // retry and create topic - return getTopic(topicName, createIfMissing, properties); - } else { - // in-progress future completed successfully - return CompletableFuture.completedFuture(value); - } - }); - } - } else { - return topicFuture; - } - } + // If topic future exists in the cache returned directly regardless of whether it fails or timeout. + CompletableFuture> tp = topics.get(topicName.toString()); + if (tp != null) { + return tp; } final boolean isPersistentTopic = topicName.getDomain().equals(TopicDomain.persistent); if (isPersistentTopic) { + if (!pulsar.getConfiguration().isEnablePersistentTopics()) { + if (log.isDebugEnabled()) { + log.debug("Broker is unable to load persistent topic {}", topicName); + } + return FutureUtil.failedFuture(new NotAllowedException( + "Broker is unable to load persistent topic")); + } return pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topicName) .thenCompose(exists -> { if (!exists && !createIfMissing) { @@ -1090,44 +1090,48 @@ public CompletableFuture> getTopic(final TopicName topicName, bo throw FutureUtil.wrapToCompletionException(new ServiceUnitNotReadyException(errorInfo)); }).thenCompose(optionalTopicPolicies -> { final TopicPolicies topicPolicies = optionalTopicPolicies.orElse(null); - return topics.computeIfAbsent(topicName.toString(), (tpName) -> { - if (topicName.isPartitioned()) { - final TopicName topicNameEntity = TopicName.get(topicName.getPartitionedTopicName()); - return fetchPartitionedTopicMetadataAsync(topicNameEntity) - .thenCompose((metadata) -> { - // Allow crate non-partitioned persistent topic that name includes - // `partition` - if (metadata.partitions == 0 - || topicName.getPartitionIndex() < metadata.partitions) { - return loadOrCreatePersistentTopic(tpName, createIfMissing, - properties, topicPolicies); - } + if (topicName.isPartitioned()) { + final TopicName topicNameEntity = TopicName.get(topicName.getPartitionedTopicName()); + return fetchPartitionedTopicMetadataAsync(topicNameEntity) + .thenCompose((metadata) -> { + // Allow crate non-partitioned persistent topic that name includes + // `partition` + if (metadata.partitions == 0 + || topicName.getPartitionIndex() < metadata.partitions) { + return topics.computeIfAbsent(topicName.toString(), (tpName) -> + loadOrCreatePersistentTopic(tpName, + createIfMissing, properties, topicPolicies)); + } else { final String errorMsg = String.format("Illegal topic partition name %s with max allowed " + "%d partitions", topicName, metadata.partitions); log.warn(errorMsg); return FutureUtil.failedFuture( new BrokerServiceException.NotAllowedException(errorMsg)); - }); - } - return loadOrCreatePersistentTopic(tpName, createIfMissing, properties, topicPolicies); - }).thenCompose(optionalTopic -> { - if (!optionalTopic.isPresent() && createIfMissing) { - log.warn("[{}] Try to recreate the topic with createIfMissing=true " - + "but the returned topic is empty", topicName); - return getTopic(topicName, createIfMissing, properties); - } - return CompletableFuture.completedFuture(optionalTopic); - }); + } + }); + } else { + return topics.computeIfAbsent(topicName.toString(), (tpName) -> + loadOrCreatePersistentTopic(tpName, createIfMissing, properties, topicPolicies)); + } }); }); } else { - return topics.computeIfAbsent(topicName.toString(), (name) -> { + if (!pulsar.getConfiguration().isEnableNonPersistentTopics()) { + if (log.isDebugEnabled()) { + log.debug("Broker is unable to load non-persistent topic {}", topicName); + } + return FutureUtil.failedFuture(new NotAllowedException( + "Broker is unable to load persistent topic")); + } + if (!topics.containsKey(topicName.toString())) { topicEventsDispatcher.notify(topicName.toString(), TopicEvent.LOAD, EventStage.BEFORE); - if (topicName.isPartitioned()) { - final TopicName partitionedTopicName = TopicName.get(topicName.getPartitionedTopicName()); - return this.fetchPartitionedTopicMetadataAsync(partitionedTopicName).thenCompose((metadata) -> { - if (topicName.getPartitionIndex() < metadata.partitions) { + } + if (topicName.isPartitioned()) { + final TopicName partitionedTopicName = TopicName.get(topicName.getPartitionedTopicName()); + return this.fetchPartitionedTopicMetadataAsync(partitionedTopicName).thenCompose((metadata) -> { + if (topicName.getPartitionIndex() < metadata.partitions) { + return topics.computeIfAbsent(topicName.toString(), (name) -> { topicEventsDispatcher .notify(topicName.toString(), TopicEvent.CREATE, EventStage.BEFORE); @@ -1138,11 +1142,13 @@ public CompletableFuture> getTopic(final TopicName topicName, bo topicEventsDispatcher .notifyOnCompletion(eventFuture, topicName.toString(), TopicEvent.LOAD); return res; - } - topicEventsDispatcher.notify(topicName.toString(), TopicEvent.LOAD, EventStage.FAILURE); - return CompletableFuture.completedFuture(Optional.empty()); - }); - } else if (createIfMissing) { + }); + } + topicEventsDispatcher.notify(topicName.toString(), TopicEvent.LOAD, EventStage.FAILURE); + return CompletableFuture.completedFuture(Optional.empty()); + }); + } else if (createIfMissing) { + return topics.computeIfAbsent(topicName.toString(), (name) -> { topicEventsDispatcher.notify(topicName.toString(), TopicEvent.CREATE, EventStage.BEFORE); CompletableFuture> res = createNonPersistentTopic(name); @@ -1152,11 +1158,15 @@ public CompletableFuture> getTopic(final TopicName topicName, bo topicEventsDispatcher .notifyOnCompletion(eventFuture, topicName.toString(), TopicEvent.LOAD); return res; - } else { + }); + } else { + CompletableFuture> topicFuture = topics.get(topicName.toString()); + if (topicFuture == null) { topicEventsDispatcher.notify(topicName.toString(), TopicEvent.LOAD, EventStage.FAILURE); - return CompletableFuture.completedFuture(Optional.empty()); + topicFuture = CompletableFuture.completedFuture(Optional.empty()); } - }); + return topicFuture; + } } } catch (IllegalArgumentException e) { log.warn("[{}] Illegalargument exception when loading topic", topicName, e); @@ -1295,15 +1305,9 @@ private CompletableFuture> createNonPersistentTopic(String topic CompletableFuture> topicFuture = new CompletableFuture<>(); topicFuture.exceptionally(t -> { pulsarStats.recordTopicLoadFailed(); + pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); return null; }); - if (!pulsar.getConfiguration().isEnableNonPersistentTopics()) { - if (log.isDebugEnabled()) { - log.debug("Broker is unable to load non-persistent topic {}", topic); - } - return FutureUtil.failedFuture( - new NotAllowedException("Broker is not unable to load non-persistent topic")); - } final long topicCreateTimeMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()); NonPersistentTopic nonPersistentTopic; try { @@ -1326,7 +1330,6 @@ private CompletableFuture> createNonPersistentTopic(String topic }).exceptionally(ex -> { log.warn("Replication check failed. Removing topic from topics list {}, {}", topic, ex.getCause()); nonPersistentTopic.stopReplProducers().whenComplete((v, exception) -> { - pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); topicFuture.completeExceptionally(ex); }); return null; @@ -1579,14 +1582,6 @@ protected CompletableFuture> loadOrCreatePersistentTopic(final S final CompletableFuture> topicFuture = FutureUtil.createFutureWithTimeout( Duration.ofSeconds(pulsar.getConfiguration().getTopicLoadTimeoutSeconds()), executor(), () -> FAILED_TO_LOAD_TOPIC_TIMEOUT_EXCEPTION); - if (!pulsar.getConfiguration().isEnablePersistentTopics()) { - if (log.isDebugEnabled()) { - log.debug("Broker is unable to load persistent topic {}", topic); - } - topicFuture.completeExceptionally(new NotAllowedException( - "Broker is unable to load persistent topic")); - return topicFuture; - } checkTopicNsOwnership(topic) .thenRun(() -> { @@ -1609,6 +1604,7 @@ protected CompletableFuture> loadOrCreatePersistentTopic(final S } } }).exceptionally(ex -> { + pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); topicFuture.completeExceptionally(ex.getCause()); return null; }); @@ -1779,6 +1775,7 @@ public void openLedgerComplete(ManagedLedger ledger, Object ctx) { + " topic", topic, FutureUtil.getException(topicFuture)); executor().submit(() -> { persistentTopic.close().whenComplete((ignore, ex) -> { + topics.remove(topic, topicFuture); if (ex != null) { log.warn("[{}] Get an error when closing topic.", topic, ex); @@ -1795,6 +1792,7 @@ public void openLedgerComplete(ManagedLedger ledger, Object ctx) { + " Removing topic from topics list {}, {}", topic, ex); executor().submit(() -> { persistentTopic.close().whenComplete((ignore, closeEx) -> { + topics.remove(topic, topicFuture); if (closeEx != null) { log.warn("[{}] Get an error when closing topic.", topic, closeEx); 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 1176106735bb8..3cc2ca2457a4b 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 @@ -19,12 +19,10 @@ package org.apache.pulsar.broker.service; import static org.apache.pulsar.broker.BrokerTestUtil.newUniqueName; -import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.retryStrategically; import static org.mockito.ArgumentMatchers.eq; 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.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; @@ -1404,13 +1402,6 @@ public void testCleanupTopic() throws Exception { // Ok } - final CompletableFuture> timedOutTopicFuture = topicFuture; - // timeout topic future should be removed from cache - retryStrategically((test) -> pulsar1.getBrokerService().getTopic(topicName, false) != timedOutTopicFuture, 5, - 1000); - - assertNotEquals(timedOutTopicFuture, pulsar1.getBrokerService().getTopics().get(topicName)); - try { Consumer consumer = client1.newConsumer().topic(topicName).subscriptionType(SubscriptionType.Shared) .subscriptionName("my-subscriber-name").subscribeAsync().get(100, TimeUnit.MILLISECONDS); @@ -1422,6 +1413,7 @@ public void testCleanupTopic() throws Exception { ManagedLedgerImpl ml = (ManagedLedgerImpl) mlFactory.open(topicMlName + "-2"); mlFuture.complete(ml); + // Re-create topic will success. Consumer consumer = client1.newConsumer().topic(topicName).subscriptionName("my-subscriber-name") .subscriptionType(SubscriptionType.Shared).subscribeAsync() .get(2 * topicLoadTimeoutSeconds, TimeUnit.SECONDS); 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 7a6189702dd8c..54b9ff1f2ea27 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 @@ -18,7 +18,9 @@ */ package org.apache.pulsar.client.api; +import static org.apache.pulsar.broker.service.persistent.PersistentTopic.DEDUPLICATION_CURSOR_NAME; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import java.util.List; import java.util.Map; @@ -26,12 +28,16 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; 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.TopicPoliciesService; import org.apache.pulsar.broker.service.TopicPolicyListener; +import org.apache.pulsar.broker.transaction.buffer.TransactionBuffer; +import org.apache.pulsar.broker.transaction.buffer.TransactionBufferProvider; +import org.apache.pulsar.broker.transaction.buffer.impl.TransactionBufferDisable; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TopicPolicies; import org.awaitility.Awaitility; @@ -104,4 +110,66 @@ public void testNoOrphanTopicAfterCreateTimeout() throws Exception { admin.topics().delete(tpName, false); pulsar.getConfig().setTopicLoadTimeoutSeconds(originalTopicLoadTimeoutSeconds); } + + @Test + public void testCloseLedgerThatTopicAfterCreateTimeout() throws Exception { + // Make the topic loading timeout faster. + long originalTopicLoadTimeoutSeconds = pulsar.getConfig().getTopicLoadTimeoutSeconds(); + int topicLoadTimeoutSeconds = 1; + pulsar.getConfig().setTopicLoadTimeoutSeconds(topicLoadTimeoutSeconds); + pulsar.getConfig().setBrokerDeduplicationEnabled(true); + pulsar.getConfig().setTransactionCoordinatorEnabled(true); + String tpName = BrokerTestUtil.newUniqueName("persistent://public/default/tp2"); + + // Mock message deduplication recovery speed topicLoadTimeoutSeconds + String mlPath = BrokerService.MANAGED_LEDGER_PATH_ZNODE + "/" + + TopicName.get(tpName).getPersistenceNamingEncoding() + "/" + DEDUPLICATION_CURSOR_NAME; + mockZooKeeper.delay(topicLoadTimeoutSeconds * 1000, (op, path) -> { + if (mlPath.equals(path)) { + log.info("Topic load timeout: " + path); + return true; + } + return false; + }); + + // First load topic will trigger timeout + // The first topic load will trigger a timeout. When the topic closes, it will call transactionBuffer.close. + // Here, we simulate a sleep to ensure that the ledger is not immediately closed. + TransactionBufferProvider mockTransactionBufferProvider = new TransactionBufferProvider() { + @Override + public TransactionBuffer newTransactionBuffer(Topic originTopic) { + return new TransactionBufferDisable(originTopic) { + @SneakyThrows + @Override + public CompletableFuture closeAsync() { + Thread.sleep(500); + return super.closeAsync(); + } + }; + } + }; + TransactionBufferProvider originalTransactionBufferProvider = pulsar.getTransactionBufferProvider(); + pulsar.setTransactionExecutorProvider(mockTransactionBufferProvider); + CompletableFuture> firstLoad = pulsar.getBrokerService().getTopic(tpName, true); + Awaitility.await().ignoreExceptions().atMost(5, TimeUnit.SECONDS) + .pollInterval(100, TimeUnit.MILLISECONDS) + // assert first create topic timeout + .untilAsserted(() -> { + assertTrue(firstLoad.isCompletedExceptionally()); + }); + + // Once the first load topic times out, immediately to load the topic again. + Producer producer = pulsarClient.newProducer().topic(tpName).create(); + for (int i = 0; i < 10; i++) { + MessageId send = producer.send("msg".getBytes()); + Thread.sleep(100); + assertNotNull(send); + } + + // set to back + pulsar.setTransactionExecutorProvider(originalTransactionBufferProvider); + pulsar.getConfig().setTopicLoadTimeoutSeconds(originalTopicLoadTimeoutSeconds); + pulsar.getConfig().setBrokerDeduplicationEnabled(false); + pulsar.getConfig().setTransactionCoordinatorEnabled(false); + } } From ee21988d57c3a3edd0fcc3c8438156480342fb81 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Fri, 14 Jun 2024 17:02:33 +0800 Subject: [PATCH 555/699] [fix][cli] Fix the pulsar-daemon parameter passthrough syntax (#22905) Co-authored-by: Lari Hotari (cherry picked from commit 7a21918cb70e6da33e1829d1f28d21bdd03be799) (cherry picked from commit d34f522615eac190084461c6dff78b812c3dac82) --- bin/pulsar-daemon | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/bin/pulsar-daemon b/bin/pulsar-daemon index 210162b6a2190..2c05cb5c49dab 100755 --- a/bin/pulsar-daemon +++ b/bin/pulsar-daemon @@ -157,7 +157,7 @@ start () echo starting $command, logging to $logfile echo Note: Set immediateFlush to true in conf/log4j2.yaml will guarantee the logging event is flushing to disk immediately. The default behavior is switched off due to performance considerations. pulsar=$PULSAR_HOME/bin/pulsar - nohup $pulsar $command "$1" > "$out" 2>&1 < /dev/null & + nohup $pulsar $command "$@" > "$out" 2>&1 < /dev/null & echo $! > $pid sleep 1; head $out sleep 2; @@ -216,7 +216,7 @@ stop () case $startStop in (start) - start "$*" + start "$@" ;; (stop) @@ -224,21 +224,20 @@ case $startStop in ;; (restart) - forceStopFlag=$(echo "$*"|grep "\-force") - if [[ "$forceStopFlag" != "" ]] + if [[ "$1" == "-force" ]] then - stop "-force" + stop -force + # remove "-force" from the arguments + shift else stop fi if [ "$?" == 0 ] then - sleep 3 - paramaters="$*" - startParamaters=${paramaters//-force/} - start "$startParamaters" + sleep 3 + start "$@" else - echo "WARNNING : $command failed restart, for $command is not stopped completely." + echo "WARNNING : $command failed restart, for $command is not stopped completely." fi ;; From 7e7e2afb8fb293302247838344af7513db90312f Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Fri, 14 Jun 2024 20:05:09 +0800 Subject: [PATCH 556/699] [fix][broker] Fix topic status for oldestBacklogMessageAgeSeconds continuously increases even when there is no backlog. (#22907) (cherry picked from commit 6831231e7aeffa39c4d79f5983ef9dc7ba25c449) (cherry picked from commit 73b50e54792759e0ba823218b269101e4c59f727) --- .../service/persistent/PersistentTopic.java | 24 ++- .../service/BacklogQuotaManagerTest.java | 180 +++++++++++++++++- 2 files changed, 194 insertions(+), 10 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 b566a6b776908..920ce65271b69 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 @@ -1407,7 +1407,7 @@ private CompletableFuture delete(boolean failIfHasSubscriptions, return FutureUtil.failedFuture( new TopicBusyException("Topic has subscriptions: " + subscriptions.keys())); } else if (failIfHasBacklogs) { - if (hasBacklogs()) { + if (hasBacklogs(false)) { List backlogSubs = subscriptions.values().stream() .filter(sub -> sub.getNumberOfEntriesInBacklog(false) > 0) @@ -2488,12 +2488,9 @@ public CompletableFuture asyncGetStats(boolean getPreciseBacklog stats.backlogQuotaLimitTime = getBacklogQuota(BacklogQuotaType.message_age).getLimitTime(); TimeBasedBacklogQuotaCheckResult backlogQuotaCheckResult = timeBasedBacklogQuotaCheckResult; - stats.oldestBacklogMessageAgeSeconds = (backlogQuotaCheckResult == null) - ? (long) -1 - : TimeUnit.MILLISECONDS.toSeconds( - Clock.systemUTC().millis() - backlogQuotaCheckResult.getPositionPublishTimestampInMillis()); - + stats.oldestBacklogMessageAgeSeconds = getBestEffortOldestUnacknowledgedMessageAgeSeconds(); stats.oldestBacklogMessageSubscriptionName = (backlogQuotaCheckResult == null) + || !hasBacklogs(getStatsOptions.isGetPreciseBacklog()) ? null : backlogQuotaCheckResult.getCursorName(); @@ -2749,7 +2746,7 @@ public boolean isActive(InactiveTopicDeleteMode deleteMode) { } break; case delete_when_subscriptions_caught_up: - if (hasBacklogs()) { + if (hasBacklogs(false)) { return true; } break; @@ -2762,8 +2759,8 @@ public boolean isActive(InactiveTopicDeleteMode deleteMode) { } } - private boolean hasBacklogs() { - return subscriptions.values().stream().anyMatch(sub -> sub.getNumberOfEntriesInBacklog(false) > 0); + private boolean hasBacklogs(boolean getPreciseBacklog) { + return subscriptions.values().stream().anyMatch(sub -> sub.getNumberOfEntriesInBacklog(getPreciseBacklog) > 0); } @Override @@ -3143,6 +3140,9 @@ public boolean isSizeBacklogExceeded() { @Override public long getBestEffortOldestUnacknowledgedMessageAgeSeconds() { + if (!hasBacklogs(false)) { + return 0; + } TimeBasedBacklogQuotaCheckResult result = timeBasedBacklogQuotaCheckResult; if (result == null) { return -1; @@ -3230,6 +3230,9 @@ public CompletableFuture checkTimeBacklogExceeded() { } if (brokerService.pulsar().getConfiguration().isPreciseTimeBasedBacklogQuotaCheck()) { + if (!hasBacklogs(true)) { + return CompletableFuture.completedFuture(false); + } CompletableFuture future = new CompletableFuture<>(); // Check if first unconsumed message(first message after mark delete position) // for slowest cursor's has expired. @@ -3283,6 +3286,9 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { return future; } else { try { + if (!hasBacklogs(false)) { + return CompletableFuture.completedFuture(false); + } EstimateTimeBasedBacklogQuotaCheckResult checkResult = estimatedTimeBasedBacklogQuotaCheck(oldestMarkDeletePosition); if (checkResult.getEstimatedOldestUnacknowledgedMessageTimestamp() != null) { 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 f194dfe1340e9..2bdd76469ef4d 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 @@ -27,6 +27,7 @@ import static org.assertj.core.api.AssertionsForClassTypes.within; 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.google.common.collect.Sets; @@ -284,8 +285,12 @@ public void testBacklogQuotaWithReader() throws Exception { } private TopicStats getTopicStats(String topic1) throws PulsarAdminException { + return getTopicStats(topic1, true); + } + + private TopicStats getTopicStats(String topic1, boolean getPreciseBacklog) throws PulsarAdminException { TopicStats stats = - admin.topics().getStats(topic1, GetStatsOptions.builder().getPreciseBacklog(true).build()); + admin.topics().getStats(topic1, GetStatsOptions.builder().getPreciseBacklog(getPreciseBacklog).build()); return stats; } @@ -490,9 +495,117 @@ public void backlogsStatsPrecise() throws PulsarAdminException, PulsarClientExce // Cache should be used, since position hasn't changed assertThat(getReadEntries(topic1)).isEqualTo(readEntries); + + // Move subscription 1 and 2 to end + Message msg = consumer1.receive(); + consumer1.acknowledge(msg); + consumer2.acknowledge(secondOldestMessage); + for (int i = 0; i < 2; i++) { + Message message = consumer2.receive(); + log.info("Subscription 2 about to ack message ID {}", message.getMessageId()); + consumer2.acknowledge(message); + } + + log.info("Subscription 1 and 2 moved to end. Now should not backlog"); + waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); + + topicStats = getTopicStats(topic1); + 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.getOldestBacklogMessageSubscriptionName()).isNull(); + + metrics = prometheusMetricsClient.getMetrics(); + 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).isEqualTo(0); + + // producer should create success. + Producer producer2 = createProducer(client, topic1); + assertNotNull(producer2); } } + @Test + public void backlogsStatsPreciseWithNoBacklog() throws PulsarAdminException, PulsarClientException, InterruptedException { + config.setPreciseTimeBasedBacklogQuotaCheck(true); + config.setExposePreciseBacklogInPrometheus(true); + final String namespace = "prop/ns-quota"; + assertEquals(admin.namespaces().getBacklogQuotaMap(namespace), new HashMap<>()); + final int timeLimitSeconds = 2; + admin.namespaces().setBacklogQuota( + namespace, + BacklogQuota.builder() + .limitTime(timeLimitSeconds) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build(), + message_age); + + try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) + .maxBackoffInterval(5, SECONDS) + .statsInterval(0, SECONDS).build()) { + final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); + + final String subName1 = "c1"; + final int numMsgs = 4; + + 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++) { + MessageId send = producer.send(content); + System.out.println(i + ":msg:" + MILLISECONDS.toSeconds(System.currentTimeMillis())); + } + + String c1MarkDeletePositionBefore = + admin.topics().getInternalStats(topic1).cursors.get(subName1).markDeletePosition; + + // Move subscription 1 to end + for (int i = 0; i < numMsgs; i++) { + Message message1 = consumer1.receive(); + consumer1.acknowledge(message1); + } + + // This code will wait about 4~5 Seconds, to make sure the oldest message is 4~5 seconds old + c1MarkDeletePositionBefore = waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); + + Metrics metrics = prometheusMetricsClient.getMetrics(); + TopicStats topicStats = getTopicStats(topic1); + + 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.getOldestBacklogMessageSubscriptionName()).isNull(); + + 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).isEqualTo(0); + + // producer should create success. + Producer producer2 = createProducer(client, topic1); + assertNotNull(producer2); + } + config.setPreciseTimeBasedBacklogQuotaCheck(false); + config.setExposePreciseBacklogInPrometheus(false); + } + private long getReadEntries(String topic1) { return ((PersistentTopic) pulsar.getBrokerService().getTopicReference(topic1).get()) .getManagedLedger().getStats().getEntriesReadTotalCount(); @@ -597,6 +710,71 @@ public void backlogsStatsNotPrecise() throws PulsarAdminException, PulsarClientE config.setManagedLedgerMaxEntriesPerLedger(MAX_ENTRIES_PER_LEDGER); } + @Test + public void backlogsStatsNotPreciseWithNoBacklog() throws PulsarAdminException, PulsarClientException, InterruptedException { + config.setPreciseTimeBasedBacklogQuotaCheck(false); + config.setExposePreciseBacklogInPrometheus(false); + config.setManagedLedgerMaxEntriesPerLedger(6); + final String namespace = "prop/ns-quota"; + assertEquals(admin.namespaces().getBacklogQuotaMap(namespace), new HashMap<>()); + final int timeLimitSeconds = 2; + admin.namespaces().setBacklogQuota( + namespace, + BacklogQuota.builder() + .limitTime(timeLimitSeconds) + .retentionPolicy(BacklogQuota.RetentionPolicy.producer_exception) + .build(), + message_age); + + try (PulsarClient client = PulsarClient.builder().serviceUrl(adminUrl.toString()) + .maxBackoffInterval(3, SECONDS) + .statsInterval(0, SECONDS).build()) { + final String topic1 = "persistent://prop/ns-quota/topic2" + UUID.randomUUID(); + + final String subName1 = "brandNewC1"; + final int numMsgs = 5; + + Consumer consumer1 = client.newConsumer().topic(topic1).subscriptionName(subName1) + .acknowledgmentGroupTime(0, SECONDS) + .isAckReceiptEnabled(true) + .subscribe(); + Producer producer = createProducer(client, topic1); + + byte[] content = new byte[1024]; + for (int i = 0; i < numMsgs; i++) { + producer.send(content); + } + + String c1MarkDeletePositionBefore = + admin.topics().getInternalStats(topic1).cursors.get(subName1).markDeletePosition; + + log.info("Moved subscription 1 to end"); + for (int i = 0; i < numMsgs; i++) { + consumer1.acknowledge(consumer1.receive()); + } + + c1MarkDeletePositionBefore = waitForMarkDeletePositionToChange(topic1, subName1, c1MarkDeletePositionBefore); + waitForQuotaCheckToRunTwice(); + + // backlog and backlogAceSeconds should be 0 + TopicStats topicStats = getTopicStats(topic1, false); + Metrics metrics = prometheusMetricsClient.getMetrics(); + assertEquals(topicStats.getSubscriptions().get(subName1).getMsgBacklog(), 0); + assertThat(topicStats.getOldestBacklogMessageSubscriptionName()).isNull(); + assertThat(topicStats.getOldestBacklogMessageAgeSeconds()).isEqualTo(0); + Metric backlogAgeMetric = + metrics.findSingleMetricByNameAndLabels("pulsar_storage_backlog_age_seconds", + Pair.of("topic", topic1)); + assertThat(backlogAgeMetric.value).isEqualTo(0); + + // producer should create success. + Producer producer2 = createProducer(client, topic1); + assertNotNull(producer2); + + config.setManagedLedgerMaxEntriesPerLedger(MAX_ENTRIES_PER_LEDGER); + } + } + private void unloadAndLoadTopic(String topic, Producer producer) throws PulsarAdminException, PulsarClientException { admin.topics().unload(topic); From 1d80bd3955b787621ab484048159356e71ae1a02 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Tue, 18 Jun 2024 17:47:55 +0800 Subject: [PATCH 557/699] fix: cannot find symbol from cherry-pick 73b50e (cherry picked from commit f089d4f5d599cb3c6648e1af38fa751072caf97c) --- .../pulsar/broker/service/persistent/PersistentTopic.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 920ce65271b69..e3ffb4974689a 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 @@ -2490,7 +2490,7 @@ public CompletableFuture asyncGetStats(boolean getPreciseBacklog TimeBasedBacklogQuotaCheckResult backlogQuotaCheckResult = timeBasedBacklogQuotaCheckResult; stats.oldestBacklogMessageAgeSeconds = getBestEffortOldestUnacknowledgedMessageAgeSeconds(); stats.oldestBacklogMessageSubscriptionName = (backlogQuotaCheckResult == null) - || !hasBacklogs(getStatsOptions.isGetPreciseBacklog()) + || !hasBacklogs(getPreciseBacklog) ? null : backlogQuotaCheckResult.getCursorName(); From a4094fac6ecc2b218870b705ffca12aa80505251 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 23 May 2024 21:15:16 +0800 Subject: [PATCH 558/699] [fix] [client] PIP-344 Do not create partitioned metadata when calling pulsarClient.getPartitionsForTopic(topicName) (#22206) (cherry picked from commit 4e5c0bcc2b44c33a966287b86c2c235be249dc51) (cherry picked from commit 9e59dd06d8478623dcd9e79200b89041c7ffb618) --- .../admin/impl/PersistentTopicsBase.java | 26 +- .../pulsar/broker/service/ServerCnx.java | 110 +++- .../admin/GetPartitionMetadataTest.java | 473 ++++++++++++++++++ .../broker/admin/TopicAutoCreationTest.java | 3 +- .../BrokerServiceAutoTopicCreationTest.java | 4 +- .../broker/service/BrokerServiceTest.java | 10 +- .../service/BrokerServiceThrottlingTest.java | 2 +- .../pulsar/broker/service/ServerCnxTest.java | 2 +- .../buffer/TransactionLowWaterMarkTest.java | 4 +- .../client/api/BrokerServiceLookupTest.java | 2 +- .../pulsar/client/api/PulsarClient.java | 23 +- .../client/impl/BinaryProtoLookupService.java | 13 +- .../client/impl/ConsumerBuilderImpl.java | 4 +- .../pulsar/client/impl/HttpLookupService.java | 7 +- .../pulsar/client/impl/LookupService.java | 27 +- .../client/impl/MultiTopicsConsumerImpl.java | 2 +- .../pulsar/client/impl/PulsarClientImpl.java | 32 +- .../TransactionCoordinatorClientImpl.java | 3 +- .../impl/MultiTopicsConsumerImplTest.java | 12 +- .../client/impl/PulsarClientImplTest.java | 3 +- .../pulsar/common/protocol/Commands.java | 7 +- .../apache/pulsar/common/util/FutureUtil.java | 4 +- pulsar-common/src/main/proto/PulsarApi.proto | 2 + 23 files changed, 687 insertions(+), 88 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java 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 f20897634d685..d5721d249b23e 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 @@ -557,19 +557,29 @@ protected CompletableFuture internalGetPartitionedMeta boolean checkAllowAutoCreation) { return getPartitionedTopicMetadataAsync(topicName, authoritative, checkAllowAutoCreation) .thenCompose(metadata -> { - CompletableFuture ret; - if (metadata.partitions == 0 && !checkAllowAutoCreation) { + if (metadata.partitions > 1) { + // Some clients does not support partitioned topic. + return internalValidateClientVersionAsync().thenApply(__ -> metadata); + } else if (metadata.partitions == 1) { + return CompletableFuture.completedFuture(metadata); + } else { + // metadata.partitions == 0 // The topic may be a non-partitioned topic, so check if it exists here. // However, when checkAllowAutoCreation is true, the client will create the topic if // it doesn't exist. In this case, `partitions == 0` means the automatically created topic // is a non-partitioned topic so we shouldn't check if the topic exists. - ret = internalCheckTopicExists(topicName); - } else if (metadata.partitions > 1) { - ret = internalValidateClientVersionAsync(); - } else { - ret = CompletableFuture.completedFuture(null); + return pulsar().getBrokerService().isAllowAutoTopicCreationAsync(topicName) + .thenCompose(brokerAllowAutoTopicCreation -> { + if (checkAllowAutoCreation) { + // Whether it exists or not, auto create a non-partitioned topic by client. + return CompletableFuture.completedFuture(metadata); + } else { + // If it does not exist, response a Not Found error. + // Otherwise, response a non-partitioned metadata. + return internalCheckTopicExists(topicName).thenApply(__ -> metadata); + } + }); } - return ret.thenApply(__ -> metadata); }); } 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 55e1d96b594e3..52135163a6ab3 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 @@ -81,6 +81,8 @@ import org.apache.pulsar.broker.authentication.AuthenticationState; import org.apache.pulsar.broker.intercept.BrokerInterceptor; import org.apache.pulsar.broker.limiter.ConnectionController; +import org.apache.pulsar.broker.resources.NamespaceResources; +import org.apache.pulsar.broker.resources.TopicResources; import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException; import org.apache.pulsar.broker.service.BrokerServiceException.ServerMetadataException; import org.apache.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException; @@ -578,35 +580,93 @@ protected void handlePartitionMetadataRequest(CommandPartitionedTopicMetadata pa isTopicOperationAllowed(topicName, TopicOperation.LOOKUP, authenticationData, originalAuthData).thenApply( isAuthorized -> { if (isAuthorized) { - unsafeGetPartitionedTopicMetadataAsync(getBrokerService().pulsar(), topicName) - .handle((metadata, ex) -> { - if (ex == null) { - int partitions = metadata.partitions; - commandSender.sendPartitionMetadataResponse(partitions, requestId); - } else { - if (ex instanceof PulsarClientException) { - log.warn("Failed to authorize {} at [{}] on topic {} : {}", getRole(), - remoteAddress, topicName, ex.getMessage()); - commandSender.sendPartitionMetadataResponse(ServerError.AuthorizationError, - ex.getMessage(), requestId); + // Get if exists, respond not found error if not exists. + getBrokerService().isAllowAutoTopicCreationAsync(topicName).thenAccept(brokerAllowAutoCreate -> { + boolean autoCreateIfNotExist = partitionMetadata.isMetadataAutoCreationEnabled(); + if (!autoCreateIfNotExist) { + final NamespaceResources namespaceResources = getBrokerService().pulsar() + .getPulsarResources().getNamespaceResources(); + final TopicResources topicResources = getBrokerService().pulsar().getPulsarResources() + .getTopicResources(); + namespaceResources.getPartitionedTopicResources() + .getPartitionedTopicMetadataAsync(topicName, false) + .handle((metadata, getMetadataEx) -> { + if (getMetadataEx != null) { + log.error("{} {} Failed to get partition metadata", topicName, + ServerCnx.this.toString(), getMetadataEx); + writeAndFlush( + Commands.newPartitionMetadataResponse(ServerError.MetadataError, + "Failed to get partition metadata", + requestId)); + } else if (metadata.isPresent()) { + commandSender.sendPartitionMetadataResponse(metadata.get().partitions, + requestId); + } else if (topicName.isPersistent()) { + topicResources.persistentTopicExists(topicName).thenAccept(exists -> { + if (exists) { + commandSender.sendPartitionMetadataResponse(0, requestId); + return; + } + writeAndFlush(Commands.newPartitionMetadataResponse( + ServerError.TopicNotFound, "", requestId)); + }).exceptionally(ex -> { + log.error("{} {} Failed to get partition metadata", topicName, + ServerCnx.this.toString(), ex); + writeAndFlush( + Commands.newPartitionMetadataResponse(ServerError.MetadataError, + "Failed to check partition metadata", + requestId)); + return null; + }); + } else { + // Regarding non-persistent topic, we do not know whether it exists or not. + // Just return a non-partitioned metadata if partitioned metadata does not + // exist. + // Broker will respond a not found error when doing subscribing or producing if + // broker not allow to auto create topics. + commandSender.sendPartitionMetadataResponse(0, requestId); + } + return null; + }).whenComplete((ignore, ignoreEx) -> { + lookupSemaphore.release(); + if (ignoreEx != null) { + log.error("{} {} Failed to handle partition metadata request", topicName, + ServerCnx.this.toString(), ignoreEx); + } + }); + } else { + // Get if exists, create a new one if not exists. + unsafeGetPartitionedTopicMetadataAsync(getBrokerService().pulsar(), topicName) + .whenComplete((metadata, ex) -> { + lookupSemaphore.release(); + if (ex == null) { + int partitions = metadata.partitions; + commandSender.sendPartitionMetadataResponse(partitions, requestId); } else { - log.warn("Failed to get Partitioned Metadata [{}] {}: {}", remoteAddress, - topicName, ex.getMessage(), ex); - ServerError error = ServerError.ServiceNotReady; - if (ex instanceof RestException restException){ - int responseCode = restException.getResponse().getStatus(); - if (responseCode == NOT_FOUND.getStatusCode()){ - error = ServerError.TopicNotFound; - } else if (responseCode < INTERNAL_SERVER_ERROR.getStatusCode()){ - error = ServerError.MetadataError; + if (ex instanceof PulsarClientException) { + log.warn("Failed to authorize {} at [{}] on topic {} : {}", getRole(), + remoteAddress, topicName, ex.getMessage()); + 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 RestException restException){ + int responseCode = restException.getResponse().getStatus(); + if (responseCode == NOT_FOUND.getStatusCode()){ + error = ServerError.TopicNotFound; + } else if (responseCode < INTERNAL_SERVER_ERROR.getStatusCode()){ + error = ServerError.MetadataError; + } } + commandSender.sendPartitionMetadataResponse(error, ex.getMessage(), + requestId); } - commandSender.sendPartitionMetadataResponse(error, ex.getMessage(), requestId); } - } - lookupSemaphore.release(); - return null; - }); + }); + } + }); } else { final String msg = "Client is not authorized to Get Partition Metadata"; log.warn("[{}] {} with role {} on topic {}", remoteAddress, msg, getPrincipal(), topicName); 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 new file mode 100644 index 0000000000000..51f643d2b7823 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataTest.java @@ -0,0 +1,473 @@ +/* + * 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.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.Semaphore; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.impl.LookupService; +import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.common.naming.TopicDomain; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.TopicType; +import org.apache.pulsar.common.util.FutureUtil; +import org.awaitility.Awaitility; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Test(groups = "broker-admin") +@Slf4j +public class GetPartitionMetadataTest extends ProducerConsumerBase { + + private static final String DEFAULT_NS = "public/default"; + + private PulsarClientImpl clientWithHttpLookup; + private PulsarClientImpl clientWitBinaryLookup; + + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + clientWithHttpLookup = + (PulsarClientImpl) PulsarClient.builder().serviceUrl(pulsar.getWebServiceAddress()).build(); + clientWitBinaryLookup = + (PulsarClientImpl) PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); + } + + @Override + @AfterMethod(alwaysRun = true) + protected void cleanup() throws Exception { + super.internalCleanup(); + if (clientWithHttpLookup != null) { + clientWithHttpLookup.close(); + } + if (clientWitBinaryLookup != null) { + clientWitBinaryLookup.close(); + } + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + } + + private LookupService getLookupService(boolean isUsingHttpLookup) { + if (isUsingHttpLookup) { + return clientWithHttpLookup.getLookup(); + } else { + return clientWitBinaryLookup.getLookup(); + } + } + + @DataProvider(name = "topicDomains") + public Object[][] topicDomains() { + return new Object[][]{ + {TopicDomain.persistent}, + {TopicDomain.non_persistent} + }; + } + + @Test(dataProvider = "topicDomains") + public void testAutoCreatingMetadataWhenCallingOldAPI(TopicDomain topicDomain) throws Exception { + conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); + conf.setDefaultNumPartitions(3); + conf.setAllowAutoTopicCreation(true); + setup(); + + Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); + int lookupPermitsBefore = semaphore.availablePermits(); + + // HTTP client. + final String tp1 = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); + clientWithHttpLookup.getPartitionsForTopic(tp1).join(); + Optional metadata1 = pulsar.getPulsarResources().getNamespaceResources() + .getPartitionedTopicResources() + .getPartitionedTopicMetadataAsync(TopicName.get(tp1), true).join(); + assertTrue(metadata1.isPresent()); + assertEquals(metadata1.get().partitions, 3); + + // Binary client. + final String tp2 = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); + clientWitBinaryLookup.getPartitionsForTopic(tp2).join(); + Optional metadata2 = pulsar.getPulsarResources().getNamespaceResources() + .getPartitionedTopicResources() + .getPartitionedTopicMetadataAsync(TopicName.get(tp2), true).join(); + assertTrue(metadata2.isPresent()); + assertEquals(metadata2.get().partitions, 3); + + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + int lookupPermitsAfter = semaphore.availablePermits(); + assertEquals(lookupPermitsAfter, lookupPermitsBefore); + }); + + // Cleanup. + admin.topics().deletePartitionedTopic(tp1, false); + admin.topics().deletePartitionedTopic(tp2, false); + } + + @DataProvider(name = "autoCreationParamsAll") + public Object[][] autoCreationParamsAll(){ + return new Object[][]{ + // configAllowAutoTopicCreation, paramCreateIfAutoCreationEnabled, isUsingHttpLookup. + {true, true, true, TopicDomain.persistent}, + {true, true, false, TopicDomain.persistent}, + {true, false, true, TopicDomain.persistent}, + {true, false, false, TopicDomain.persistent}, + {false, true, true, TopicDomain.persistent}, + {false, true, false, TopicDomain.persistent}, + {false, false, true, TopicDomain.persistent}, + {false, false, false, TopicDomain.persistent}, + {true, true, true, TopicDomain.non_persistent}, + {true, true, false, TopicDomain.non_persistent}, + {true, false, true, TopicDomain.non_persistent}, + {true, false, false, TopicDomain.non_persistent}, + {false, true, true, TopicDomain.non_persistent}, + {false, true, false, TopicDomain.non_persistent}, + {false, false, true, TopicDomain.non_persistent}, + {false, false, false, TopicDomain.non_persistent} + }; + } + + @Test(dataProvider = "autoCreationParamsAll") + public void testGetMetadataIfNonPartitionedTopicExists(boolean configAllowAutoTopicCreation, + boolean paramMetadataAutoCreationEnabled, + boolean isUsingHttpLookup, + TopicDomain topicDomain) throws Exception { + conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); + conf.setDefaultNumPartitions(3); + conf.setAllowAutoTopicCreation(configAllowAutoTopicCreation); + setup(); + + Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); + int lookupPermitsBefore = semaphore.availablePermits(); + + LookupService lookup = getLookupService(isUsingHttpLookup); + // Create topic. + final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); + final TopicName topicName = TopicName.get(topicNameStr); + admin.topics().createNonPartitionedTopic(topicNameStr); + // Verify. + PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); + PartitionedTopicMetadata response = + lookup.getPartitionedTopicMetadata(topicName, paramMetadataAutoCreationEnabled).join(); + assertEquals(response.partitions, 0); + List partitionedTopics = admin.topics().getPartitionedTopicList("public/default"); + assertFalse(partitionedTopics.contains(topicNameStr)); + List topicList = admin.topics().getList("public/default"); + for (int i = 0; i < 3; i++) { + assertFalse(topicList.contains(topicName.getPartition(i))); + } + + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + int lookupPermitsAfter = semaphore.availablePermits(); + assertEquals(lookupPermitsAfter, lookupPermitsBefore); + }); + + // Cleanup. + client.close(); + admin.topics().delete(topicNameStr, false); + } + + @Test(dataProvider = "autoCreationParamsAll") + public void testGetMetadataIfPartitionedTopicExists(boolean configAllowAutoTopicCreation, + boolean paramMetadataAutoCreationEnabled, + boolean isUsingHttpLookup, + TopicDomain topicDomain) throws Exception { + conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); + conf.setDefaultNumPartitions(3); + conf.setAllowAutoTopicCreation(configAllowAutoTopicCreation); + setup(); + + Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); + int lookupPermitsBefore = semaphore.availablePermits(); + + LookupService lookup = getLookupService(isUsingHttpLookup); + // Create topic. + final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); + final TopicName topicName = TopicName.get(topicNameStr); + admin.topics().createPartitionedTopic(topicNameStr, 3); + // Verify. + PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); + PartitionedTopicMetadata response = + lookup.getPartitionedTopicMetadata(topicName, paramMetadataAutoCreationEnabled).join(); + assertEquals(response.partitions, 3); + List topicList = admin.topics().getList("public/default"); + assertFalse(topicList.contains(topicNameStr)); + + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + int lookupPermitsAfter = semaphore.availablePermits(); + assertEquals(lookupPermitsAfter, lookupPermitsBefore); + }); + + // Cleanup. + client.close(); + admin.topics().deletePartitionedTopic(topicNameStr, false); + } + + @DataProvider(name = "clients") + public Object[][] clients(){ + return new Object[][]{ + // isUsingHttpLookup. + {true, TopicDomain.persistent}, + {false, TopicDomain.non_persistent} + }; + } + + @Test(dataProvider = "clients") + public void testAutoCreatePartitionedTopic(boolean isUsingHttpLookup, TopicDomain topicDomain) throws Exception { + conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); + conf.setDefaultNumPartitions(3); + conf.setAllowAutoTopicCreation(true); + setup(); + + Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); + int lookupPermitsBefore = semaphore.availablePermits(); + + LookupService lookup = getLookupService(isUsingHttpLookup); + // Create topic. + final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); + final TopicName topicName = TopicName.get(topicNameStr); + // Verify. + PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); + PartitionedTopicMetadata response = lookup.getPartitionedTopicMetadata(topicName, true).join(); + assertEquals(response.partitions, 3); + List partitionedTopics = admin.topics().getPartitionedTopicList("public/default"); + assertTrue(partitionedTopics.contains(topicNameStr)); + List topicList = admin.topics().getList("public/default"); + assertFalse(topicList.contains(topicNameStr)); + for (int i = 0; i < 3; i++) { + // The API "getPartitionedTopicMetadata" only creates the partitioned metadata, it will not create the + // partitions. + assertFalse(topicList.contains(topicName.getPartition(i))); + } + + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + int lookupPermitsAfter = semaphore.availablePermits(); + assertEquals(lookupPermitsAfter, lookupPermitsBefore); + }); + + // Cleanup. + client.close(); + admin.topics().deletePartitionedTopic(topicNameStr, false); + } + + @Test(dataProvider = "clients") + public void testAutoCreateNonPartitionedTopic(boolean isUsingHttpLookup, TopicDomain topicDomain) throws Exception { + conf.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); + conf.setAllowAutoTopicCreation(true); + setup(); + + Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); + int lookupPermitsBefore = semaphore.availablePermits(); + + LookupService lookup = getLookupService(isUsingHttpLookup); + // Create topic. + final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); + final TopicName topicName = TopicName.get(topicNameStr); + // Verify. + PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); + PartitionedTopicMetadata response = lookup.getPartitionedTopicMetadata(topicName, true).join(); + assertEquals(response.partitions, 0); + List partitionedTopics = admin.topics().getPartitionedTopicList("public/default"); + assertFalse(partitionedTopics.contains(topicNameStr)); + List topicList = admin.topics().getList("public/default"); + assertFalse(topicList.contains(topicNameStr)); + + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + int lookupPermitsAfter = semaphore.availablePermits(); + assertEquals(lookupPermitsAfter, lookupPermitsBefore); + }); + + // Cleanup. + client.close(); + try { + admin.topics().delete(topicNameStr, false); + } catch (Exception ex) {} + } + + @DataProvider(name = "autoCreationParamsNotAllow") + public Object[][] autoCreationParamsNotAllow(){ + return new Object[][]{ + // configAllowAutoTopicCreation, paramCreateIfAutoCreationEnabled, isUsingHttpLookup. + {true, false, true}, + {true, false, false}, + {false, false, true}, + {false, false, false}, + {false, true, true}, + {false, true, false}, + }; + } + + @Test(dataProvider = "autoCreationParamsNotAllow") + public void testGetMetadataIfNotAllowedCreate(boolean configAllowAutoTopicCreation, + boolean paramMetadataAutoCreationEnabled, + boolean isUsingHttpLookup) throws Exception { + if (!configAllowAutoTopicCreation && paramMetadataAutoCreationEnabled) { + // These test cases are for the following PR. + // Which was described in the Motivation of https://github.com/apache/pulsar/pull/22206. + return; + } + conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); + conf.setDefaultNumPartitions(3); + conf.setAllowAutoTopicCreation(configAllowAutoTopicCreation); + setup(); + + Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); + int lookupPermitsBefore = semaphore.availablePermits(); + + LookupService lookup = getLookupService(isUsingHttpLookup); + // Define topic. + final String topicNameStr = BrokerTestUtil.newUniqueName("persistent://" + DEFAULT_NS + "/tp"); + final TopicName topicName = TopicName.get(topicNameStr); + // Verify. + PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); + try { + lookup.getPartitionedTopicMetadata(TopicName.get(topicNameStr), paramMetadataAutoCreationEnabled).join(); + fail("Expect a not found exception"); + } catch (Exception e) { + log.warn("", e); + Throwable unwrapEx = FutureUtil.unwrapCompletionException(e); + assertTrue(unwrapEx instanceof PulsarClientException.TopicDoesNotExistException + || unwrapEx instanceof PulsarClientException.NotFoundException); + } + + List partitionedTopics = admin.topics().getPartitionedTopicList("public/default"); + pulsar.getPulsarResources().getNamespaceResources().getPartitionedTopicResources().partitionedTopicExists(topicName); + assertFalse(partitionedTopics.contains(topicNameStr)); + List topicList = admin.topics().getList("public/default"); + assertFalse(topicList.contains(topicNameStr)); + for (int i = 0; i < 3; i++) { + assertFalse(topicList.contains(topicName.getPartition(i))); + } + + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + int lookupPermitsAfter = semaphore.availablePermits(); + assertEquals(lookupPermitsAfter, lookupPermitsBefore); + }); + + // Cleanup. + client.close(); + } + + @DataProvider(name = "autoCreationParamsForNonPersistentTopic") + public Object[][] autoCreationParamsForNonPersistentTopic(){ + return new Object[][]{ + // configAllowAutoTopicCreation, paramCreateIfAutoCreationEnabled, isUsingHttpLookup. + {true, true, true}, + {true, true, false}, + {false, true, true}, + {false, true, false}, + {false, false, true} + }; + } + + /** + * Regarding the API "get partitioned metadata" about non-persistent topic. + * The original behavior is: + * param-auto-create = true, broker-config-auto-create = true + * HTTP API: default configuration {@link ServiceConfiguration#getDefaultNumPartitions()} + * binary API: default configuration {@link ServiceConfiguration#getDefaultNumPartitions()} + * param-auto-create = true, broker-config-auto-create = false + * HTTP API: {partitions: 0} + * binary API: {partitions: 0} + * param-auto-create = false + * HTTP API: not found error + * binary API: not support + * This test only guarantees that the behavior is the same as before. The following separated PR will fix the + * incorrect behavior. + */ + @Test(dataProvider = "autoCreationParamsForNonPersistentTopic") + public void testGetNonPersistentMetadataIfNotAllowedCreate(boolean configAllowAutoTopicCreation, + boolean paramMetadataAutoCreationEnabled, + boolean isUsingHttpLookup) throws Exception { + conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); + conf.setDefaultNumPartitions(3); + conf.setAllowAutoTopicCreation(configAllowAutoTopicCreation); + setup(); + + Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); + int lookupPermitsBefore = semaphore.availablePermits(); + + LookupService lookup = getLookupService(isUsingHttpLookup); + // Define topic. + final String topicNameStr = BrokerTestUtil.newUniqueName("non-persistent://" + DEFAULT_NS + "/tp"); + final TopicName topicName = TopicName.get(topicNameStr); + // Verify. + // Regarding non-persistent topic, we do not know whether it exists or not. + // Broker will return a non-partitioned metadata if partitioned metadata does not exist. + PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); + + if (!configAllowAutoTopicCreation && !paramMetadataAutoCreationEnabled && isUsingHttpLookup) { + try { + lookup.getPartitionedTopicMetadata(TopicName.get(topicNameStr), paramMetadataAutoCreationEnabled) + .join(); + Assert.fail("Expected a not found ex"); + } catch (Exception ex) { + // Cleanup. + client.close(); + return; + } + } + + PartitionedTopicMetadata metadata = lookup + .getPartitionedTopicMetadata(TopicName.get(topicNameStr), paramMetadataAutoCreationEnabled).join(); + if (configAllowAutoTopicCreation && paramMetadataAutoCreationEnabled) { + assertEquals(metadata.partitions, 3); + } else { + assertEquals(metadata.partitions, 0); + } + + List partitionedTopics = admin.topics().getPartitionedTopicList("public/default"); + pulsar.getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .partitionedTopicExists(topicName); + if (configAllowAutoTopicCreation && paramMetadataAutoCreationEnabled) { + assertTrue(partitionedTopics.contains(topicNameStr)); + } else { + assertFalse(partitionedTopics.contains(topicNameStr)); + } + + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + int lookupPermitsAfter = semaphore.availablePermits(); + assertEquals(lookupPermitsAfter, lookupPermitsBefore); + }); + + // Cleanup. + client.close(); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java index a75ae78cef393..4712682e71b57 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.admin; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; @@ -133,7 +134,7 @@ public void testPartitionedTopicAutoCreationForbiddenDuringNamespaceDeletion() // we want to skip the "lookup" phase, because it is blocked by the HTTP API LookupService mockLookup = mock(LookupService.class); ((PulsarClientImpl) pulsarClient).setLookup(mockLookup); - when(mockLookup.getPartitionedTopicMetadata(any())).thenAnswer( + when(mockLookup.getPartitionedTopicMetadata(any(), anyBoolean())).thenAnswer( i -> CompletableFuture.completedFuture(new PartitionedTopicMetadata(0))); when(mockLookup.getBroker(any())).thenAnswer(i -> { InetSocketAddress brokerAddress = 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 0a6cffc7685d4..ea5365bcf4b2c 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 @@ -566,13 +566,13 @@ public void testExtensibleLoadManagerImplInternalTopicAutoCreations() try { pulsarClient.newProducer().topic(ExtensibleLoadManagerImpl.BROKER_LOAD_DATA_STORE_TOPIC).create(); Assert.fail("Create should have failed."); - } catch (PulsarClientException.TopicDoesNotExistException e) { + } catch (PulsarClientException.TopicDoesNotExistException | PulsarClientException.NotFoundException e) { // expected } try { pulsarClient.newProducer().topic(ExtensibleLoadManagerImpl.TOP_BUNDLES_LOAD_DATA_STORE_TOPIC).create(); Assert.fail("Create should have failed."); - } catch (PulsarClientException.TopicDoesNotExistException e) { + } catch (PulsarClientException.TopicDoesNotExistException | PulsarClientException.NotFoundException e) { // expected } 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 9f561889aa825..2ce6728e98a08 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 @@ -1022,12 +1022,12 @@ protected void handlePartitionResponse(CommandPartitionedTopicMetadataResponse l // for PMR // 2 lookup will succeed long reqId1 = reqId++; - ByteBuf request1 = Commands.newPartitionMetadataRequest(topicName, reqId1); + ByteBuf request1 = Commands.newPartitionMetadataRequest(topicName, reqId1, true); CompletableFuture f1 = pool.getConnection(resolver.resolveHost()) .thenCompose(clientCnx -> clientCnx.newLookup(request1, reqId1)); long reqId2 = reqId++; - ByteBuf request2 = Commands.newPartitionMetadataRequest(topicName, reqId2); + ByteBuf request2 = Commands.newPartitionMetadataRequest(topicName, reqId2, true); CompletableFuture f2 = pool.getConnection(resolver.resolveHost()) .thenCompose(clientCnx -> { CompletableFuture future = clientCnx.newLookup(request2, reqId2); @@ -1042,17 +1042,17 @@ protected void handlePartitionResponse(CommandPartitionedTopicMetadataResponse l // 3 lookup will fail latchRef.set(new CountDownLatch(1)); long reqId3 = reqId++; - ByteBuf request3 = Commands.newPartitionMetadataRequest(topicName, reqId3); + ByteBuf request3 = Commands.newPartitionMetadataRequest(topicName, reqId3, true); f1 = pool.getConnection(resolver.resolveHost()) .thenCompose(clientCnx -> clientCnx.newLookup(request3, reqId3)); long reqId4 = reqId++; - ByteBuf request4 = Commands.newPartitionMetadataRequest(topicName, reqId4); + ByteBuf request4 = Commands.newPartitionMetadataRequest(topicName, reqId4, true); f2 = pool.getConnection(resolver.resolveHost()) .thenCompose(clientCnx -> clientCnx.newLookup(request4, reqId4)); long reqId5 = reqId++; - ByteBuf request5 = Commands.newPartitionMetadataRequest(topicName, reqId5); + ByteBuf request5 = Commands.newPartitionMetadataRequest(topicName, reqId5, true); CompletableFuture f3 = pool.getConnection(resolver.resolveHost()) .thenCompose(clientCnx -> { CompletableFuture future = clientCnx.newLookup(request5, reqId5); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java index b2cfe63e2e5b4..0d984e0675db5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerServiceThrottlingTest.java @@ -167,7 +167,7 @@ public void testLookupThrottlingForClientByBroker() throws Exception { for (int i = 0; i < totalConsumers; i++) { long reqId = 0xdeadbeef + i; Future f = executor.submit(() -> { - ByteBuf request = Commands.newPartitionMetadataRequest(topicName, reqId); + ByteBuf request = Commands.newPartitionMetadataRequest(topicName, reqId, true); pool.getConnection(resolver.resolveHost()) .thenCompose(clientCnx -> clientCnx.newLookup(request, reqId)) .get(); 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 ad7728319c9a7..626bce380b9de 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 @@ -3573,7 +3573,7 @@ public void handlePartitionMetadataRequestWithServiceNotReady() throws Exception doReturn(false).when(pulsar).isRunning(); assertTrue(channel.isActive()); - ByteBuf clientCommand = Commands.newPartitionMetadataRequest(successTopicName, 1); + ByteBuf clientCommand = Commands.newPartitionMetadataRequest(successTopicName, 1, true); channel.writeInbound(clientCommand); Object response = getResponse(); assertTrue(response instanceof CommandPartitionedTopicMetadataResponse); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionLowWaterMarkTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionLowWaterMarkTest.java index 3901f186d81c7..3f268c4b7c973 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionLowWaterMarkTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionLowWaterMarkTest.java @@ -146,7 +146,7 @@ public void testTransactionBufferLowWaterMark() throws Exception { PartitionedTopicMetadata partitionedTopicMetadata = ((PulsarClientImpl) pulsarClient).getLookup() - .getPartitionedTopicMetadata(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN).get(); + .getPartitionedTopicMetadata(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, false).get(); Transaction lowWaterMarkTxn = null; for (int i = 0; i < partitionedTopicMetadata.partitions; i++) { lowWaterMarkTxn = pulsarClient.newTransaction() @@ -251,7 +251,7 @@ public void testPendingAckLowWaterMark() throws Exception { PartitionedTopicMetadata partitionedTopicMetadata = ((PulsarClientImpl) pulsarClient).getLookup() - .getPartitionedTopicMetadata(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN).get(); + .getPartitionedTopicMetadata(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, false).get(); Transaction lowWaterMarkTxn = null; for (int i = 0; i < partitionedTopicMetadata.partitions; i++) { lowWaterMarkTxn = pulsarClient.newTransaction() 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 0a4c5b7a318b3..9319ea4e876b7 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 @@ -852,7 +852,7 @@ public void testMergeGetPartitionedMetadataRequests() throws Exception { // Verify the request is works after merge the requests. List> futures = new ArrayList<>(); for (int i = 0; i < 100; i++) { - futures.add(lookupService.getPartitionedTopicMetadata(TopicName.get(tpName))); + futures.add(lookupService.getPartitionedTopicMetadata(TopicName.get(tpName), false)); } for (CompletableFuture future : futures) { assertEquals(future.join().partitions, topicPartitions); diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClient.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClient.java index 78952fcaed8b3..6c46bce254f6f 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClient.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/PulsarClient.java @@ -308,14 +308,33 @@ static ClientBuilder builder() { * *

    This can be used to discover the partitions and create {@link Reader}, {@link Consumer} or {@link Producer} * instances directly on a particular partition. - * + * @Deprecated it is not suggested to use now; please use {@link #getPartitionsForTopic(String, boolean)}. * @param topic * the topic name * @return a future that will yield a list of the topic partitions or {@link PulsarClientException} if there was any * error in the operation. + * * @since 2.3.0 */ - CompletableFuture> getPartitionsForTopic(String topic); + @Deprecated + default CompletableFuture> getPartitionsForTopic(String topic) { + return getPartitionsForTopic(topic, true); + } + + /** + * 1. Get the partitions if the topic exists. Return "[{partition-0}, {partition-1}....{partition-n}}]" if a + * partitioned topic exists; return "[{topic}]" if a non-partitioned topic exists. + * 2. When {@param metadataAutoCreationEnabled} is "false", neither the partitioned topic nor non-partitioned + * topic does not exist. You will get an {@link PulsarClientException.NotFoundException} or a + * {@link PulsarClientException.TopicDoesNotExistException}. + * 2-1. You will get a {@link PulsarClientException.NotSupportedException} with metadataAutoCreationEnabled=false + * on an old broker version which does not support getting partitions without partitioned metadata auto-creation. + * 3. When {@param metadataAutoCreationEnabled} is "true," it will trigger an auto-creation for this topic(using + * the default topic auto-creation strategy you set for the broker), and the corresponding result is returned. + * For the result, see case 1. + * @version 3.3.0. + */ + CompletableFuture> getPartitionsForTopic(String topic, boolean metadataAutoCreationEnabled); /** * Close the PulsarClient and release all the resources. diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index 9d01d863143e2..d6f4dd2dcac14 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -121,12 +121,14 @@ public CompletableFuture> getBroker(T * calls broker binaryProto-lookup api to get metadata of partitioned-topic. * */ - public CompletableFuture getPartitionedTopicMetadata(TopicName topicName) { + @Override + public CompletableFuture getPartitionedTopicMetadata( + TopicName topicName, boolean metadataAutoCreationEnabled) { final MutableObject newFutureCreated = new MutableObject<>(); try { return partitionedMetadataInProgress.computeIfAbsent(topicName, tpName -> { - CompletableFuture newFuture = - getPartitionedTopicMetadata(serviceNameResolver.resolveHost(), topicName); + CompletableFuture newFuture = getPartitionedTopicMetadata( + serviceNameResolver.resolveHost(), topicName, metadataAutoCreationEnabled); newFutureCreated.setValue(newFuture); return newFuture; }); @@ -222,13 +224,14 @@ private CompletableFuture> findBroker } private CompletableFuture getPartitionedTopicMetadata(InetSocketAddress socketAddress, - TopicName topicName) { + TopicName topicName, boolean metadataAutoCreationEnabled) { CompletableFuture partitionFuture = new CompletableFuture<>(); client.getCnxPool().getConnection(socketAddress).thenAccept(clientCnx -> { long requestId = client.newRequestId(); - ByteBuf request = Commands.newPartitionMetadataRequest(topicName.toString(), requestId); + ByteBuf request = Commands.newPartitionMetadataRequest(topicName.toString(), requestId, + metadataAutoCreationEnabled); clientCnx.newLookup(request, requestId).whenComplete((r, t) -> { if (t != null) { log.warn("[{}] failed to get Partitioned metadata : {}", topicName, 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 7686d0072cffb..7735f66e7838a 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 @@ -136,9 +136,9 @@ public CompletableFuture> subscribeAsync() { if (deadLetterPolicy == null || StringUtils.isBlank(deadLetterPolicy.getRetryLetterTopic()) || StringUtils.isBlank(deadLetterPolicy.getDeadLetterTopic())) { CompletableFuture retryLetterTopicMetadata = - client.getPartitionedTopicMetadata(oldRetryLetterTopic); + client.getPartitionedTopicMetadata(oldRetryLetterTopic, true); CompletableFuture deadLetterTopicMetadata = - client.getPartitionedTopicMetadata(oldDeadLetterTopic); + client.getPartitionedTopicMetadata(oldDeadLetterTopic, true); applyDLQConfig = CompletableFuture.allOf(retryLetterTopicMetadata, deadLetterTopicMetadata) .thenAccept(__ -> { String retryLetterTopic = topicFirst + "-" + conf.getSubscriptionName() diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java index 6e8c2b4314e17..ba04aaa3b3117 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java @@ -108,10 +108,11 @@ public CompletableFuture> getBroker(T } @Override - public CompletableFuture getPartitionedTopicMetadata(TopicName topicName) { + public CompletableFuture getPartitionedTopicMetadata( + TopicName topicName, boolean metadataAutoCreationEnabled) { String format = topicName.isV2() ? "admin/v2/%s/partitions" : "admin/%s/partitions"; - return httpClient.get(String.format(format, topicName.getLookupName()) + "?checkAllowAutoCreation=true", - PartitionedTopicMetadata.class); + return httpClient.get(String.format(format, topicName.getLookupName()) + "?checkAllowAutoCreation=" + + metadataAutoCreationEnabled, PartitionedTopicMetadata.class); } @Override diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java index 48ef67eae2047..978450ed6894d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java @@ -60,11 +60,30 @@ public interface LookupService extends AutoCloseable { /** * Returns {@link PartitionedTopicMetadata} for a given topic. - * - * @param topicName topic-name - * @return + * Note: this method will try to create the topic partitioned metadata if it does not exist. + * @deprecated Please call {{@link #getPartitionedTopicMetadata(TopicName, boolean)}}. + */ + @Deprecated + default CompletableFuture getPartitionedTopicMetadata(TopicName topicName) { + return getPartitionedTopicMetadata(topicName, true); + } + + /** + * 1.Get the partitions if the topic exists. Return "{partition: n}" if a partitioned topic exists; + * return "{partition: 0}" if a non-partitioned topic exists. + * 2. When {@param metadataAutoCreationEnabled} is "false," neither partitioned topic nor non-partitioned topic + * does not exist. You will get a {@link PulsarClientException.NotFoundException} or + * a {@link PulsarClientException.TopicDoesNotExistException}. + * 2-1. You will get a {@link PulsarClientException.NotSupportedException} with metadataAutoCreationEnabled=false + * on an old broker version which does not support getting partitions without partitioned metadata + * auto-creation. + * 3.When {@param metadataAutoCreationEnabled} is "true," it will trigger an auto-creation for this topic(using + * the default topic auto-creation strategy you set for the broker), and the corresponding result is returned. + * For the result, see case 1. + * @version 3.3.0. */ - CompletableFuture getPartitionedTopicMetadata(TopicName topicName); + CompletableFuture getPartitionedTopicMetadata(TopicName topicName, + boolean metadataAutoCreationEnabled); /** * Returns current SchemaInfo {@link SchemaInfo} for a given topic. 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 f2bce59a1e68e..62b6612fa3c26 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 @@ -954,7 +954,7 @@ public CompletableFuture subscribeAsync(String topicName, boolean createTo CompletableFuture subscribeResult = new CompletableFuture<>(); - client.getPartitionedTopicMetadata(topicName) + client.getPartitionedTopicMetadata(topicName, true) .thenAccept(metadata -> subscribeTopicPartitions(subscribeResult, fullTopicName, metadata.partitions, createTopicIfDoesNotExist)) .exceptionally(ex1 -> { 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 b92e039e5facd..9a5ec8b874bb6 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 @@ -377,7 +377,7 @@ private CompletableFuture> createProducerAsync(String topic, ProducerInterceptors interceptors) { CompletableFuture> producerCreatedFuture = new CompletableFuture<>(); - getPartitionedTopicMetadata(topic).thenAccept(metadata -> { + getPartitionedTopicMetadata(topic, true).thenAccept(metadata -> { if (log.isDebugEnabled()) { log.debug("[{}] Received topic metadata. partitions: {}", topic, metadata.partitions); } @@ -519,7 +519,7 @@ private CompletableFuture> doSingleTopicSubscribeAsync(ConsumerC String topic = conf.getSingleTopic(); - getPartitionedTopicMetadata(topic).thenAccept(metadata -> { + getPartitionedTopicMetadata(topic, true).thenAccept(metadata -> { if (log.isDebugEnabled()) { log.debug("[{}] Received topic metadata. partitions: {}", topic, metadata.partitions); } @@ -659,7 +659,7 @@ protected CompletableFuture> createSingleTopicReaderAsync( CompletableFuture> readerFuture = new CompletableFuture<>(); - getPartitionedTopicMetadata(topic).thenAccept(metadata -> { + getPartitionedTopicMetadata(topic, true).thenAccept(metadata -> { if (log.isDebugEnabled()) { log.debug("[{}] Received topic metadata. partitions: {}", topic, metadata.partitions); } @@ -1028,11 +1028,8 @@ public void reloadLookUp() throws PulsarClientException { } } - public CompletableFuture getNumberOfPartitions(String topic) { - return getPartitionedTopicMetadata(topic).thenApply(metadata -> metadata.partitions); - } - - public CompletableFuture getPartitionedTopicMetadata(String topic) { + public CompletableFuture getPartitionedTopicMetadata( + String topic, boolean metadataAutoCreationEnabled) { CompletableFuture metadataFuture = new CompletableFuture<>(); @@ -1045,7 +1042,7 @@ public CompletableFuture getPartitionedTopicMetadata(S .setMax(conf.getMaxBackoffIntervalNanos(), TimeUnit.NANOSECONDS) .create(); getPartitionedTopicMetadata(topicName, backoff, opTimeoutMs, - metadataFuture, new ArrayList<>()); + metadataFuture, new ArrayList<>(), metadataAutoCreationEnabled); } catch (IllegalArgumentException e) { return FutureUtil.failedFuture(new PulsarClientException.InvalidConfigurationException(e.getMessage())); } @@ -1056,15 +1053,19 @@ private void getPartitionedTopicMetadata(TopicName topicName, Backoff backoff, AtomicLong remainingTime, CompletableFuture future, - List previousExceptions) { + List previousExceptions, + boolean metadataAutoCreationEnabled) { long startTime = System.nanoTime(); - lookup.getPartitionedTopicMetadata(topicName).thenAccept(future::complete).exceptionally(e -> { + CompletableFuture queryFuture = + lookup.getPartitionedTopicMetadata(topicName, metadataAutoCreationEnabled); + queryFuture.thenAccept(future::complete).exceptionally(e -> { remainingTime.addAndGet(-1 * TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); long nextDelay = Math.min(backoff.next(), remainingTime.get()); // skip retry scheduler when set lookup throttle in client or server side which will lead to // `TooManyRequestsException` boolean isLookupThrottling = !PulsarClientException.isRetriableError(e.getCause()) - || e.getCause() instanceof PulsarClientException.AuthenticationException; + || e.getCause() instanceof PulsarClientException.AuthenticationException + || e.getCause() instanceof PulsarClientException.NotFoundException; if (nextDelay <= 0 || isLookupThrottling) { PulsarClientException.setPreviousExceptions(e, previousExceptions); future.completeExceptionally(e); @@ -1076,15 +1077,16 @@ private void getPartitionedTopicMetadata(TopicName topicName, log.warn("[topic: {}] Could not get connection while getPartitionedTopicMetadata -- " + "Will try again in {} ms", topicName, nextDelay); remainingTime.addAndGet(-nextDelay); - getPartitionedTopicMetadata(topicName, backoff, remainingTime, future, previousExceptions); + getPartitionedTopicMetadata(topicName, backoff, remainingTime, future, previousExceptions, + metadataAutoCreationEnabled); }, nextDelay, TimeUnit.MILLISECONDS); return null; }); } @Override - public CompletableFuture> getPartitionsForTopic(String topic) { - return getPartitionedTopicMetadata(topic).thenApply(metadata -> { + public CompletableFuture> getPartitionsForTopic(String topic, boolean metadataAutoCreationEnabled) { + return getPartitionedTopicMetadata(topic, metadataAutoCreationEnabled).thenApply(metadata -> { if (metadata.partitions > 0) { TopicName topicName = TopicName.get(topic); List partitions = new ArrayList<>(metadata.partitions); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/transaction/TransactionCoordinatorClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/transaction/TransactionCoordinatorClientImpl.java index 9e79fc203c225..499627f9c73f2 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/transaction/TransactionCoordinatorClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/transaction/TransactionCoordinatorClientImpl.java @@ -79,7 +79,8 @@ public void start() throws TransactionCoordinatorClientException { @Override public CompletableFuture startAsync() { if (STATE_UPDATER.compareAndSet(this, State.NONE, State.STARTING)) { - return pulsarClient.getLookup().getPartitionedTopicMetadata(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN) + return pulsarClient.getLookup() + .getPartitionedTopicMetadata(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, true) .thenCompose(partitionMeta -> { List> connectFutureList = new ArrayList<>(); if (LOG.isDebugEnabled()) { diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImplTest.java index febec2bff3285..191124bb7b002 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImplTest.java @@ -22,6 +22,7 @@ import static org.apache.pulsar.client.impl.ClientTestFixtures.createExceptionFuture; import static org.apache.pulsar.client.impl.ClientTestFixtures.createPulsarClientMockWithMockedClientCnx; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -153,7 +154,8 @@ private MultiTopicsConsumerImpl createMultiTopicsConsumer( int completionDelayMillis = 100; Schema schema = Schema.BYTES; PulsarClientImpl clientMock = createPulsarClientMockWithMockedClientCnx(executorProvider, internalExecutor); - when(clientMock.getPartitionedTopicMetadata(any())).thenAnswer(invocation -> createDelayedCompletedFuture( + when(clientMock.getPartitionedTopicMetadata(any(), anyBoolean())) + .thenAnswer(invocation -> createDelayedCompletedFuture( new PartitionedTopicMetadata(), completionDelayMillis)); MultiTopicsConsumerImpl impl = new MultiTopicsConsumerImpl( clientMock, consumerConfData, executorProvider, @@ -201,7 +203,8 @@ public void testConsumerCleanupOnSubscribeFailure() { int completionDelayMillis = 10; Schema schema = Schema.BYTES; PulsarClientImpl clientMock = createPulsarClientMockWithMockedClientCnx(executorProvider, internalExecutor); - when(clientMock.getPartitionedTopicMetadata(any())).thenAnswer(invocation -> createExceptionFuture( + when(clientMock.getPartitionedTopicMetadata(any(), anyBoolean())) + .thenAnswer(invocation -> createExceptionFuture( new PulsarClientException.InvalidConfigurationException("a mock exception"), completionDelayMillis)); CompletableFuture> completeFuture = new CompletableFuture<>(); MultiTopicsConsumerImpl impl = new MultiTopicsConsumerImpl(clientMock, consumerConfData, @@ -237,7 +240,8 @@ public void testDontCheckForPartitionsUpdatesOnNonPartitionedTopics() throws Exc // Simulate non partitioned topics PartitionedTopicMetadata metadata = new PartitionedTopicMetadata(0); - when(clientMock.getPartitionedTopicMetadata(any())).thenReturn(CompletableFuture.completedFuture(metadata)); + when(clientMock.getPartitionedTopicMetadata(any(), anyBoolean())) + .thenReturn(CompletableFuture.completedFuture(metadata)); CompletableFuture> completeFuture = new CompletableFuture<>(); MultiTopicsConsumerImpl impl = new MultiTopicsConsumerImpl<>( @@ -248,7 +252,7 @@ public void testDontCheckForPartitionsUpdatesOnNonPartitionedTopics() throws Exc // getPartitionedTopicMetadata should have been called only the first time, for each of the 3 topics, // but not anymore since the topics are not partitioned. - verify(clientMock, times(3)).getPartitionedTopicMetadata(any()); + verify(clientMock, times(3)).getPartitionedTopicMetadata(any(), anyBoolean()); } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java index e0b25db891247..e13c060a052ec 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.client.impl; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.nullable; @@ -106,7 +107,7 @@ public void testConsumerIsClosed() throws Exception { nullable(String.class))) .thenReturn(CompletableFuture.completedFuture( new GetTopicsResult(Collections.emptyList(), null, false, true))); - when(lookup.getPartitionedTopicMetadata(any(TopicName.class))) + when(lookup.getPartitionedTopicMetadata(any(TopicName.class), anyBoolean())) .thenReturn(CompletableFuture.completedFuture(new PartitionedTopicMetadata())); when(lookup.getBroker(any())) .thenReturn(CompletableFuture.completedFuture( 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 faa5fbcd30130..01a1bd69e07b8 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 @@ -188,6 +188,7 @@ private static void setFeatureFlags(FeatureFlags flags) { flags.setSupportsAuthRefresh(true); flags.setSupportsBrokerEntryMetadata(true); flags.setSupportsPartialProducer(true); + flags.setSupportsGetPartitionedMetadataWithoutAutoCreation(true); } public static ByteBuf newConnect(String authMethodName, String authData, int protocolVersion, String libVersion, @@ -880,11 +881,13 @@ public static ByteBuf newPartitionMetadataResponse(ServerError error, String err return serializeWithSize(newPartitionMetadataResponseCommand(error, errorMsg, requestId)); } - public static ByteBuf newPartitionMetadataRequest(String topic, long requestId) { + public static ByteBuf newPartitionMetadataRequest(String topic, long requestId, + boolean metadataAutoCreationEnabled) { BaseCommand cmd = localCmd(Type.PARTITIONED_METADATA); cmd.setPartitionMetadata() .setTopic(topic) - .setRequestId(requestId); + .setRequestId(requestId) + .setMetadataAutoCreationEnabled(metadataAutoCreationEnabled); return serializeWithSize(cmd); } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/FutureUtil.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/FutureUtil.java index f6fcb12f35939..0628d494af3af 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/FutureUtil.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/FutureUtil.java @@ -199,9 +199,9 @@ public static CompletableFuture failedFuture(Throwable t) { public static Throwable unwrapCompletionException(Throwable ex) { if (ex instanceof CompletionException) { - return ex.getCause(); + return unwrapCompletionException(ex.getCause()); } else if (ex instanceof ExecutionException) { - return ex.getCause(); + return unwrapCompletionException(ex.getCause()); } else { return ex; } diff --git a/pulsar-common/src/main/proto/PulsarApi.proto b/pulsar-common/src/main/proto/PulsarApi.proto index afe193eeb7e9d..f56df6ae9d103 100644 --- a/pulsar-common/src/main/proto/PulsarApi.proto +++ b/pulsar-common/src/main/proto/PulsarApi.proto @@ -300,6 +300,7 @@ message FeatureFlags { optional bool supports_broker_entry_metadata = 2 [default = false]; optional bool supports_partial_producer = 3 [default = false]; optional bool supports_topic_watchers = 4 [default = false]; + optional bool supports_get_partitioned_metadata_without_auto_creation = 5 [default = false]; } message CommandConnected { @@ -413,6 +414,7 @@ message CommandPartitionedTopicMetadata { // to the proxy. optional string original_auth_data = 4; optional string original_auth_method = 5; + optional bool metadata_auto_creation_enabled = 6 [default = true]; } message CommandPartitionedTopicMetadataResponse { From 87013b4bd43166f34c06c44eb3600763556738c6 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 6 Jun 2024 16:09:38 +0800 Subject: [PATCH 559/699] [improve] [client] PIP-344 support feature flag supportsGetPartitionedMetadataWithoutAutoCreation (#22773) (cherry picked from commit 6236116754472c61b2166da6d4797fc63c83f364) (cherry picked from commit 03c0975345aed55e344ca422a9f3e7ce1491ec99) --- .../pulsar/client/impl/ClientCnxTest.java | 44 +++++++++++++++++++ .../client/impl/BinaryProtoLookupService.java | 6 +++ .../apache/pulsar/client/impl/ClientCnx.java | 5 +++ .../pulsar/common/protocol/Commands.java | 1 + 4 files changed, 56 insertions(+) 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 d2f610ae53f65..1a9b4bbcb0d21 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 @@ -20,13 +20,17 @@ import com.google.common.collect.Sets; import io.netty.channel.ChannelHandlerContext; +import java.lang.reflect.Field; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import org.apache.pulsar.PulsarVersion; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.api.Consumer; 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.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.TenantInfoImpl; @@ -123,4 +127,44 @@ public void testClientVersion() throws Exception { producer.close(); consumer.close(); } + + @Test + public void testSupportsGetPartitionedMetadataWithoutAutoCreation() throws Exception { + final String topic = BrokerTestUtil.newUniqueName( "persistent://" + NAMESPACE + "/tp"); + admin.topics().createNonPartitionedTopic(topic); + PulsarClientImpl clientWitBinaryLookup = (PulsarClientImpl) PulsarClient.builder() + .maxNumberOfRejectedRequestPerConnection(1) + .connectionMaxIdleSeconds(Integer.MAX_VALUE) + .serviceUrl(pulsar.getBrokerServiceUrl()) + .build(); + ProducerImpl producer = (ProducerImpl) clientWitBinaryLookup.newProducer().topic(topic).create(); + + // Verify: the variable "isSupportsGetPartitionedMetadataWithoutAutoCreation" responded from the broker is true. + Awaitility.await().untilAsserted(() -> { + ClientCnx clientCnx = producer.getClientCnx(); + Assert.assertNotNull(clientCnx); + Assert.assertTrue(clientCnx.isSupportsGetPartitionedMetadataWithoutAutoCreation()); + }); + Assert.assertEquals( + clientWitBinaryLookup.getPartitionsForTopic(topic, true).get().size(), 1); + + // Inject a "false" value for the variable "isSupportsGetPartitionedMetadataWithoutAutoCreation". + // Verify: client will get a not support error. + Field field = ClientCnx.class.getDeclaredField("supportsGetPartitionedMetadataWithoutAutoCreation"); + field.setAccessible(true); + for (CompletableFuture clientCnxFuture : clientWitBinaryLookup.getCnxPool().getConnections()) { + field.set(clientCnxFuture.get(), false); + } + try { + clientWitBinaryLookup.getPartitionsForTopic(topic, false).join(); + Assert.fail("Expected an error that the broker version is too old."); + } catch (Exception ex) { + Assert.assertTrue(ex.getMessage().contains("without auto-creation is not supported from the broker")); + } + + // cleanup. + producer.close(); + clientWitBinaryLookup.close(); + admin.topics().delete(topic, false); + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index d6f4dd2dcac14..980e8a0c78627 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -229,6 +229,12 @@ private CompletableFuture getPartitionedTopicMetadata( CompletableFuture partitionFuture = new CompletableFuture<>(); client.getCnxPool().getConnection(socketAddress).thenAccept(clientCnx -> { + if (!metadataAutoCreationEnabled && !clientCnx.isSupportsGetPartitionedMetadataWithoutAutoCreation()) { + partitionFuture.completeExceptionally(new PulsarClientException.NotSupportedException("The feature of" + + " getting partitions without auto-creation is not supported from the broker," + + " please upgrade the broker to the latest version.")); + return; + } long requestId = client.newRequestId(); ByteBuf request = Commands.newPartitionMetadataRequest(topicName.toString(), requestId, metadataAutoCreationEnabled); 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 8694dad7a2d84..b8caa0f438361 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 @@ -186,6 +186,8 @@ public class ClientCnx extends PulsarHandler { protected AuthenticationDataProvider authenticationDataProvider; private TransactionBufferHandler transactionBufferHandler; private boolean supportsTopicWatchers; + @Getter + private boolean supportsGetPartitionedMetadataWithoutAutoCreation; /** Idle stat. **/ @Getter @@ -382,6 +384,9 @@ protected void handleConnected(CommandConnected connected) { supportsTopicWatchers = connected.hasFeatureFlags() && connected.getFeatureFlags().isSupportsTopicWatchers(); + supportsGetPartitionedMetadataWithoutAutoCreation = + connected.hasFeatureFlags() + && connected.getFeatureFlags().isSupportsGetPartitionedMetadataWithoutAutoCreation(); // set remote protocol version to the correct version before we complete the connection future setRemoteEndpointProtocolVersion(connected.getProtocolVersion()); 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 01a1bd69e07b8..c352da0c871ed 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 @@ -299,6 +299,7 @@ public static BaseCommand newConnectedCommand(int clientProtocolVersion, int max connected.setProtocolVersion(versionToAdvertise); connected.setFeatureFlags().setSupportsTopicWatchers(supportsTopicWatchers); + connected.setFeatureFlags().setSupportsGetPartitionedMetadataWithoutAutoCreation(true); return cmd; } From b9c9930aa5722a6f961a11aa18fa40a7e04df4c0 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 17 Jun 2024 23:39:08 +0800 Subject: [PATCH 560/699] [fix] [broker] response not-found error if topic does not exist when calling getPartitionedTopicMetadata (#22838) (cherry picked from commit 9aed73653e1f706e3517072cce4a352d0838f8d7) (cherry picked from commit 1d2959bab7b77f82579c269f285b1d08ed26ac6c) --- .../admin/impl/PersistentTopicsBase.java | 21 +- .../broker/admin/v2/NonPersistentTopics.java | 16 +- .../pulsar/broker/lookup/TopicLookupBase.java | 22 +- .../broker/namespace/NamespaceService.java | 101 ++- .../broker/namespace/TopicExistsInfo.java | 82 +++ .../pulsar/broker/service/BrokerService.java | 117 ++-- .../pulsar/broker/service/ServerCnx.java | 81 +-- .../GetPartitionMetadataMultiBrokerTest.java | 222 +++++++ .../admin/GetPartitionMetadataTest.java | 608 ++++++++++-------- .../pulsar/broker/admin/TopicsTest.java | 13 +- .../lookup/http/HttpTopicLookupv2Test.java | 46 +- .../namespace/NamespaceServiceTest.java | 7 +- .../pulsar/broker/service/TopicGCTest.java | 2 + .../client/impl/ConsumerBuilderImpl.java | 37 +- 14 files changed, 928 insertions(+), 447 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/TopicExistsInfo.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataMultiBrokerTest.java 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 d5721d249b23e..2a6b89b413371 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 @@ -570,13 +570,13 @@ protected CompletableFuture internalGetPartitionedMeta // is a non-partitioned topic so we shouldn't check if the topic exists. return pulsar().getBrokerService().isAllowAutoTopicCreationAsync(topicName) .thenCompose(brokerAllowAutoTopicCreation -> { - if (checkAllowAutoCreation) { + if (checkAllowAutoCreation && brokerAllowAutoTopicCreation) { // Whether it exists or not, auto create a non-partitioned topic by client. return CompletableFuture.completedFuture(metadata); } else { // If it does not exist, response a Not Found error. // Otherwise, response a non-partitioned metadata. - return internalCheckTopicExists(topicName).thenApply(__ -> metadata); + return internalCheckNonPartitionedTopicExists(topicName).thenApply(__ -> metadata); } }); } @@ -724,6 +724,17 @@ public void updatePropertiesFailed(ManagedLedgerException exception, Object ctx) 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())); + } + }); + } + + protected CompletableFuture internalCheckNonPartitionedTopicExists(TopicName topicName) { + return pulsar().getNamespaceService().checkNonPartitionedTopicExists(topicName) .thenAccept(exist -> { if (!exist) { throw new RestException(Status.NOT_FOUND, getTopicNotFoundErrorMessage(topicName.toString())); @@ -5541,8 +5552,10 @@ protected CompletableFuture validateShadowTopics(List shadowTopics "Only persistent topic can be set as shadow topic")); } futures.add(pulsar().getNamespaceService().checkTopicExists(shadowTopicName) - .thenAccept(isExists -> { - if (!isExists) { + .thenAccept(info -> { + boolean exists = info.isExists(); + info.recycle(); + if (!exists) { throw new RestException(Status.PRECONDITION_FAILED, "Shadow topic [" + shadowTopic + "] not exists."); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java index cae7c651ce791..945ad69477c70 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/v2/NonPersistentTopics.java @@ -98,8 +98,20 @@ public void getPartitionedMetadata( @QueryParam("authoritative") @DefaultValue("false") boolean authoritative, @ApiParam(value = "Is check configuration required to automatically create topic") @QueryParam("checkAllowAutoCreation") @DefaultValue("false") boolean checkAllowAutoCreation) { - super.getPartitionedMetadata(asyncResponse, tenant, namespace, encodedTopic, authoritative, - checkAllowAutoCreation); + validateTopicName(tenant, namespace, encodedTopic); + validateTopicOwnershipAsync(topicName, authoritative).whenComplete((__, ex) -> { + if (ex != null) { + Throwable actEx = FutureUtil.unwrapCompletionException(ex); + if (isNot307And404Exception(actEx)) { + log.error("[{}] Failed to get internal stats for topic {}", clientAppId(), topicName, ex); + } + resumeAsyncResponseExceptionally(asyncResponse, actEx); + } else { + // "super.getPartitionedMetadata" will handle error itself. + super.getPartitionedMetadata(asyncResponse, tenant, namespace, encodedTopic, authoritative, + checkAllowAutoCreation); + } + }); } @GET 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 7b2c777414884..9a05c3d992aaf 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 @@ -67,16 +67,22 @@ protected CompletableFuture internalLookupTopicAsync(final TopicName .thenCompose(__ -> validateGlobalNamespaceOwnershipAsync(topicName.getNamespaceObject())) .thenCompose(__ -> validateTopicOperationAsync(topicName, TopicOperation.LOOKUP, null)) .thenCompose(__ -> { + // Case-1: Non-persistent topic. // Currently, it's hard to check the non-persistent-non-partitioned topic, because it only exists // in the broker, it doesn't have metadata. If the topic is non-persistent and non-partitioned, - // we'll return the true flag. - CompletableFuture existFuture = (!topicName.isPersistent() && !topicName.isPartitioned()) - ? CompletableFuture.completedFuture(true) - : pulsar().getNamespaceService().checkTopicExists(topicName) - .thenCompose(exists -> exists ? CompletableFuture.completedFuture(true) - : pulsar().getBrokerService().isAllowAutoTopicCreationAsync(topicName)); - - return existFuture; + // we'll return the true flag. So either it is a partitioned topic or not, the result will be true. + if (!topicName.isPersistent()) { + return CompletableFuture.completedFuture(true); + } + // Case-2: Persistent topic. + return pulsar().getNamespaceService().checkTopicExists(topicName).thenCompose(info -> { + boolean exists = info.isExists(); + info.recycle(); + if (exists) { + return CompletableFuture.completedFuture(true); + } + return pulsar().getBrokerService().isAllowAutoTopicCreationAsync(topicName); + }); }) .thenCompose(exist -> { if (!exist) { 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 626a187f46388..c62e9c52a768e 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 @@ -47,6 +47,7 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; import javax.annotation.Nullable; +import lombok.extern.slf4j.Slf4j; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.collections4.ListUtils; import org.apache.commons.lang3.StringUtils; @@ -68,6 +69,7 @@ import org.apache.pulsar.broker.stats.prometheus.metrics.Summary; import org.apache.pulsar.broker.web.PulsarWebResource; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; @@ -117,6 +119,7 @@ * * @see org.apache.pulsar.broker.PulsarService */ +@Slf4j public class NamespaceService implements AutoCloseable { private static final Logger LOG = LoggerFactory.getLogger(NamespaceService.class); @@ -1312,40 +1315,86 @@ public CompletableFuture> getOwnedTopicListForNamespaceBundle(Names }); } - public CompletableFuture checkTopicExists(TopicName topic) { - CompletableFuture future; - // If the topic is persistent and the name includes `-partition-`, find the topic from the managed/ledger. - if (topic.isPersistent() && topic.isPartitioned()) { - future = pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); + /*** + * Check topic exists( partitioned or non-partitioned ). + */ + public CompletableFuture checkTopicExists(TopicName topic) { + return pulsar.getBrokerService() + .fetchPartitionedTopicMetadataAsync(TopicName.get(topic.toString())) + .thenCompose(metadata -> { + if (metadata.partitions > 0) { + return CompletableFuture.completedFuture( + TopicExistsInfo.newPartitionedTopicExists(metadata.partitions)); + } + return checkNonPartitionedTopicExists(topic) + .thenApply(b -> b ? TopicExistsInfo.newNonPartitionedTopicExists() + : TopicExistsInfo.newTopicNotExists()); + }); + } + + /*** + * Check non-partitioned topic exists. + */ + public CompletableFuture checkNonPartitionedTopicExists(TopicName topic) { + if (topic.isPersistent()) { + return pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); } else { - future = CompletableFuture.completedFuture(false); + return checkNonPersistentNonPartitionedTopicExists(topic.toString()); } + } - return future.thenCompose(found -> { - if (found != null && found) { - return CompletableFuture.completedFuture(true); + /** + * Regarding non-persistent topic, we do not know whether it exists or not. Redirect the request to the ownership + * broker of this topic. HTTP API has implemented the mechanism that redirect to ownership broker, so just call + * HTTP API here. + */ + public CompletableFuture checkNonPersistentNonPartitionedTopicExists(String topic) { + TopicName topicName = TopicName.get(topic); + // "non-partitioned & non-persistent" topics only exist on the owner broker. + return checkTopicOwnership(TopicName.get(topic)).thenCompose(isOwned -> { + // The current broker is the owner. + if (isOwned) { + CompletableFuture> nonPersistentTopicFuture = pulsar.getBrokerService() + .getTopic(topic, false); + if (nonPersistentTopicFuture != null) { + return nonPersistentTopicFuture.thenApply(Optional::isPresent); + } else { + return CompletableFuture.completedFuture(false); + } } - return pulsar.getBrokerService() - .fetchPartitionedTopicMetadataAsync(TopicName.get(topic.getPartitionedTopicName())) - .thenCompose(metadata -> { - if (metadata.partitions > 0) { - return CompletableFuture.completedFuture(true); - } - - if (topic.isPersistent()) { - return pulsar.getPulsarResources().getTopicResources().persistentTopicExists(topic); - } else { - // The non-partitioned non-persistent topic only exist in the broker topics. - CompletableFuture> nonPersistentTopicFuture = - pulsar.getBrokerService().getTopics().get(topic.toString()); - if (nonPersistentTopicFuture == null) { + // Forward to the owner broker. + PulsarClientImpl pulsarClient; + try { + pulsarClient = (PulsarClientImpl) pulsar.getClient(); + } catch (Exception ex) { + // This error will never occur. + log.error("{} Failed to get partition metadata due to create internal admin client fails", topic, ex); + return FutureUtil.failedFuture(ex); + } + LookupOptions lookupOptions = LookupOptions.builder().readOnly(false).authoritative(true).build(); + return getBrokerServiceUrlAsync(TopicName.get(topic), lookupOptions) + .thenCompose(lookupResult -> { + if (!lookupResult.isPresent()) { + log.error("{} Failed to get partition metadata due can not find the owner broker", topic); + return FutureUtil.failedFuture(new ServiceUnitNotReadyException( + "No broker was available to own " + topicName)); + } + return pulsarClient.getLookup(lookupResult.get().getLookupData().getBrokerUrl()) + .getPartitionedTopicMetadata(topicName, false) + .thenApply(metadata -> true) + .exceptionallyCompose(ex -> { + Throwable actEx = FutureUtil.unwrapCompletionException(ex); + if (actEx instanceof PulsarClientException.NotFoundException + || actEx instanceof PulsarClientException.TopicDoesNotExistException + || actEx instanceof PulsarAdminException.NotFoundException) { return CompletableFuture.completedFuture(false); } else { - return nonPersistentTopicFuture.thenApply(Optional::isPresent); + log.error("{} Failed to get partition metadata due to redirecting fails", topic, ex); + return CompletableFuture.failedFuture(ex); } - } - }); + }); + }); }); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/TopicExistsInfo.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/TopicExistsInfo.java new file mode 100644 index 0000000000000..1c3f117719e8e --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/namespace/TopicExistsInfo.java @@ -0,0 +1,82 @@ +/* + * 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.namespace; + +import io.netty.util.Recycler; +import lombok.Getter; +import org.apache.pulsar.common.policies.data.TopicType; + +public class TopicExistsInfo { + + private static final Recycler RECYCLER = new Recycler<>() { + @Override + protected TopicExistsInfo newObject(Handle handle) { + return new TopicExistsInfo(handle); + } + }; + + private static TopicExistsInfo nonPartitionedExists = new TopicExistsInfo(true, 0); + + private static TopicExistsInfo notExists = new TopicExistsInfo(false, 0); + + public static TopicExistsInfo newPartitionedTopicExists(Integer partitions){ + TopicExistsInfo info = RECYCLER.get(); + info.exists = true; + info.partitions = partitions.intValue(); + return info; + } + + public static TopicExistsInfo newNonPartitionedTopicExists(){ + return nonPartitionedExists; + } + + public static TopicExistsInfo newTopicNotExists(){ + return notExists; + } + + private final Recycler.Handle handle; + + @Getter + private int partitions; + @Getter + private boolean exists; + + private TopicExistsInfo(Recycler.Handle handle) { + this.handle = handle; + } + + private TopicExistsInfo(boolean exists, int partitions) { + this.handle = null; + this.partitions = partitions; + this.exists = exists; + } + + public void recycle() { + if (this == notExists || this == nonPartitionedExists || this.handle == null) { + return; + } + this.exists = false; + this.partitions = 0; + this.handle.recycle(this); + } + + public TopicType getTopicType() { + return this.partitions > 0 ? TopicType.PARTITIONED : TopicType.NON_PARTITIONED; + } +} 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 9da21c35c31aa..7ce9898bb38a5 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 @@ -3212,65 +3212,66 @@ public CompletableFuture fetchPartitionedTopicMetadata if (pulsar.getNamespaceService() == null) { return FutureUtil.failedFuture(new NamingException("namespace service is not ready")); } - return pulsar.getPulsarResources().getNamespaceResources().getPoliciesAsync(topicName.getNamespaceObject()) - .thenCompose(policies -> pulsar.getNamespaceService().checkTopicExists(topicName) - .thenCompose(topicExists -> fetchPartitionedTopicMetadataAsync(topicName) - .thenCompose(metadata -> { - CompletableFuture future = new CompletableFuture<>(); - - // There are a couple of potentially blocking calls, which we cannot make from the - // MetadataStore callback thread. - pulsar.getExecutor().execute(() -> { - // If topic is already exist, creating partitioned topic is not allowed. - - if (metadata.partitions == 0 - && !topicExists - && !topicName.isPartitioned() - && pulsar.getBrokerService() - .isDefaultTopicTypePartitioned(topicName, policies)) { - isAllowAutoTopicCreationAsync(topicName, policies).thenAccept(allowed -> { - if (allowed) { - pulsar.getBrokerService() - .createDefaultPartitionedTopicAsync(topicName, policies) - .thenAccept(md -> future.complete(md)) - .exceptionally(ex -> { - if (ex.getCause() - instanceof MetadataStoreException - .AlreadyExistsException) { - log.info("[{}] The partitioned topic is already" - + " created, try to refresh the cache and read" - + " again.", topicName); - // The partitioned topic might be created concurrently - fetchPartitionedTopicMetadataAsync(topicName, true) - .whenComplete((metadata2, ex2) -> { - if (ex2 == null) { - future.complete(metadata2); - } else { - future.completeExceptionally(ex2); - } - }); - } else { - log.error("[{}] operation of creating partitioned" - + " topic metadata failed", - topicName, ex); - future.completeExceptionally(ex); - } - return null; - }); - } else { - future.complete(metadata); - } - }).exceptionally(ex -> { - future.completeExceptionally(ex); - return null; - }); - } else { - future.complete(metadata); - } - }); + return pulsar.getNamespaceService().checkTopicExists(topicName).thenComposeAsync(topicExistsInfo -> { + final boolean topicExists = topicExistsInfo.isExists(); + final TopicType topicType = topicExistsInfo.getTopicType(); + final Integer partitions = topicExistsInfo.getPartitions(); + topicExistsInfo.recycle(); + + // Topic exists. + if (topicExists) { + if (topicType.equals(TopicType.PARTITIONED)) { + return CompletableFuture.completedFuture(new PartitionedTopicMetadata(partitions)); + } + return CompletableFuture.completedFuture(new PartitionedTopicMetadata(0)); + } - return future; - }))); + // Try created if allowed to create a partitioned topic automatically. + return pulsar.getPulsarResources().getNamespaceResources().getPoliciesAsync(topicName.getNamespaceObject()) + .thenComposeAsync(policies -> { + return isAllowAutoTopicCreationAsync(topicName, policies).thenComposeAsync(allowed -> { + // Not Allow auto-creation. + if (!allowed) { + // Do not change the original behavior, or default return a non-partitioned topic. + return CompletableFuture.completedFuture(new PartitionedTopicMetadata(0)); + } + + // Allow auto create non-partitioned topic. + boolean autoCreatePartitionedTopic = pulsar.getBrokerService() + .isDefaultTopicTypePartitioned(topicName, policies); + if (!autoCreatePartitionedTopic || topicName.isPartitioned()) { + return CompletableFuture.completedFuture(new PartitionedTopicMetadata(0)); + } + + // Create partitioned metadata. + return pulsar.getBrokerService().createDefaultPartitionedTopicAsync(topicName, policies) + .exceptionallyCompose(ex -> { + // The partitioned topic might be created concurrently. + if (ex.getCause() instanceof MetadataStoreException.AlreadyExistsException) { + log.info("[{}] The partitioned topic is already created, try to refresh the cache" + + " and read again.", topicName); + CompletableFuture recheckFuture = + fetchPartitionedTopicMetadataAsync(topicName, true); + recheckFuture.exceptionally(ex2 -> { + // Just for printing a log if error occurs. + log.error("[{}] Fetch partitioned topic metadata failed", topicName, ex); + return null; + }); + return recheckFuture; + } else { + log.error("[{}] operation of creating partitioned topic metadata failed", + topicName, ex); + return CompletableFuture.failedFuture(ex); + } + }); + }, pulsar.getExecutor()).exceptionallyCompose(ex -> { + log.error("[{}] operation of get partitioned metadata failed due to calling" + + " isAllowAutoTopicCreationAsync failed", + topicName, ex); + return CompletableFuture.failedFuture(ex); + }); + }, pulsar.getExecutor()); + }, pulsar.getExecutor()); } @SuppressWarnings("deprecation") 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 52135163a6ab3..2514be55fffa9 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 @@ -81,8 +81,7 @@ import org.apache.pulsar.broker.authentication.AuthenticationState; import org.apache.pulsar.broker.intercept.BrokerInterceptor; import org.apache.pulsar.broker.limiter.ConnectionController; -import org.apache.pulsar.broker.resources.NamespaceResources; -import org.apache.pulsar.broker.resources.TopicResources; +import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException; import org.apache.pulsar.broker.service.BrokerServiceException.ServerMetadataException; import org.apache.pulsar.broker.service.BrokerServiceException.ServiceUnitNotReadyException; @@ -157,6 +156,7 @@ import org.apache.pulsar.common.policies.data.ClusterData.ClusterUrl; import org.apache.pulsar.common.policies.data.NamespaceOperation; import org.apache.pulsar.common.policies.data.TopicOperation; +import org.apache.pulsar.common.policies.data.TopicType; import org.apache.pulsar.common.policies.data.stats.ConsumerStatsImpl; import org.apache.pulsar.common.protocol.ByteBufPair; import org.apache.pulsar.common.protocol.CommandUtils; @@ -582,58 +582,33 @@ protected void handlePartitionMetadataRequest(CommandPartitionedTopicMetadata pa if (isAuthorized) { // Get if exists, respond not found error if not exists. getBrokerService().isAllowAutoTopicCreationAsync(topicName).thenAccept(brokerAllowAutoCreate -> { - boolean autoCreateIfNotExist = partitionMetadata.isMetadataAutoCreationEnabled(); + boolean autoCreateIfNotExist = partitionMetadata.isMetadataAutoCreationEnabled() + && brokerAllowAutoCreate; if (!autoCreateIfNotExist) { - final NamespaceResources namespaceResources = getBrokerService().pulsar() - .getPulsarResources().getNamespaceResources(); - final TopicResources topicResources = getBrokerService().pulsar().getPulsarResources() - .getTopicResources(); - namespaceResources.getPartitionedTopicResources() - .getPartitionedTopicMetadataAsync(topicName, false) - .handle((metadata, getMetadataEx) -> { - if (getMetadataEx != null) { - log.error("{} {} Failed to get partition metadata", topicName, - ServerCnx.this.toString(), getMetadataEx); - writeAndFlush( - Commands.newPartitionMetadataResponse(ServerError.MetadataError, - "Failed to get partition metadata", - requestId)); - } else if (metadata.isPresent()) { - commandSender.sendPartitionMetadataResponse(metadata.get().partitions, - requestId); - } else if (topicName.isPersistent()) { - topicResources.persistentTopicExists(topicName).thenAccept(exists -> { - if (exists) { - commandSender.sendPartitionMetadataResponse(0, requestId); - return; - } - writeAndFlush(Commands.newPartitionMetadataResponse( - ServerError.TopicNotFound, "", requestId)); - }).exceptionally(ex -> { - log.error("{} {} Failed to get partition metadata", topicName, - ServerCnx.this.toString(), ex); - writeAndFlush( - Commands.newPartitionMetadataResponse(ServerError.MetadataError, - "Failed to check partition metadata", - requestId)); - return null; - }); - } else { - // Regarding non-persistent topic, we do not know whether it exists or not. - // Just return a non-partitioned metadata if partitioned metadata does not - // exist. - // Broker will respond a not found error when doing subscribing or producing if - // broker not allow to auto create topics. - commandSender.sendPartitionMetadataResponse(0, requestId); - } - return null; - }).whenComplete((ignore, ignoreEx) -> { - lookupSemaphore.release(); - if (ignoreEx != null) { - log.error("{} {} Failed to handle partition metadata request", topicName, - ServerCnx.this.toString(), ignoreEx); - } - }); + NamespaceService namespaceService = getBrokerService().getPulsar().getNamespaceService(); + namespaceService.checkTopicExists(topicName).thenAccept(topicExistsInfo -> { + lookupSemaphore.release(); + if (!topicExistsInfo.isExists()) { + writeAndFlush(Commands.newPartitionMetadataResponse( + ServerError.TopicNotFound, "", requestId)); + } else if (topicExistsInfo.getTopicType().equals(TopicType.PARTITIONED)) { + commandSender.sendPartitionMetadataResponse(topicExistsInfo.getPartitions(), + requestId); + } else { + commandSender.sendPartitionMetadataResponse(0, requestId); + } + // release resources. + topicExistsInfo.recycle(); + }).exceptionally(ex -> { + lookupSemaphore.release(); + log.error("{} {} Failed to get partition metadata", topicName, + ServerCnx.this.toString(), ex); + writeAndFlush( + Commands.newPartitionMetadataResponse(ServerError.MetadataError, + "Failed to get partition metadata", + requestId)); + return null; + }); } else { // Get if exists, create a new one if not exists. unsafeGetPartitionedTopicMetadataAsync(getBrokerService().pulsar(), topicName) 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 new file mode 100644 index 0000000000000..28cf91ee165e2 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/GetPartitionMetadataMultiBrokerTest.java @@ -0,0 +1,222 @@ +/* + * 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.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; +import java.net.URL; +import java.util.List; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.common.naming.TopicDomain; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.TopicType; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Test(groups = "broker-admin") +@Slf4j +public class GetPartitionMetadataMultiBrokerTest extends GetPartitionMetadataTest { + + private PulsarService pulsar2; + private URL url2; + private PulsarAdmin admin2; + private PulsarClientImpl clientWithHttpLookup2; + private PulsarClientImpl clientWitBinaryLookup2; + + @BeforeClass(alwaysRun = true) + protected void setup() throws Exception { + super.setup(); + } + + @Override + @AfterClass(alwaysRun = true) + protected void cleanup() throws Exception { + super.cleanup(); + } + + @Override + protected void cleanupBrokers() throws Exception { + // Cleanup broker2. + if (clientWithHttpLookup2 != null) { + clientWithHttpLookup2.close(); + clientWithHttpLookup2 = null; + } + if (clientWitBinaryLookup2 != null) { + clientWitBinaryLookup2.close(); + clientWitBinaryLookup2 = null; + } + if (admin2 != null) { + admin2.close(); + admin2 = null; + } + if (pulsar2 != null) { + pulsar2.close(); + pulsar2 = null; + } + + // Super cleanup. + super.cleanupBrokers(); + } + + @Override + protected void setupBrokers() throws Exception { + super.setupBrokers(); + doInitConf(); + pulsar2 = new PulsarService(conf); + pulsar2.start(); + url2 = new URL(pulsar2.getWebServiceAddress()); + admin2 = PulsarAdmin.builder().serviceHttpUrl(url2.toString()).build(); + clientWithHttpLookup2 = + (PulsarClientImpl) PulsarClient.builder().serviceUrl(pulsar2.getWebServiceAddress()).build(); + clientWitBinaryLookup2 = + (PulsarClientImpl) PulsarClient.builder().serviceUrl(pulsar2.getBrokerServiceUrl()).build(); + } + + @Override + protected PulsarClientImpl[] getClientsToTest() { + return new PulsarClientImpl[] {clientWithHttpLookup1, clientWitBinaryLookup1, + clientWithHttpLookup2, clientWitBinaryLookup2}; + } + + protected PulsarClientImpl[] getClientsToTest(boolean isUsingHttpLookup) { + if (isUsingHttpLookup) { + return new PulsarClientImpl[]{clientWithHttpLookup1, clientWithHttpLookup2}; + } else { + return new PulsarClientImpl[]{clientWitBinaryLookup1, clientWitBinaryLookup2}; + } + } + + @Override + protected int getLookupRequestPermits() { + return pulsar1.getBrokerService().getLookupRequestSemaphore().availablePermits() + + pulsar2.getBrokerService().getLookupRequestSemaphore().availablePermits(); + } + + protected void verifyPartitionsNeverCreated(String topicNameStr) throws Exception { + TopicName topicName = TopicName.get(topicNameStr); + try { + List topicList = admin1.topics().getList("public/default"); + for (int i = 0; i < 3; i++) { + assertFalse(topicList.contains(topicName.getPartition(i))); + } + } catch (Exception ex) { + // If the namespace bundle has not been loaded yet, it means no non-persistent topic was created. So + // this behavior is also correct. + // This error is not expected, a seperated PR is needed to fix this issue. + assertTrue(ex.getMessage().contains("Failed to find ownership for")); + } + } + + protected void verifyNonPartitionedTopicNeverCreated(String topicNameStr) throws Exception { + TopicName topicName = TopicName.get(topicNameStr); + try { + List topicList = admin1.topics().getList("public/default"); + assertFalse(topicList.contains(topicName.getPartitionedTopicName())); + } catch (Exception ex) { + // If the namespace bundle has not been loaded yet, it means no non-persistent topic was created. So + // this behavior is also correct. + // This error is not expected, a seperated PR is needed to fix this issue. + assertTrue(ex.getMessage().contains("Failed to find ownership for")); + } + } + + protected void modifyTopicAutoCreation(boolean allowAutoTopicCreation, + TopicType allowAutoTopicCreationType, + int defaultNumPartitions) throws Exception { + doModifyTopicAutoCreation(admin1, pulsar1, allowAutoTopicCreation, allowAutoTopicCreationType, + defaultNumPartitions); + doModifyTopicAutoCreation(admin2, pulsar2, allowAutoTopicCreation, allowAutoTopicCreationType, + defaultNumPartitions); + } + + /** + * {@inheritDoc} + */ + @Test(dataProvider = "topicDomains") + public void testAutoCreatingMetadataWhenCallingOldAPI(TopicDomain topicDomain) throws Exception { + super.testAutoCreatingMetadataWhenCallingOldAPI(topicDomain); + } + + /** + * {@inheritDoc} + */ + @Test(dataProvider = "autoCreationParamsAll", enabled = false) + public void testGetMetadataIfNonPartitionedTopicExists(boolean configAllowAutoTopicCreation, + boolean paramMetadataAutoCreationEnabled, + boolean isUsingHttpLookup, + TopicDomain topicDomain) throws Exception { + super.testGetMetadataIfNonPartitionedTopicExists(configAllowAutoTopicCreation, paramMetadataAutoCreationEnabled, + isUsingHttpLookup, topicDomain); + } + + /** + * {@inheritDoc} + */ + @Test(dataProvider = "autoCreationParamsAll") + public void testGetMetadataIfPartitionedTopicExists(boolean configAllowAutoTopicCreation, + boolean paramMetadataAutoCreationEnabled, + boolean isUsingHttpLookup, + TopicDomain topicDomain) throws Exception { + super.testGetMetadataIfNonPartitionedTopicExists(configAllowAutoTopicCreation, paramMetadataAutoCreationEnabled, + isUsingHttpLookup, topicDomain); + } + + /** + * {@inheritDoc} + */ + @Test(dataProvider = "clients") + public void testAutoCreatePartitionedTopic(boolean isUsingHttpLookup, TopicDomain topicDomain) throws Exception { + super.testAutoCreatePartitionedTopic(isUsingHttpLookup, topicDomain); + } + + /** + * {@inheritDoc} + */ + @Test(dataProvider = "clients") + public void testAutoCreateNonPartitionedTopic(boolean isUsingHttpLookup, TopicDomain topicDomain) throws Exception { + super.testAutoCreateNonPartitionedTopic(isUsingHttpLookup, topicDomain); + } + + /** + * {@inheritDoc} + */ + @Test(dataProvider = "autoCreationParamsNotAllow") + public void testGetMetadataIfNotAllowedCreate(boolean configAllowAutoTopicCreation, + boolean paramMetadataAutoCreationEnabled, + boolean isUsingHttpLookup) throws Exception { + super.testGetMetadataIfNotAllowedCreate(configAllowAutoTopicCreation, paramMetadataAutoCreationEnabled, + isUsingHttpLookup); + } + + /** + * {@inheritDoc} + */ + @Test(dataProvider = "autoCreationParamsNotAllow") + public void testGetMetadataIfNotAllowedCreateOfNonPersistentTopic(boolean configAllowAutoTopicCreation, + boolean paramMetadataAutoCreationEnabled, + boolean isUsingHttpLookup) throws Exception { + super.testGetMetadataIfNotAllowedCreateOfNonPersistentTopic(configAllowAutoTopicCreation, + paramMetadataAutoCreationEnabled, isUsingHttpLookup); + } +} 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 51f643d2b7823..bf99b172829a7 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 @@ -22,70 +22,150 @@ import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import com.google.common.collect.Sets; +import java.net.URL; +import java.util.Collections; import java.util.List; import java.util.Optional; -import java.util.concurrent.Semaphore; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.client.api.ProducerConsumerBase; +import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.impl.LookupService; import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.ClusterDataImpl; +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.zookeeper.LocalBookkeeperEnsemble; import org.awaitility.Awaitility; -import org.testng.Assert; -import org.testng.annotations.AfterMethod; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Test(groups = "broker-admin") @Slf4j -public class GetPartitionMetadataTest extends ProducerConsumerBase { +public class GetPartitionMetadataTest { - private static final String DEFAULT_NS = "public/default"; + protected static final String DEFAULT_NS = "public/default"; - private PulsarClientImpl clientWithHttpLookup; - private PulsarClientImpl clientWitBinaryLookup; + protected String clusterName = "c1"; - @Override + protected LocalBookkeeperEnsemble bkEnsemble; + + protected ServiceConfiguration conf = new ServiceConfiguration(); + + protected PulsarService pulsar1; + protected URL url1; + protected PulsarAdmin admin1; + protected PulsarClientImpl clientWithHttpLookup1; + protected PulsarClientImpl clientWitBinaryLookup1; + + @BeforeClass(alwaysRun = true) protected void setup() throws Exception { - super.internalSetup(); - super.producerBaseSetup(); - clientWithHttpLookup = - (PulsarClientImpl) PulsarClient.builder().serviceUrl(pulsar.getWebServiceAddress()).build(); - clientWitBinaryLookup = - (PulsarClientImpl) PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); + bkEnsemble = new LocalBookkeeperEnsemble(3, 0, () -> 0); + bkEnsemble.start(); + // Start broker. + setupBrokers(); + // Create default NS. + admin1.clusters().createCluster(clusterName, new ClusterDataImpl()); + admin1.tenants().createTenant(NamespaceName.get(DEFAULT_NS).getTenant(), + new TenantInfoImpl(Collections.emptySet(), Sets.newHashSet(clusterName))); + admin1.namespaces().createNamespace(DEFAULT_NS); } - @Override - @AfterMethod(alwaysRun = true) + @AfterClass(alwaysRun = true) protected void cleanup() throws Exception { - super.internalCleanup(); - if (clientWithHttpLookup != null) { - clientWithHttpLookup.close(); + cleanupBrokers(); + if (bkEnsemble != null) { + bkEnsemble.stop(); + bkEnsemble = null; + } + } + + protected void cleanupBrokers() throws Exception { + // Cleanup broker2. + if (clientWithHttpLookup1 != null) { + clientWithHttpLookup1.close(); + clientWithHttpLookup1 = null; + } + if (clientWitBinaryLookup1 != null) { + clientWitBinaryLookup1.close(); + clientWitBinaryLookup1 = null; } - if (clientWitBinaryLookup != null) { - clientWitBinaryLookup.close(); + if (admin1 != null) { + admin1.close(); + admin1 = null; } + if (pulsar1 != null) { + pulsar1.close(); + pulsar1 = null; + } + // Reset configs. + conf = new ServiceConfiguration(); + } + + protected void setupBrokers() throws Exception { + doInitConf(); + // Start broker. + pulsar1 = new PulsarService(conf); + pulsar1.start(); + url1 = new URL(pulsar1.getWebServiceAddress()); + admin1 = PulsarAdmin.builder().serviceHttpUrl(url1.toString()).build(); + clientWithHttpLookup1 = + (PulsarClientImpl) PulsarClient.builder().serviceUrl(pulsar1.getWebServiceAddress()).build(); + clientWitBinaryLookup1 = + (PulsarClientImpl) PulsarClient.builder().serviceUrl(pulsar1.getBrokerServiceUrl()).build(); } - @Override - protected void doInitConf() throws Exception { - super.doInitConf(); + protected void doInitConf() { + conf.setClusterName(clusterName); + conf.setAdvertisedAddress("localhost"); + conf.setBrokerServicePort(Optional.of(0)); + conf.setWebServicePort(Optional.of(0)); + conf.setMetadataStoreUrl("zk:127.0.0.1:" + bkEnsemble.getZookeeperPort()); + conf.setConfigurationMetadataStoreUrl("zk:127.0.0.1:" + bkEnsemble.getZookeeperPort() + "/foo"); + conf.setBrokerDeleteInactiveTopicsEnabled(false); + conf.setBrokerShutdownTimeoutMs(0L); + conf.setLoadBalancerSheddingEnabled(false); } - private LookupService getLookupService(boolean isUsingHttpLookup) { + protected PulsarClientImpl[] getClientsToTest() { + return new PulsarClientImpl[] {clientWithHttpLookup1, clientWitBinaryLookup1}; + } + + protected PulsarClientImpl[] getClientsToTest(boolean isUsingHttpLookup) { if (isUsingHttpLookup) { - return clientWithHttpLookup.getLookup(); + return new PulsarClientImpl[] {clientWithHttpLookup1}; } else { - return clientWitBinaryLookup.getLookup(); + return new PulsarClientImpl[] {clientWitBinaryLookup1}; } + + } + + protected int getLookupRequestPermits() { + return pulsar1.getBrokerService().getLookupRequestSemaphore().availablePermits(); + } + + protected void verifyPartitionsNeverCreated(String topicNameStr) throws Exception { + TopicName topicName = TopicName.get(topicNameStr); + List topicList = admin1.topics().getList("public/default"); + for (int i = 0; i < 3; i++) { + assertFalse(topicList.contains(topicName.getPartition(i))); + } + } + + protected void verifyNonPartitionedTopicNeverCreated(String topicNameStr) throws Exception { + TopicName topicName = TopicName.get(topicNameStr); + List topicList = admin1.topics().getList("public/default"); + assertFalse(topicList.contains(topicName.getPartitionedTopicName())); } @DataProvider(name = "topicDomains") @@ -96,43 +176,53 @@ public Object[][] topicDomains() { }; } - @Test(dataProvider = "topicDomains") - public void testAutoCreatingMetadataWhenCallingOldAPI(TopicDomain topicDomain) throws Exception { - conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); - conf.setDefaultNumPartitions(3); - conf.setAllowAutoTopicCreation(true); - setup(); - - Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); - int lookupPermitsBefore = semaphore.availablePermits(); - - // HTTP client. - final String tp1 = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); - clientWithHttpLookup.getPartitionsForTopic(tp1).join(); - Optional metadata1 = pulsar.getPulsarResources().getNamespaceResources() - .getPartitionedTopicResources() - .getPartitionedTopicMetadataAsync(TopicName.get(tp1), true).join(); - assertTrue(metadata1.isPresent()); - assertEquals(metadata1.get().partitions, 3); - - // Binary client. - final String tp2 = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); - clientWitBinaryLookup.getPartitionsForTopic(tp2).join(); - Optional metadata2 = pulsar.getPulsarResources().getNamespaceResources() - .getPartitionedTopicResources() - .getPartitionedTopicMetadataAsync(TopicName.get(tp2), true).join(); - assertTrue(metadata2.isPresent()); - assertEquals(metadata2.get().partitions, 3); - - // Verify: lookup semaphore has been releases. + protected static void doModifyTopicAutoCreation(PulsarAdmin admin1, PulsarService pulsar1, + boolean allowAutoTopicCreation, TopicType allowAutoTopicCreationType, + int defaultNumPartitions) throws Exception { + admin1.brokers().updateDynamicConfiguration( + "allowAutoTopicCreation", allowAutoTopicCreation + ""); + admin1.brokers().updateDynamicConfiguration( + "allowAutoTopicCreationType", allowAutoTopicCreationType + ""); + admin1.brokers().updateDynamicConfiguration( + "defaultNumPartitions", defaultNumPartitions + ""); Awaitility.await().untilAsserted(() -> { - int lookupPermitsAfter = semaphore.availablePermits(); - assertEquals(lookupPermitsAfter, lookupPermitsBefore); + assertEquals(pulsar1.getConfiguration().isAllowAutoTopicCreation(), allowAutoTopicCreation); + assertEquals(pulsar1.getConfiguration().getAllowAutoTopicCreationType(), allowAutoTopicCreationType); + assertEquals(pulsar1.getConfiguration().getDefaultNumPartitions(), defaultNumPartitions); }); + } - // Cleanup. - admin.topics().deletePartitionedTopic(tp1, false); - admin.topics().deletePartitionedTopic(tp2, false); + protected void modifyTopicAutoCreation(boolean allowAutoTopicCreation, + TopicType allowAutoTopicCreationType, + int defaultNumPartitions) throws Exception { + doModifyTopicAutoCreation(admin1, pulsar1, allowAutoTopicCreation, allowAutoTopicCreationType, + defaultNumPartitions); + } + + @Test(dataProvider = "topicDomains") + public void testAutoCreatingMetadataWhenCallingOldAPI(TopicDomain topicDomain) throws Exception { + modifyTopicAutoCreation(true, TopicType.PARTITIONED, 3); + + int lookupPermitsBefore = getLookupRequestPermits(); + + for (PulsarClientImpl client : getClientsToTest()) { + // Verify: the behavior of topic creation. + final String tp = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); + client.getPartitionsForTopic(tp).join(); + Optional metadata1 = pulsar1.getPulsarResources().getNamespaceResources() + .getPartitionedTopicResources() + .getPartitionedTopicMetadataAsync(TopicName.get(tp), true).join(); + assertTrue(metadata1.isPresent()); + assertEquals(metadata1.get().partitions, 3); + + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + assertEquals(getLookupRequestPermits(), lookupPermitsBefore); + }); + + // Cleanup. + admin1.topics().deletePartitionedTopic(tp, false); + } } @DataProvider(name = "autoCreationParamsAll") @@ -163,40 +253,32 @@ public void testGetMetadataIfNonPartitionedTopicExists(boolean configAllowAutoTo boolean paramMetadataAutoCreationEnabled, boolean isUsingHttpLookup, TopicDomain topicDomain) throws Exception { - conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); - conf.setDefaultNumPartitions(3); - conf.setAllowAutoTopicCreation(configAllowAutoTopicCreation); - setup(); + modifyTopicAutoCreation(configAllowAutoTopicCreation, TopicType.PARTITIONED, 3); - Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); - int lookupPermitsBefore = semaphore.availablePermits(); + int lookupPermitsBefore = getLookupRequestPermits(); - LookupService lookup = getLookupService(isUsingHttpLookup); // Create topic. - final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); - final TopicName topicName = TopicName.get(topicNameStr); - admin.topics().createNonPartitionedTopic(topicNameStr); - // Verify. - PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); - PartitionedTopicMetadata response = - lookup.getPartitionedTopicMetadata(topicName, paramMetadataAutoCreationEnabled).join(); - assertEquals(response.partitions, 0); - List partitionedTopics = admin.topics().getPartitionedTopicList("public/default"); - assertFalse(partitionedTopics.contains(topicNameStr)); - List topicList = admin.topics().getList("public/default"); - for (int i = 0; i < 3; i++) { - assertFalse(topicList.contains(topicName.getPartition(i))); - } + final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp_"); + admin1.topics().createNonPartitionedTopic(topicNameStr); + + PulsarClientImpl[] clientArray = getClientsToTest(isUsingHttpLookup); + for (PulsarClientImpl client : clientArray) { + // Verify: the result of get partitioned topic metadata. + PartitionedTopicMetadata response = + client.getPartitionedTopicMetadata(topicNameStr, paramMetadataAutoCreationEnabled).join(); + assertEquals(response.partitions, 0); + List partitionedTopics = admin1.topics().getPartitionedTopicList("public/default"); + assertFalse(partitionedTopics.contains(topicNameStr)); + verifyPartitionsNeverCreated(topicNameStr); - // Verify: lookup semaphore has been releases. - Awaitility.await().untilAsserted(() -> { - int lookupPermitsAfter = semaphore.availablePermits(); - assertEquals(lookupPermitsAfter, lookupPermitsBefore); - }); + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + assertEquals(getLookupRequestPermits(), lookupPermitsBefore); + }); + } // Cleanup. - client.close(); - admin.topics().delete(topicNameStr, false); + admin1.topics().delete(topicNameStr, false); } @Test(dataProvider = "autoCreationParamsAll") @@ -204,36 +286,30 @@ public void testGetMetadataIfPartitionedTopicExists(boolean configAllowAutoTopic boolean paramMetadataAutoCreationEnabled, boolean isUsingHttpLookup, TopicDomain topicDomain) throws Exception { - conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); - conf.setDefaultNumPartitions(3); - conf.setAllowAutoTopicCreation(configAllowAutoTopicCreation); - setup(); + modifyTopicAutoCreation(configAllowAutoTopicCreation, TopicType.PARTITIONED, 3); - Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); - int lookupPermitsBefore = semaphore.availablePermits(); + int lookupPermitsBefore = getLookupRequestPermits(); - LookupService lookup = getLookupService(isUsingHttpLookup); // Create topic. final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); - final TopicName topicName = TopicName.get(topicNameStr); - admin.topics().createPartitionedTopic(topicNameStr, 3); - // Verify. - PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); - PartitionedTopicMetadata response = - lookup.getPartitionedTopicMetadata(topicName, paramMetadataAutoCreationEnabled).join(); - assertEquals(response.partitions, 3); - List topicList = admin.topics().getList("public/default"); - assertFalse(topicList.contains(topicNameStr)); - - // Verify: lookup semaphore has been releases. - Awaitility.await().untilAsserted(() -> { - int lookupPermitsAfter = semaphore.availablePermits(); - assertEquals(lookupPermitsAfter, lookupPermitsBefore); - }); + admin1.topics().createPartitionedTopic(topicNameStr, 3); + + PulsarClientImpl[] clientArray = getClientsToTest(isUsingHttpLookup); + for (PulsarClientImpl client : clientArray) { + // Verify: the result of get partitioned topic metadata. + PartitionedTopicMetadata response = + client.getPartitionedTopicMetadata(topicNameStr, paramMetadataAutoCreationEnabled).join(); + assertEquals(response.partitions, 3); + verifyNonPartitionedTopicNeverCreated(topicNameStr); + + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + assertEquals(getLookupRequestPermits(), lookupPermitsBefore); + }); + } // Cleanup. - client.close(); - admin.topics().deletePartitionedTopic(topicNameStr, false); + admin1.topics().deletePartitionedTopic(topicNameStr, false); } @DataProvider(name = "clients") @@ -247,76 +323,96 @@ public Object[][] clients(){ @Test(dataProvider = "clients") public void testAutoCreatePartitionedTopic(boolean isUsingHttpLookup, TopicDomain topicDomain) throws Exception { - conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); - conf.setDefaultNumPartitions(3); - conf.setAllowAutoTopicCreation(true); - setup(); - - Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); - int lookupPermitsBefore = semaphore.availablePermits(); - - LookupService lookup = getLookupService(isUsingHttpLookup); - // Create topic. - final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); - final TopicName topicName = TopicName.get(topicNameStr); - // Verify. - PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); - PartitionedTopicMetadata response = lookup.getPartitionedTopicMetadata(topicName, true).join(); - assertEquals(response.partitions, 3); - List partitionedTopics = admin.topics().getPartitionedTopicList("public/default"); - assertTrue(partitionedTopics.contains(topicNameStr)); - List topicList = admin.topics().getList("public/default"); - assertFalse(topicList.contains(topicNameStr)); - for (int i = 0; i < 3; i++) { + modifyTopicAutoCreation(true, TopicType.PARTITIONED, 3); + + int lookupPermitsBefore = getLookupRequestPermits(); + + PulsarClientImpl[] clientArray = getClientsToTest(isUsingHttpLookup); + for (PulsarClientImpl client : clientArray) { + // Case-1: normal topic. + final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); + // Verify: the result of get partitioned topic metadata. + PartitionedTopicMetadata response = client.getPartitionedTopicMetadata(topicNameStr, true).join(); + assertEquals(response.partitions, 3); + // Verify: the behavior of topic creation. + List partitionedTopics = admin1.topics().getPartitionedTopicList("public/default"); + assertTrue(partitionedTopics.contains(topicNameStr)); + verifyNonPartitionedTopicNeverCreated(topicNameStr); // The API "getPartitionedTopicMetadata" only creates the partitioned metadata, it will not create the // partitions. - assertFalse(topicList.contains(topicName.getPartition(i))); + verifyPartitionsNeverCreated(topicNameStr); + + // Case-2: topic with suffix "-partition-1". + final String topicNameStrWithSuffix = BrokerTestUtil.newUniqueName( + topicDomain.value() + "://" + DEFAULT_NS + "/tp") + "-partition-1"; + // Verify: the result of get partitioned topic metadata. + PartitionedTopicMetadata response2 = + client.getPartitionedTopicMetadata(topicNameStrWithSuffix, true).join(); + assertEquals(response2.partitions, 0); + // Verify: the behavior of topic creation. + List partitionedTopics2 = + admin1.topics().getPartitionedTopicList("public/default"); + assertFalse(partitionedTopics2.contains(topicNameStrWithSuffix)); + assertFalse(partitionedTopics2.contains( + TopicName.get(topicNameStrWithSuffix).getPartitionedTopicName())); + + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + assertEquals(getLookupRequestPermits(), lookupPermitsBefore); + }); + // Cleanup. + admin1.topics().deletePartitionedTopic(topicNameStr, false); + try { + admin1.topics().delete(topicNameStrWithSuffix, false); + } catch (Exception ex) {} } - // Verify: lookup semaphore has been releases. - Awaitility.await().untilAsserted(() -> { - int lookupPermitsAfter = semaphore.availablePermits(); - assertEquals(lookupPermitsAfter, lookupPermitsBefore); - }); - - // Cleanup. - client.close(); - admin.topics().deletePartitionedTopic(topicNameStr, false); } @Test(dataProvider = "clients") public void testAutoCreateNonPartitionedTopic(boolean isUsingHttpLookup, TopicDomain topicDomain) throws Exception { - conf.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); - conf.setAllowAutoTopicCreation(true); - setup(); - - Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); - int lookupPermitsBefore = semaphore.availablePermits(); - - LookupService lookup = getLookupService(isUsingHttpLookup); - // Create topic. - final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); - final TopicName topicName = TopicName.get(topicNameStr); - // Verify. - PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); - PartitionedTopicMetadata response = lookup.getPartitionedTopicMetadata(topicName, true).join(); - assertEquals(response.partitions, 0); - List partitionedTopics = admin.topics().getPartitionedTopicList("public/default"); - assertFalse(partitionedTopics.contains(topicNameStr)); - List topicList = admin.topics().getList("public/default"); - assertFalse(topicList.contains(topicNameStr)); - - // Verify: lookup semaphore has been releases. - Awaitility.await().untilAsserted(() -> { - int lookupPermitsAfter = semaphore.availablePermits(); - assertEquals(lookupPermitsAfter, lookupPermitsBefore); - }); - - // Cleanup. - client.close(); - try { - admin.topics().delete(topicNameStr, false); - } catch (Exception ex) {} + modifyTopicAutoCreation(true, TopicType.NON_PARTITIONED, 3); + + int lookupPermitsBefore = getLookupRequestPermits(); + + PulsarClientImpl[] clientArray = getClientsToTest(isUsingHttpLookup); + for (PulsarClientImpl client : clientArray) { + // Case 1: normal topic. + final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); + // Verify: the result of get partitioned topic metadata. + PartitionedTopicMetadata response = client.getPartitionedTopicMetadata(topicNameStr, true).join(); + assertEquals(response.partitions, 0); + // Verify: the behavior of topic creation. + List partitionedTopics = admin1.topics().getPartitionedTopicList("public/default"); + assertFalse(partitionedTopics.contains(topicNameStr)); + verifyPartitionsNeverCreated(topicNameStr); + + // Case-2: topic with suffix "-partition-1". + final String topicNameStrWithSuffix = BrokerTestUtil.newUniqueName( + topicDomain.value() + "://" + DEFAULT_NS + "/tp") + "-partition-1"; + // Verify: the result of get partitioned topic metadata. + PartitionedTopicMetadata response2 = + client.getPartitionedTopicMetadata(topicNameStrWithSuffix, true).join(); + assertEquals(response2.partitions, 0); + // Verify: the behavior of topic creation. + List partitionedTopics2 = + admin1.topics().getPartitionedTopicList("public/default"); + assertFalse(partitionedTopics2.contains(topicNameStrWithSuffix)); + assertFalse(partitionedTopics2.contains( + TopicName.get(topicNameStrWithSuffix).getPartitionedTopicName())); + + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + assertEquals(getLookupRequestPermits(), lookupPermitsBefore); + }); + // Cleanup. + try { + admin1.topics().delete(topicNameStr, false); + } catch (Exception ex) {} + try { + admin1.topics().delete(topicNameStrWithSuffix, false); + } catch (Exception ex) {} + } } @DataProvider(name = "autoCreationParamsNotAllow") @@ -336,64 +432,38 @@ public Object[][] autoCreationParamsNotAllow(){ public void testGetMetadataIfNotAllowedCreate(boolean configAllowAutoTopicCreation, boolean paramMetadataAutoCreationEnabled, boolean isUsingHttpLookup) throws Exception { - if (!configAllowAutoTopicCreation && paramMetadataAutoCreationEnabled) { - // These test cases are for the following PR. - // Which was described in the Motivation of https://github.com/apache/pulsar/pull/22206. - return; - } - conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); - conf.setDefaultNumPartitions(3); - conf.setAllowAutoTopicCreation(configAllowAutoTopicCreation); - setup(); - - Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); - int lookupPermitsBefore = semaphore.availablePermits(); - - LookupService lookup = getLookupService(isUsingHttpLookup); - // Define topic. - final String topicNameStr = BrokerTestUtil.newUniqueName("persistent://" + DEFAULT_NS + "/tp"); - final TopicName topicName = TopicName.get(topicNameStr); - // Verify. - PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); - try { - lookup.getPartitionedTopicMetadata(TopicName.get(topicNameStr), paramMetadataAutoCreationEnabled).join(); - fail("Expect a not found exception"); - } catch (Exception e) { - log.warn("", e); - Throwable unwrapEx = FutureUtil.unwrapCompletionException(e); - assertTrue(unwrapEx instanceof PulsarClientException.TopicDoesNotExistException - || unwrapEx instanceof PulsarClientException.NotFoundException); - } + modifyTopicAutoCreation(configAllowAutoTopicCreation, TopicType.PARTITIONED, 3); - List partitionedTopics = admin.topics().getPartitionedTopicList("public/default"); - pulsar.getPulsarResources().getNamespaceResources().getPartitionedTopicResources().partitionedTopicExists(topicName); - assertFalse(partitionedTopics.contains(topicNameStr)); - List topicList = admin.topics().getList("public/default"); - assertFalse(topicList.contains(topicNameStr)); - for (int i = 0; i < 3; i++) { - assertFalse(topicList.contains(topicName.getPartition(i))); - } + int lookupPermitsBefore = getLookupRequestPermits(); - // Verify: lookup semaphore has been releases. - Awaitility.await().untilAsserted(() -> { - int lookupPermitsAfter = semaphore.availablePermits(); - assertEquals(lookupPermitsAfter, lookupPermitsBefore); - }); - - // Cleanup. - client.close(); - } + PulsarClientImpl[] clientArray = getClientsToTest(isUsingHttpLookup); + for (PulsarClientImpl client : clientArray) { + // Define topic. + final String topicNameStr = BrokerTestUtil.newUniqueName("persistent://" + DEFAULT_NS + "/tp"); + final TopicName topicName = TopicName.get(topicNameStr); + // Verify: the result of get partitioned topic metadata. + try { + client.getPartitionedTopicMetadata(topicNameStr, paramMetadataAutoCreationEnabled) + .join(); + fail("Expect a not found exception"); + } catch (Exception e) { + Throwable unwrapEx = FutureUtil.unwrapCompletionException(e); + assertTrue(unwrapEx instanceof PulsarClientException.TopicDoesNotExistException + || unwrapEx instanceof PulsarClientException.NotFoundException); + } + // Verify: the behavior of topic creation. + List partitionedTopics = admin1.topics().getPartitionedTopicList("public/default"); + pulsar1.getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .partitionedTopicExists(topicName); + assertFalse(partitionedTopics.contains(topicNameStr)); + verifyNonPartitionedTopicNeverCreated(topicNameStr); + verifyPartitionsNeverCreated(topicNameStr); - @DataProvider(name = "autoCreationParamsForNonPersistentTopic") - public Object[][] autoCreationParamsForNonPersistentTopic(){ - return new Object[][]{ - // configAllowAutoTopicCreation, paramCreateIfAutoCreationEnabled, isUsingHttpLookup. - {true, true, true}, - {true, true, false}, - {false, true, true}, - {false, true, false}, - {false, false, true} - }; + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + assertEquals(getLookupRequestPermits(), lookupPermitsBefore); + }); + } } /** @@ -408,66 +478,46 @@ public Object[][] autoCreationParamsForNonPersistentTopic(){ * param-auto-create = false * HTTP API: not found error * binary API: not support - * This test only guarantees that the behavior is the same as before. The following separated PR will fix the - * incorrect behavior. + * After PIP-344, the behavior will be the same as persistent topics, which was described in PIP-344. */ - @Test(dataProvider = "autoCreationParamsForNonPersistentTopic") - public void testGetNonPersistentMetadataIfNotAllowedCreate(boolean configAllowAutoTopicCreation, + @Test(dataProvider = "autoCreationParamsNotAllow") + public void testGetMetadataIfNotAllowedCreateOfNonPersistentTopic(boolean configAllowAutoTopicCreation, boolean paramMetadataAutoCreationEnabled, boolean isUsingHttpLookup) throws Exception { - conf.setAllowAutoTopicCreationType(TopicType.PARTITIONED); - conf.setDefaultNumPartitions(3); - conf.setAllowAutoTopicCreation(configAllowAutoTopicCreation); - setup(); - - Semaphore semaphore = pulsar.getBrokerService().getLookupRequestSemaphore(); - int lookupPermitsBefore = semaphore.availablePermits(); - - LookupService lookup = getLookupService(isUsingHttpLookup); - // Define topic. - final String topicNameStr = BrokerTestUtil.newUniqueName("non-persistent://" + DEFAULT_NS + "/tp"); - final TopicName topicName = TopicName.get(topicNameStr); - // Verify. - // Regarding non-persistent topic, we do not know whether it exists or not. - // Broker will return a non-partitioned metadata if partitioned metadata does not exist. - PulsarClient client = PulsarClient.builder().serviceUrl(pulsar.getBrokerServiceUrl()).build(); - - if (!configAllowAutoTopicCreation && !paramMetadataAutoCreationEnabled && isUsingHttpLookup) { + modifyTopicAutoCreation(configAllowAutoTopicCreation, TopicType.PARTITIONED, 3); + + int lookupPermitsBefore = getLookupRequestPermits(); + + PulsarClientImpl[] clientArray = getClientsToTest(isUsingHttpLookup); + for (PulsarClientImpl client : clientArray) { + // Define topic. + final String topicNameStr = BrokerTestUtil.newUniqueName("non-persistent://" + DEFAULT_NS + "/tp"); + final TopicName topicName = TopicName.get(topicNameStr); + // Verify: the result of get partitioned topic metadata. try { - lookup.getPartitionedTopicMetadata(TopicName.get(topicNameStr), paramMetadataAutoCreationEnabled) + PartitionedTopicMetadata topicMetadata = client + .getPartitionedTopicMetadata(topicNameStr, paramMetadataAutoCreationEnabled) .join(); - Assert.fail("Expected a not found ex"); + log.info("Get topic metadata: {}", topicMetadata.partitions); + fail("Expected a not found ex"); } catch (Exception ex) { - // Cleanup. - client.close(); - return; + Throwable unwrapEx = FutureUtil.unwrapCompletionException(ex); + assertTrue(unwrapEx instanceof PulsarClientException.TopicDoesNotExistException + || unwrapEx instanceof PulsarClientException.NotFoundException); } - } - PartitionedTopicMetadata metadata = lookup - .getPartitionedTopicMetadata(TopicName.get(topicNameStr), paramMetadataAutoCreationEnabled).join(); - if (configAllowAutoTopicCreation && paramMetadataAutoCreationEnabled) { - assertEquals(metadata.partitions, 3); - } else { - assertEquals(metadata.partitions, 0); - } - - List partitionedTopics = admin.topics().getPartitionedTopicList("public/default"); - pulsar.getPulsarResources().getNamespaceResources().getPartitionedTopicResources() - .partitionedTopicExists(topicName); - if (configAllowAutoTopicCreation && paramMetadataAutoCreationEnabled) { - assertTrue(partitionedTopics.contains(topicNameStr)); - } else { + // Verify: the behavior of topic creation. + List partitionedTopics = admin1.topics().getPartitionedTopicList("public/default"); + pulsar1.getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .partitionedTopicExists(topicName); assertFalse(partitionedTopics.contains(topicNameStr)); + verifyNonPartitionedTopicNeverCreated(topicNameStr); + verifyPartitionsNeverCreated(topicNameStr); } // Verify: lookup semaphore has been releases. Awaitility.await().untilAsserted(() -> { - int lookupPermitsAfter = semaphore.availablePermits(); - assertEquals(lookupPermitsAfter, lookupPermitsBefore); + assertEquals(getLookupRequestPermits(), lookupPermitsBefore); }); - - // Cleanup. - client.close(); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicsTest.java index 9aa29f08c5ce8..c9457e1a8883f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicsTest.java @@ -56,6 +56,7 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationDataHttps; import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.broker.namespace.TopicExistsInfo; import org.apache.pulsar.broker.rest.Topics; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.BrokerServiceException; @@ -357,9 +358,12 @@ public void testLookUpWithException() throws Exception { CompletableFuture future = new CompletableFuture(); future.completeExceptionally(new BrokerServiceException("Fake Exception")); CompletableFuture existFuture = new CompletableFuture(); - existFuture.complete(true); + existFuture.complete(TopicExistsInfo.newNonPartitionedTopicExists()); doReturn(future).when(nameSpaceService).getBrokerServiceUrlAsync(any(), any()); doReturn(existFuture).when(nameSpaceService).checkTopicExists(any()); + CompletableFuture existBooleanFuture = new CompletableFuture(); + existBooleanFuture.complete(false); + doReturn(existBooleanFuture).when(nameSpaceService).checkNonPartitionedTopicExists(any()); doReturn(nameSpaceService).when(pulsar).getNamespaceService(); AsyncResponse asyncResponse = mock(AsyncResponse.class); ProducerMessages producerMessages = new ProducerMessages(); @@ -370,7 +374,7 @@ public void testLookUpWithException() throws Exception { topics.produceOnPersistentTopic(asyncResponse, testTenant, testNamespace, testTopicName, false, producerMessages); ArgumentCaptor responseCaptor = ArgumentCaptor.forClass(RestException.class); verify(asyncResponse, timeout(5000).times(1)).resume(responseCaptor.capture()); - Assert.assertEquals(responseCaptor.getValue().getMessage(), "Can't find owner of given topic."); + Assert.assertTrue(responseCaptor.getValue().getMessage().contains(topicName + " not found")); } @Test @@ -378,8 +382,11 @@ public void testLookUpTopicNotExist() throws Exception { String topicName = "persistent://" + testTenant + "/" + testNamespace + "/" + testTopicName; NamespaceService nameSpaceService = mock(NamespaceService.class); CompletableFuture existFuture = new CompletableFuture(); - existFuture.complete(false); + existFuture.complete(TopicExistsInfo.newTopicNotExists()); + CompletableFuture existBooleanFuture = new CompletableFuture(); + existBooleanFuture.complete(false); doReturn(existFuture).when(nameSpaceService).checkTopicExists(any()); + doReturn(existBooleanFuture).when(nameSpaceService).checkNonPartitionedTopicExists(any()); doReturn(nameSpaceService).when(pulsar).getNamespaceService(); AsyncResponse asyncResponse = mock(AsyncResponse.class); ProducerMessages producerMessages = new ProducerMessages(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpTopicLookupv2Test.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpTopicLookupv2Test.java index 6a6065bc289f6..783a92b485ed8 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpTopicLookupv2Test.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/lookup/http/HttpTopicLookupv2Test.java @@ -44,6 +44,7 @@ import org.apache.pulsar.broker.lookup.RedirectData; import org.apache.pulsar.broker.lookup.v1.TopicLookup; import org.apache.pulsar.broker.namespace.NamespaceService; +import org.apache.pulsar.broker.namespace.TopicExistsInfo; import org.apache.pulsar.broker.resources.ClusterResources; import org.apache.pulsar.broker.resources.NamespaceResources; import org.apache.pulsar.broker.resources.PulsarResources; @@ -149,9 +150,12 @@ public void testLookupTopicNotExist() throws Exception { config.setAuthorizationEnabled(true); NamespaceService namespaceService = pulsar.getNamespaceService(); - CompletableFuture future = new CompletableFuture<>(); - future.complete(false); + CompletableFuture future = new CompletableFuture<>(); + future.complete(TopicExistsInfo.newTopicNotExists()); doReturn(future).when(namespaceService).checkTopicExists(any(TopicName.class)); + CompletableFuture booleanFuture = new CompletableFuture<>(); + booleanFuture.complete(false); + doReturn(booleanFuture).when(namespaceService).checkNonPartitionedTopicExists(any(TopicName.class)); AsyncResponse asyncResponse1 = mock(AsyncResponse.class); destLookup.lookupTopicAsync(asyncResponse1, TopicDomain.persistent.value(), "myprop", "usc", "ns2", "topic_not_exist", false, null, null); @@ -260,9 +264,12 @@ public void testValidateReplicationSettingsOnNamespace() throws Exception { policies3Future.complete(Optional.of(policies3)); doReturn(policies3Future).when(namespaceResources).getPoliciesAsync(namespaceName2); NamespaceService namespaceService = pulsar.getNamespaceService(); - CompletableFuture future = new CompletableFuture<>(); - future.complete(false); + CompletableFuture future = new CompletableFuture<>(); + future.complete(TopicExistsInfo.newTopicNotExists()); doReturn(future).when(namespaceService).checkTopicExists(any(TopicName.class)); + CompletableFuture booleanFuture = new CompletableFuture<>(); + booleanFuture.complete(false); + doReturn(future).when(namespaceService).checkNonPartitionedTopicExists(any(TopicName.class)); destLookup.lookupTopicAsync(asyncResponse, TopicDomain.persistent.value(), property, cluster, ns2, "invalid-localCluster", false, null, null); verify(asyncResponse).resume(arg.capture()); @@ -278,4 +285,35 @@ public void testDataPojo() { assertEquals(data2.getRedirectLookupAddress(), url); } + @Test + public void topicNotFound() throws Exception { + MockTopicLookup destLookup = spy(MockTopicLookup.class); + doReturn(false).when(destLookup).isRequestHttps(); + BrokerService brokerService = pulsar.getBrokerService(); + doReturn(new Semaphore(1000,true)).when(brokerService).getLookupRequestSemaphore(); + destLookup.setPulsar(pulsar); + doReturn("null").when(destLookup).clientAppId(); + Field uriField = PulsarWebResource.class.getDeclaredField("uri"); + uriField.setAccessible(true); + UriInfo uriInfo = mock(UriInfo.class); + uriField.set(destLookup, uriInfo); + URI uri = URI.create("http://localhost:8080/lookup/v2/destination/topic/myprop/usc/ns2/topic1"); + doReturn(uri).when(uriInfo).getRequestUri(); + config.setAuthorizationEnabled(true); + NamespaceService namespaceService = pulsar.getNamespaceService(); + CompletableFuture future = new CompletableFuture<>(); + future.complete(TopicExistsInfo.newTopicNotExists()); + doReturn(future).when(namespaceService).checkTopicExists(any(TopicName.class)); + + // Get the current semaphore first + Integer state1 = pulsar.getBrokerService().getLookupRequestSemaphore().availablePermits(); + AsyncResponse asyncResponse1 = mock(AsyncResponse.class); + // We used a nonexistent topic to test + destLookup.lookupTopicAsync(asyncResponse1, TopicDomain.persistent.value(), "myprop", "usc", "ns2", "topic2", false, null, null); + // Gets semaphore status + Integer state2 = pulsar.getBrokerService().getLookupRequestSemaphore().availablePermits(); + // If it is successfully released, it should be equal + assertEquals(state1, state2); + + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java index 2a8a849ef9c06..38a60165d5606 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java @@ -816,14 +816,15 @@ public void testCheckTopicExists(String topicDomain) throws Exception { String topic = topicDomain + "://prop/ns-abc/" + UUID.randomUUID(); admin.topics().createNonPartitionedTopic(topic); Awaitility.await().untilAsserted(() -> { - assertTrue(pulsar.getNamespaceService().checkTopicExists(TopicName.get(topic)).get()); + assertTrue(pulsar.getNamespaceService().checkTopicExists(TopicName.get(topic)).get().isExists()); }); String partitionedTopic = topicDomain + "://prop/ns-abc/" + UUID.randomUUID(); admin.topics().createPartitionedTopic(partitionedTopic, 5); Awaitility.await().untilAsserted(() -> { - assertTrue(pulsar.getNamespaceService().checkTopicExists(TopicName.get(partitionedTopic)).get()); - assertTrue(pulsar.getNamespaceService().checkTopicExists(TopicName.get(partitionedTopic + "-partition-2")).get()); + assertTrue(pulsar.getNamespaceService().checkTopicExists(TopicName.get(partitionedTopic)).get().isExists()); + assertTrue(pulsar.getNamespaceService() + .checkTopicExists(TopicName.get(partitionedTopic + "-partition-2")).get().isExists()); }); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java index 7790940c1327f..8fdf0723ea8d1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service; +import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -99,6 +100,7 @@ public void testCreateConsumerAfterOnePartDeleted() throws Exception { Consumer consumerAllPartition = pulsarClient.newConsumer(Schema.STRING).topic(topic) .subscriptionName(subscription).isAckReceiptEnabled(true).subscribe(); Message msg = consumerAllPartition.receive(2, TimeUnit.SECONDS); + assertNotNull(msg); String receivedMsgValue = msg.getValue(); log.info("received msg: {}", receivedMsgValue); consumerAllPartition.acknowledge(msg); 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 7735f66e7838a..4d6cf96a01068 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 @@ -32,6 +32,7 @@ import lombok.Getter; import lombok.NonNull; import org.apache.commons.lang3.StringUtils; +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.ConsumerBuilder; @@ -58,7 +59,6 @@ import org.apache.pulsar.client.impl.conf.TopicConsumerConfigurationData; import org.apache.pulsar.client.util.RetryMessageUtil; import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.util.FutureUtil; @Getter(AccessLevel.PUBLIC) @@ -104,6 +104,31 @@ public Consumer subscribe() throws PulsarClientException { } } + private CompletableFuture checkDlqAlreadyExists(String topic) { + CompletableFuture existsFuture = new CompletableFuture<>(); + client.getPartitionedTopicMetadata(topic, false).thenAccept(metadata -> { + TopicName topicName = TopicName.get(topic); + if (topicName.isPersistent()) { + // Either partitioned or non-partitioned, it exists. + existsFuture.complete(true); + } else { + // If it is a non-persistent topic, return true only it is a partitioned topic. + existsFuture.complete(metadata != null && metadata.partitions > 0); + } + }).exceptionally(ex -> { + Throwable actEx = FutureUtil.unwrapCompletionException(ex); + if (actEx instanceof PulsarClientException.NotFoundException + || actEx instanceof PulsarClientException.TopicDoesNotExistException + || actEx instanceof PulsarAdminException.NotFoundException) { + existsFuture.complete(false); + } else { + existsFuture.completeExceptionally(ex); + } + return null; + }); + return existsFuture; + } + @Override public CompletableFuture> subscribeAsync() { if (conf.getTopicNames().isEmpty() && conf.getTopicsPattern() == null) { @@ -135,20 +160,18 @@ public CompletableFuture> subscribeAsync() { DeadLetterPolicy deadLetterPolicy = conf.getDeadLetterPolicy(); if (deadLetterPolicy == null || StringUtils.isBlank(deadLetterPolicy.getRetryLetterTopic()) || StringUtils.isBlank(deadLetterPolicy.getDeadLetterTopic())) { - CompletableFuture retryLetterTopicMetadata = - client.getPartitionedTopicMetadata(oldRetryLetterTopic, true); - CompletableFuture deadLetterTopicMetadata = - client.getPartitionedTopicMetadata(oldDeadLetterTopic, true); + CompletableFuture retryLetterTopicMetadata = checkDlqAlreadyExists(oldRetryLetterTopic); + CompletableFuture deadLetterTopicMetadata = checkDlqAlreadyExists(oldDeadLetterTopic); applyDLQConfig = CompletableFuture.allOf(retryLetterTopicMetadata, deadLetterTopicMetadata) .thenAccept(__ -> { String retryLetterTopic = topicFirst + "-" + conf.getSubscriptionName() + RetryMessageUtil.RETRY_GROUP_TOPIC_SUFFIX; String deadLetterTopic = topicFirst + "-" + conf.getSubscriptionName() + RetryMessageUtil.DLQ_GROUP_TOPIC_SUFFIX; - if (retryLetterTopicMetadata.join().partitions > 0) { + if (retryLetterTopicMetadata.join()) { retryLetterTopic = oldRetryLetterTopic; } - if (deadLetterTopicMetadata.join().partitions > 0) { + if (deadLetterTopicMetadata.join()) { deadLetterTopic = oldDeadLetterTopic; } if (deadLetterPolicy == null) { From db274ecba2ca6cb955fb3eee04a414f042ff717d Mon Sep 17 00:00:00 2001 From: Rajan Dhabalia Date: Sat, 14 Oct 2023 07:34:08 -0700 Subject: [PATCH 561/699] [fix][client] fix producer/consumer perform lookup for migrated topic (#21356) Co-authored-by: Rajan Dhabalia (cherry picked from commit d09642c7cbfc18cf532aaebf550f5ac6206c5c4b) (cherry picked from commit d4f3c59f4f35b23ff581f0a2f94eb772c47e6fca) --- .../pulsar/client/impl/ConnectionHandler.java | 13 +++++++-- .../pulsar/client/impl/LookupService.java | 1 - .../pulsar/client/impl/PulsarClientImpl.java | 29 +++++++++++++++++-- 3 files changed, 36 insertions(+), 7 deletions(-) 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 6403d48d7be0c..2b7fb90b14a47 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 @@ -96,9 +96,16 @@ protected void grabCnx() { try { CompletableFuture cnxFuture; if (state.redirectedClusterURI != null) { - InetSocketAddress address = InetSocketAddress.createUnresolved(state.redirectedClusterURI.getHost(), - state.redirectedClusterURI.getPort()); - cnxFuture = state.client.getConnection(address, address, randomKeyForSelectConnection); + if (state.topic == null) { + InetSocketAddress address = InetSocketAddress.createUnresolved(state.redirectedClusterURI.getHost(), + state.redirectedClusterURI.getPort()); + cnxFuture = state.client.getConnection(address, address, randomKeyForSelectConnection); + } else { + // once, client receives redirection url, client has to perform lookup on migrated + // cluster to find the broker that owns the topic and then create connection. + // below method, performs the lookup for a given topic and then creates connection + cnxFuture = state.client.getConnection(state.topic, (state.redirectedClusterURI.toString())); + } } else if (state.topic == null) { cnxFuture = state.client.getConnectionToServiceUrl(); } else { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java index 978450ed6894d..ba99cb77550f5 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java @@ -124,5 +124,4 @@ CompletableFuture getPartitionedTopicMetadata(TopicNam */ CompletableFuture getTopicsUnderNamespace(NamespaceName namespace, Mode mode, String topicPattern, String topicsHash); - } 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 9a5ec8b874bb6..c4c2268270fe5 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 @@ -33,6 +33,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -106,6 +107,7 @@ public class PulsarClientImpl implements PulsarClient { private final boolean createdScheduledProviders; private LookupService lookup; + private Map urlLookupMap = new ConcurrentHashMap<>(); private final ConnectionPool cnxPool; @Getter private final Timer timer; @@ -962,6 +964,23 @@ public CompletableFuture getConnection(final String topic) { .thenCompose(pair -> getConnection(pair.getLeft(), pair.getRight(), cnxPool.genRandomKeyToSelectCon())); } + public CompletableFuture getConnection(final String topic, final String url) { + TopicName topicName = TopicName.get(topic); + return getLookup(url).getBroker(topicName) + .thenCompose(pair -> getConnection(pair.getLeft(), pair.getRight(), cnxPool.genRandomKeyToSelectCon())); + } + + public LookupService getLookup(String serviceUrl) { + return urlLookupMap.computeIfAbsent(serviceUrl, url -> { + try { + return createLookup(serviceUrl); + } catch (PulsarClientException e) { + log.warn("Failed to update url to lookup service {}, {}", url, e.getMessage()); + throw new IllegalStateException("Failed to update url " + url); + } + }); + } + public CompletableFuture getConnectionToServiceUrl() { if (!(lookup instanceof BinaryProtoLookupService)) { return FutureUtil.failedFuture(new PulsarClientException.InvalidServiceURL( @@ -1020,10 +1039,14 @@ public LookupService getLookup() { } public void reloadLookUp() throws PulsarClientException { - if (conf.getServiceUrl().startsWith("http")) { - lookup = new HttpLookupService(conf, eventLoopGroup); + lookup = createLookup(conf.getServiceUrl()); + } + + public LookupService createLookup(String url) throws PulsarClientException { + if (url.startsWith("http")) { + return new HttpLookupService(conf, eventLoopGroup); } else { - lookup = new BinaryProtoLookupService(this, conf.getServiceUrl(), conf.getListenerName(), conf.isUseTls(), + return new BinaryProtoLookupService(this, url, conf.getListenerName(), conf.isUseTls(), externalExecutorProvider.getExecutor()); } } From 1ba6b3adb097e3afb0d4dd326fd497926ee71355 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Tue, 25 Jun 2024 11:25:04 +0530 Subject: [PATCH 562/699] [fix][proxy] Add missing parameter in newPartitionMetadataRequest call --- .../apache/pulsar/proxy/server/DefaultLookupProxyHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DefaultLookupProxyHandler.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DefaultLookupProxyHandler.java index 15163dd7f6063..c976e44c05cdf 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DefaultLookupProxyHandler.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/DefaultLookupProxyHandler.java @@ -251,7 +251,7 @@ private void handlePartitionMetadataResponse(CommandPartitionedTopicMetadata par // Connected to backend broker long requestId = proxyConnection.newRequestId(); ByteBuf command; - command = Commands.newPartitionMetadataRequest(topicName.toString(), requestId); + command = Commands.newPartitionMetadataRequest(topicName.toString(), requestId, true); clientCnx.newLookup(command, requestId).whenComplete((r, t) -> { if (t != null) { log.warn("[{}] failed to get Partitioned metadata : {}", topicName.toString(), From 2c987c8402fd098d36a34a2be28f26fcc5acb144 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 24 Oct 2023 13:35:04 +0300 Subject: [PATCH 563/699] [fix][test] Fix thread leaks in Managed Ledger tests and remove duplicate shutdown code (#21426) (cherry picked from commit c6704dfcd977c790168e4bbad36ac67b555a3041) (cherry picked from commit eb9a95d4e2d4d4ab6a9940a1846a192651cf23a2) --- .../impl/ManagedLedgerFactoryImpl.java | 68 +++---------------- .../mledger/impl/ManagedLedgerTest.java | 8 ++- .../test/BookKeeperClusterTestCase.java | 4 +- .../test/MockedBookKeeperTestCase.java | 10 ++- .../broker/testcontext/PulsarTestContext.java | 3 +- .../metadata/impl/AbstractMetadataStore.java | 4 +- .../BookKeeperClusterTestCase.java | 2 +- .../test/MockedBookKeeperTestCase.java | 4 +- 8 files changed, 35 insertions(+), 68 deletions(-) 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 805958531403f..49d0cf847d7c2 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 @@ -36,6 +36,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; @@ -535,13 +536,12 @@ public CompletableFuture shutdownAsync() throws ManagedLedgerException { int numLedgers = ledgerNames.size(); log.info("Closing {} ledgers", numLedgers); for (String ledgerName : ledgerNames) { - CompletableFuture future = new CompletableFuture<>(); - futures.add(future); CompletableFuture ledgerFuture = ledgers.remove(ledgerName); if (ledgerFuture == null) { - future.complete(null); continue; } + CompletableFuture future = new CompletableFuture<>(); + futures.add(future); ledgerFuture.whenCompleteAsync((managedLedger, throwable) -> { if (throwable != null || managedLedger == null) { future.complete(null); @@ -606,68 +606,20 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { })); } })); - entryCacheManager.clear(); - return FutureUtil.waitForAll(futures).thenAccept(__ -> { + return FutureUtil.waitForAll(futures).thenAcceptAsync(__ -> { //wait for tasks in scheduledExecutor executed. - scheduledExecutor.shutdown(); + scheduledExecutor.shutdownNow(); + entryCacheManager.clear(); }); } @Override public void shutdown() throws InterruptedException, ManagedLedgerException { - if (closed) { - throw new ManagedLedgerException.ManagedLedgerFactoryClosedException(); + try { + shutdownAsync().get(); + } catch (ExecutionException e) { + throw getManagedLedgerException(e.getCause()); } - closed = true; - - statsTask.cancel(true); - flushCursorsTask.cancel(true); - cacheEvictionExecutor.shutdownNow(); - - // take a snapshot of ledgers currently in the map to prevent race conditions - List> ledgers = new ArrayList<>(this.ledgers.values()); - int numLedgers = ledgers.size(); - final CountDownLatch latch = new CountDownLatch(numLedgers); - log.info("Closing {} ledgers", numLedgers); - - for (CompletableFuture ledgerFuture : ledgers) { - ManagedLedgerImpl ledger = ledgerFuture.getNow(null); - if (ledger == null) { - latch.countDown(); - continue; - } - - ledger.asyncClose(new AsyncCallbacks.CloseCallback() { - @Override - public void closeComplete(Object ctx) { - latch.countDown(); - } - - @Override - public void closeFailed(ManagedLedgerException exception, Object ctx) { - log.warn("[{}] Got exception when closing managed ledger: {}", ledger.getName(), exception); - latch.countDown(); - } - }, null); - } - - latch.await(); - log.info("{} ledgers closed", numLedgers); - - if (isBookkeeperManaged) { - try { - BookKeeper bookkeeper = bookkeeperFactory.get(); - if (bookkeeper != null) { - bookkeeper.close(); - } - } catch (BKException e) { - throw new ManagedLedgerException(e); - } - } - - scheduledExecutor.shutdownNow(); - - entryCacheManager.clear(); } @Override 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 797dbe88a0196..fc47b8c3f72a4 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 @@ -200,10 +200,10 @@ private DeleteLedgerInfo makeDelayIfDoLedgerDelete(LedgerHandle ledger, final At bkc.asyncDeleteLedger(ledgerId, originalCb, ctx); } else { deleteLedgerInfo.hasCalled = true; - new Thread(() -> { + cachedExecutor.submit(() -> { Awaitility.await().atMost(Duration.ofSeconds(60)).until(signal::get); bkc.asyncDeleteLedger(ledgerId, cb, ctx); - }).start(); + }); } return null; }).when(spyBookKeeper).asyncDeleteLedger(any(long.class), any(AsyncCallback.DeleteCallback.class), any()); @@ -220,6 +220,7 @@ private DeleteLedgerInfo makeDelayIfDoLedgerDelete(LedgerHandle ledger, final At public void testLedgerInfoMetaCorrectIfAddEntryTimeOut() throws Exception { String mlName = "testLedgerInfoMetaCorrectIfAddEntryTimeOut"; BookKeeper spyBookKeeper = spy(bkc); + @Cleanup("shutdown") ManagedLedgerFactoryImpl factory = new ManagedLedgerFactoryImpl(metadataStore, spyBookKeeper); ManagedLedgerImpl ml = (ManagedLedgerImpl) factory.open(mlName); @@ -3938,6 +3939,7 @@ public void testCancellationOfScheduledTasks() throws Exception { public void testInactiveLedgerRollOver() throws Exception { int inactiveLedgerRollOverTimeMs = 5; ManagedLedgerFactoryConfig factoryConf = new ManagedLedgerFactoryConfig(); + @Cleanup("shutdown") ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); ManagedLedgerConfig config = new ManagedLedgerConfig(); config.setInactiveLedgerRollOverTime(inactiveLedgerRollOverTimeMs, TimeUnit.MILLISECONDS); @@ -3969,7 +3971,6 @@ public void testInactiveLedgerRollOver() throws Exception { List ledgers = ledger.getLedgersInfoAsList(); assertEquals(ledgers.size(), totalAddEntries); ledger.close(); - factory.shutdown(); } @Test @@ -4022,6 +4023,7 @@ public void testDontRollOverInactiveLedgersWhenMetadataServiceInvalid() throws E @Test public void testOffloadTaskCancelled() throws Exception { + @Cleanup("shutdown") ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc); ManagedLedgerConfig config = new ManagedLedgerConfig(); config.setMaxEntriesPerLedger(2); diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java b/managed-ledger/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java index e316083e837fd..a323ecfeb8ea6 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java @@ -242,7 +242,9 @@ protected void startZKCluster() throws Exception { zkc = zkUtil.getZooKeeperClient(); metadataStore = new FaultInjectionMetadataStore( MetadataStoreExtended.create(zkUtil.getZooKeeperConnectString(), - MetadataStoreConfig.builder().build())); + MetadataStoreConfig.builder() + .metadataStoreName("metastore-" + getClass().getSimpleName()) + .build())); } /** diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/test/MockedBookKeeperTestCase.java b/managed-ledger/src/test/java/org/apache/bookkeeper/test/MockedBookKeeperTestCase.java index e2101268b09e6..645563eb78c4d 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/test/MockedBookKeeperTestCase.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/test/MockedBookKeeperTestCase.java @@ -26,6 +26,7 @@ import lombok.SneakyThrows; import org.apache.bookkeeper.client.PulsarMockBookKeeper; import org.apache.bookkeeper.common.util.OrderedScheduler; +import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerFactoryConfig; import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; import org.apache.pulsar.metadata.api.MetadataStoreConfig; @@ -70,7 +71,8 @@ public MockedBookKeeperTestCase(int numBookies) { public final void setUp(Method method) throws Exception { LOG.info(">>>>>> starting {}", method); metadataStore = new FaultInjectionMetadataStore( - MetadataStoreExtended.create("memory:local", MetadataStoreConfig.builder().build())); + MetadataStoreExtended.create("memory:local", + MetadataStoreConfig.builder().metadataStoreName("metastore-" + method.getName()).build())); try { // start bookkeeper service @@ -102,7 +104,11 @@ public final void tearDown(Method method) { } try { LOG.info("@@@@@@@@@ stopping " + method); - factory.shutdownAsync().get(10, TimeUnit.SECONDS); + try { + factory.shutdownAsync().get(10, TimeUnit.SECONDS); + } catch (ManagedLedgerException.ManagedLedgerFactoryClosedException e) { + // ignore + } factory = null; stopBookKeeper(); metadataStore.close(); 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 449f7fc7d57e1..326cfaf2b713d 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 @@ -635,7 +635,8 @@ private void initializeCommonPulsarServices(SpyConfig spyConfig) { } else { try { MetadataStoreExtended store = MetadataStoreFactoryImpl.createExtended("memory:local", - MetadataStoreConfig.builder().build()); + MetadataStoreConfig.builder() + .metadataStoreName(MetadataStoreConfig.METADATA_STORE).build()); registerCloseable(() -> { store.close(); resetSpyOrMock(store); 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 b520818fb28d6..d099d79d05c4d 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 @@ -89,7 +89,9 @@ public abstract class AbstractMetadataStore implements MetadataStoreExtended, Co protected abstract CompletableFuture existsFromStore(String path); protected AbstractMetadataStore(String metadataStoreName) { - this.executor = new ScheduledThreadPoolExecutor(1, new DefaultThreadFactory(metadataStoreName)); + this.executor = new ScheduledThreadPoolExecutor(1, + new DefaultThreadFactory( + StringUtils.isNotBlank(metadataStoreName) ? metadataStoreName : getClass().getSimpleName())); registerListener(this); this.childrenCache = Caffeine.newBuilder() 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 c681a1f0764ee..9a8e3ef5a2d4f 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 @@ -238,7 +238,7 @@ protected void startZKCluster() throws Exception { zkc = zkUtil.getZooKeeperClient(); metadataStore = new FaultInjectionMetadataStore( MetadataStoreExtended.create(zkUtil.getZooKeeperConnectString(), - MetadataStoreConfig.builder().build())); + MetadataStoreConfig.builder().metadataStoreName("metastore-" + getClass().getSimpleName()).build())); } /** 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 e0b10ca0280d2..ac5aa3bd8927e 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 @@ -71,7 +71,9 @@ public MockedBookKeeperTestCase(int numBookies) { public void setUp(Method method) throws Exception { LOG.info(">>>>>> starting {}", method); metadataStore = new FaultInjectionMetadataStore(MetadataStoreExtended.create("memory:local", - MetadataStoreConfig.builder().build())); + MetadataStoreConfig.builder() + .metadataStoreName("metastore-" + method.getName()) + .build())); try { // start bookkeeper service startBookKeeper(); From 94f9ec3636a4248127a41f1b7dbcba7e7e7c7b8d Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 19 Jun 2024 15:13:57 +0800 Subject: [PATCH 564/699] [fix] [broker] Messages lost on the remote cluster when using topic level replication (#22890) (cherry picked from commit feae58988d672767c076daa0c7caa5613cbba36e) (cherry picked from commit 3f28fa3e7ba9152a3b6c4f9dbde460b6fec675f1) --- .../service/persistent/PersistentTopic.java | 49 ++++----- .../broker/service/OneWayReplicatorTest.java | 102 ++++++++++++++++++ .../service/OneWayReplicatorTestBase.java | 22 ++++ .../OneWayReplicatorUsingGlobalZKTest.java | 5 + 4 files changed, 151 insertions(+), 27 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 e3ffb4974689a..f2b723c38cbca 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 @@ -403,14 +403,6 @@ public CompletableFuture initialize() { PersistentTopic.this.topicCompactionService = service; this.createPersistentSubscriptions(); })); - - for (ManagedCursor cursor : ledger.getCursors()) { - if (cursor.getName().startsWith(replicatorPrefix)) { - String localCluster = brokerService.pulsar().getConfiguration().getClusterName(); - String remoteCluster = PersistentReplicator.getRemoteCluster(cursor.getName()); - futures.add(addReplicationCluster(remoteCluster, cursor, localCluster)); - } - } return FutureUtil.waitForAll(futures).thenCompose(__ -> brokerService.pulsar().getPulsarResources().getNamespaceResources() .getPoliciesAsync(TopicName.get(topic).getNamespaceObject()) @@ -441,6 +433,7 @@ public CompletableFuture initialize() { isAllowAutoUpdateSchema = policies.is_allow_auto_update_schema; }, getOrderedExecutor()) .thenCompose(ignore -> initTopicPolicy()) + .thenCompose(ignore -> removeOrphanReplicationCursors()) .exceptionally(ex -> { log.warn("[{}] Error getting policies {} and isEncryptionRequired will be set to false", topic, ex.getMessage()); @@ -518,6 +511,21 @@ private void createPersistentSubscriptions() { checkReplicatedSubscriptionControllerState(); } + private CompletableFuture removeOrphanReplicationCursors() { + List> futures = new ArrayList<>(); + List replicationClusters = topicPolicies.getReplicationClusters().get(); + for (ManagedCursor cursor : ledger.getCursors()) { + if (cursor.getName().startsWith(replicatorPrefix)) { + String remoteCluster = PersistentReplicator.getRemoteCluster(cursor.getName()); + if (!replicationClusters.contains(remoteCluster)) { + log.warn("Remove the orphan replicator because the cluster '{}' does not exist", remoteCluster); + futures.add(removeReplicator(remoteCluster)); + } + } + } + return FutureUtil.waitForAll(futures); + } + /** * Unload a subscriber. * @throws SubscriptionNotFoundException If subscription not founded. @@ -1933,30 +1941,17 @@ public void openCursorFailed(ManagedLedgerException exception, Object ctx) { return future; } - private CompletableFuture checkReplicationCluster(String remoteCluster) { - return brokerService.getPulsar().getPulsarResources().getNamespaceResources() - .getPoliciesAsync(TopicName.get(topic).getNamespaceObject()) - .thenApply(optPolicies -> optPolicies.map(policies -> policies.replication_clusters) - .orElse(Collections.emptySet()).contains(remoteCluster) - || topicPolicies.getReplicationClusters().get().contains(remoteCluster)); - } - protected CompletableFuture addReplicationCluster(String remoteCluster, ManagedCursor cursor, String localCluster) { return AbstractReplicator.validatePartitionedTopicAsync(PersistentTopic.this.getName(), brokerService) - .thenCompose(__ -> checkReplicationCluster(remoteCluster)) - .thenCompose(clusterExists -> { - if (!clusterExists) { - log.warn("Remove the replicator because the cluster '{}' does not exist", remoteCluster); - return removeReplicator(remoteCluster).thenApply(__ -> null); - } - return brokerService.pulsar().getPulsarResources().getClusterResources() - .getClusterAsync(remoteCluster) - .thenApply(clusterData -> - brokerService.getReplicationClient(remoteCluster, clusterData)); - }) + .thenCompose(__ -> brokerService.pulsar().getPulsarResources().getClusterResources() + .getClusterAsync(remoteCluster) + .thenApply(clusterData -> + brokerService.getReplicationClient(remoteCluster, clusterData))) .thenAccept(replicationClient -> { if (replicationClient == null) { + log.error("[{}] Can not create replicator because the remote client can not be created." + + " remote cluster: {}.", topic, remoteCluster); return; } lock.readLock().lock(); 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 e6e45eebf8d24..e69165fe9495c 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 @@ -43,6 +43,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiFunction; +import java.util.function.Supplier; import lombok.AllArgsConstructor; import lombok.Data; import lombok.SneakyThrows; @@ -55,7 +56,9 @@ import org.apache.pulsar.broker.service.persistent.GeoPersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.admin.PulsarAdmin; 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.ProducerBuilder; @@ -76,6 +79,7 @@ import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Slf4j @@ -778,4 +782,102 @@ public void testExpandTopicPartitionsOnNamespaceLevelReplication() throws Except admin2.topics().deletePartitionedTopic(topicName, false); }); } + + private String getTheLatestMessage(String topic, PulsarClient client, PulsarAdmin admin) throws Exception { + String dummySubscription = "s_" + UUID.randomUUID().toString().replace("-", ""); + admin.topics().createSubscription(topic, dummySubscription, MessageId.earliest); + Consumer c = client.newConsumer(Schema.STRING).topic(topic).subscriptionName(dummySubscription) + .subscribe(); + String lastMsgValue = null; + while (true) { + Message msg = c.receive(2, TimeUnit.SECONDS); + if (msg == null) { + break; + } + lastMsgValue = msg.getValue(); + } + c.unsubscribe(); + return lastMsgValue; + } + + enum ReplicationLevel { + TOPIC_LEVEL, + NAMESPACE_LEVEL; + } + + @DataProvider(name = "replicationLevels") + public Object[][] replicationLevels() { + return new Object[][]{ + {ReplicationLevel.TOPIC_LEVEL}, + {ReplicationLevel.NAMESPACE_LEVEL} + }; + } + + @Test(dataProvider = "replicationLevels") + public void testReloadWithTopicLevelGeoReplication(ReplicationLevel replicationLevel) throws Exception { + final String topicName = ((Supplier) () -> { + if (replicationLevel.equals(ReplicationLevel.TOPIC_LEVEL)) { + return BrokerTestUtil.newUniqueName("persistent://" + nonReplicatedNamespace + "/tp_"); + } else { + return BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); + } + }).get(); + admin1.topics().createNonPartitionedTopic(topicName); + admin2.topics().createNonPartitionedTopic(topicName); + admin2.topics().createSubscription(topicName, "s1", MessageId.earliest); + if (replicationLevel.equals(ReplicationLevel.TOPIC_LEVEL)) { + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1, cluster2)); + } else { + pulsar1.getConfig().setTopicLevelPoliciesEnabled(false); + } + verifyReplicationWorks(topicName); + + /** + * Verify: + * 1. Inject an error to make the replicator is not able to work. + * 2. Send one message, since the replicator does not work anymore, this message will not be replicated. + * 3. Unload topic, the replicator will be re-created. + * 4. Verify: the message can be replicated to the remote cluster. + */ + // Step 1: Inject an error to make the replicator is not able to work. + Replicator replicator = broker1.getTopic(topicName, false).join().get().getReplicators().get(cluster2); + replicator.terminate(); + + // Step 2: Send one message, since the replicator does not work anymore, this message will not be replicated. + String msg = UUID.randomUUID().toString(); + Producer p1 = client1.newProducer(Schema.STRING).topic(topicName).create(); + p1.send(msg); + p1.close(); + // The result of "peek message" will be the messages generated, so it is not the same as the message just sent. + Thread.sleep(3000); + assertNotEquals(getTheLatestMessage(topicName, client2, admin2), msg); + assertEquals(admin1.topics().getStats(topicName).getReplication().get(cluster2).getReplicationBacklog(), 1); + + // Step 3: Unload topic, the replicator will be re-created. + admin1.topics().unload(topicName); + + // Step 4. Verify: the message can be replicated to the remote cluster. + Awaitility.await().atMost(Duration.ofSeconds(300)).untilAsserted(() -> { + log.info("replication backlog: {}", + admin1.topics().getStats(topicName).getReplication().get(cluster2).getReplicationBacklog()); + assertEquals(admin1.topics().getStats(topicName).getReplication().get(cluster2).getReplicationBacklog(), 0); + assertEquals(getTheLatestMessage(topicName, client2, admin2), msg); + }); + + // Cleanup. + if (replicationLevel.equals(ReplicationLevel.TOPIC_LEVEL)) { + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1)); + Awaitility.await().untilAsserted(() -> { + assertEquals(broker1.getTopic(topicName, false).join().get().getReplicators().size(), 0); + }); + admin1.topics().delete(topicName, false); + admin2.topics().delete(topicName, false); + } else { + pulsar1.getConfig().setTopicLevelPoliciesEnabled(true); + cleanupTopics(() -> { + admin1.topics().delete(topicName); + admin2.topics().delete(topicName); + }); + } + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java index 7372b2e478475..ffe6147412e56 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java @@ -350,6 +350,28 @@ protected PulsarClient initClient(ClientBuilder clientBuilder) throws Exception } protected void verifyReplicationWorks(String topic) throws Exception { + // Wait for replicator starting. + Awaitility.await().until(() -> { + try { + PersistentTopic persistentTopic = (PersistentTopic) pulsar1.getBrokerService() + .getTopic(topic, false).join().get(); + if (persistentTopic.getReplicators().size() > 0) { + return true; + } + } catch (Exception ex) {} + + try { + String partition0 = TopicName.get(topic).getPartition(0).toString(); + PersistentTopic persistentTopic = (PersistentTopic) pulsar1.getBrokerService() + .getTopic(partition0, false).join().get(); + if (persistentTopic.getReplicators().size() > 0) { + return true; + } + } catch (Exception ex) {} + + return false; + }); + // Verify: pub & sub. final String subscription = "__subscribe_1"; final String msgValue = "__msg1"; Producer producer1 = client1.newProducer(Schema.STRING).topic(topic).create(); 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 b4747a8bd0e47..b8f8edce2477e 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 @@ -104,4 +104,9 @@ public void testNoExpandTopicPartitionsWhenDisableTopicLevelReplication() throws public void testExpandTopicPartitionsOnNamespaceLevelReplication() throws Exception { super.testExpandTopicPartitionsOnNamespaceLevelReplication(); } + + @Test(enabled = false) + public void testReloadWithTopicLevelGeoReplication(ReplicationLevel replicationLevel) throws Exception { + super.testReloadWithTopicLevelGeoReplication(replicationLevel); + } } From 01a2e20a9f0c2d3b52a0eb8280d7ad77b369c9f5 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Jun 2024 09:26:28 +0300 Subject: [PATCH 565/699] [fix][fn] Enable optimized Netty direct byte buffer support for Pulsar Function runtimes (#22910) (cherry picked from commit f3d4d5ac0442eed2b538b8587186cdc0b8df9987) (cherry picked from commit 20de952a5e9b6a47e1069ed284e4cb055344b18a) --- .../functions/runtime/RuntimeUtils.java | 18 ++++++++-- .../kubernetes/KubernetesRuntimeTest.java | 36 ++++++++++--------- .../runtime/process/ProcessRuntimeTest.java | 16 +++++---- 3 files changed, 46 insertions(+), 24 deletions(-) diff --git a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/RuntimeUtils.java b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/RuntimeUtils.java index 0214b18fb2326..78347948688dd 100644 --- a/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/RuntimeUtils.java +++ b/pulsar-functions/runtime/src/main/java/org/apache/pulsar/functions/runtime/RuntimeUtils.java @@ -361,12 +361,26 @@ public static List getCmd(InstanceConfig instanceConfig, instanceConfig.getFunctionDetails().getName(), shardId)); + // Needed for optimized Netty direct byte buffer support args.add("-Dio.netty.tryReflectionSetAccessible=true"); + // Handle possible shaded Netty versions + args.add("-Dorg.apache.pulsar.shade.io.netty.tryReflectionSetAccessible=true"); + args.add("-Dio.grpc.netty.shaded.io.netty.tryReflectionSetAccessible=true"); + + if (SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_11)) { + // Needed for optimized Netty direct byte buffer support + args.add("--add-opens"); + args.add("java.base/java.nio=ALL-UNNAMED"); + args.add("--add-opens"); + args.add("java.base/jdk.internal.misc=ALL-UNNAMED"); + } - // Needed for netty.DnsResolverUtil on JDK9+ if (SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9)) { + // Needed for optimized checksum calculation when com.scurrilous.circe.checksum.Java9IntHash + // is used. That gets used when the native library libcirce-checksum is not available or cannot + // be loaded. args.add("--add-opens"); - args.add("java.base/sun.net=ALL-UNNAMED"); + args.add("java.base/java.util.zip=ALL-UNNAMED"); } if (instanceConfig.getAdditionalJavaRuntimeArguments() != null) { diff --git a/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/kubernetes/KubernetesRuntimeTest.java b/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/kubernetes/KubernetesRuntimeTest.java index 02f3c0d23fb17..6ed9849412910 100644 --- a/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/kubernetes/KubernetesRuntimeTest.java +++ b/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/kubernetes/KubernetesRuntimeTest.java @@ -441,14 +441,14 @@ private void verifyJavaInstance(InstanceConfig config, String depsDir, boolean s if (null != depsDir) { extraDepsEnv = " -Dpulsar.functions.extra.dependencies.dir=" + depsDir; classpath = classpath + ":" + depsDir + "/*"; - totalArgs = 46; - portArg = 33; - metricsPortArg = 35; + totalArgs = 52; + portArg = 39; + metricsPortArg = 41; } else { extraDepsEnv = ""; - portArg = 32; - metricsPortArg = 34; - totalArgs = 45; + portArg = 38; + metricsPortArg = 40; + totalArgs = 51; } if (secretsAttached) { totalArgs += 4; @@ -479,7 +479,11 @@ private void verifyJavaInstance(InstanceConfig config, String depsDir, boolean s + "-Dpulsar.function.log.dir=" + logDirectory + "/" + FunctionCommon.getFullyQualifiedName(config.getFunctionDetails()) + " -Dpulsar.function.log.file=" + config.getFunctionDetails().getName() + "-$SHARD_ID" + " -Dio.netty.tryReflectionSetAccessible=true" - + " --add-opens java.base/sun.net=ALL-UNNAMED" + + " -Dorg.apache.pulsar.shade.io.netty.tryReflectionSetAccessible=true" + + " -Dio.grpc.netty.shaded.io.netty.tryReflectionSetAccessible=true" + + " --add-opens java.base/java.nio=ALL-UNNAMED" + + " --add-opens java.base/jdk.internal.misc=ALL-UNNAMED" + + " --add-opens java.base/java.util.zip=ALL-UNNAMED" + " -Xmx" + RESOURCES.getRam() + " org.apache.pulsar.functions.instance.JavaInstanceMain" + " --jar " + jarLocation @@ -1306,7 +1310,7 @@ private void assertMetricsPortConfigured(Map functionRuntimeFact .contains("--metrics_port 0")); } } - + @Test public void testDeleteStatefulSetWithTranslatedKubernetesLabelChars() throws Exception { InstanceConfig config = createJavaInstanceConfig(FunctionDetails.Runtime.JAVA, false); @@ -1315,22 +1319,22 @@ public void testDeleteStatefulSetWithTranslatedKubernetesLabelChars() throws Exc CoreV1Api coreApi = mock(CoreV1Api.class); AppsV1Api appsApi = mock(AppsV1Api.class); - + Call successfulCall = mock(Call.class); Response okResponse = mock(Response.class); when(okResponse.code()).thenReturn(HttpURLConnection.HTTP_OK); when(okResponse.isSuccessful()).thenReturn(true); when(okResponse.message()).thenReturn(""); when(successfulCall.execute()).thenReturn(okResponse); - + final String expectedFunctionNamePrefix = String.format("pf-%s-%s-%s", "c-tenant", "c-ns", "c-fn"); - + factory = createKubernetesRuntimeFactory(null, 10, 1.0, 1.0); factory.setCoreClient(coreApi); factory.setAppsClient(appsApi); ArgumentMatcher hasTranslatedFunctionName = (String t) -> t.startsWith(expectedFunctionNamePrefix); - + when(appsApi.deleteNamespacedStatefulSetCall( argThat(hasTranslatedFunctionName), anyString(), isNull(), isNull(), anyInt(), isNull(), anyString(), any(), isNull())).thenReturn(successfulCall); @@ -1342,14 +1346,14 @@ public void testDeleteStatefulSetWithTranslatedKubernetesLabelChars() throws Exc V1PodList podList = mock(V1PodList.class); when(podList.getItems()).thenReturn(Collections.emptyList()); - + String expectedLabels = String.format("tenant=%s,namespace=%s,name=%s", "c-tenant", "c-ns", "c-fn"); - + when(coreApi.listNamespacedPod(anyString(), isNull(), isNull(), isNull(), isNull(), eq(expectedLabels), isNull(), isNull(), isNull(), isNull(), isNull())).thenReturn(podList); - KubernetesRuntime kr = factory.createContainer(config, "/test/code", "code.yml", "/test/transforms", "transform.yml", Long.MIN_VALUE); + KubernetesRuntime kr = factory.createContainer(config, "/test/code", "code.yml", "/test/transforms", "transform.yml", Long.MIN_VALUE); kr.deleteStatefulSet(); - + verify(coreApi).listNamespacedPod(anyString(), isNull(), isNull(), isNull(), isNull(), eq(expectedLabels), isNull(), isNull(), isNull(), isNull(), isNull()); } diff --git a/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/process/ProcessRuntimeTest.java b/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/process/ProcessRuntimeTest.java index f63f24dc25624..365704ea0b4ed 100644 --- a/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/process/ProcessRuntimeTest.java +++ b/pulsar-functions/runtime/src/test/java/org/apache/pulsar/functions/runtime/process/ProcessRuntimeTest.java @@ -297,7 +297,7 @@ private void verifyJavaInstance(InstanceConfig config, Path depsDir, String webS String extraDepsEnv; int portArg; int metricsPortArg; - int totalArgCount = 48; + int totalArgCount = 54; if (webServiceUrl != null && config.isExposePulsarAdminClientEnabled()) { totalArgCount += 3; } @@ -305,13 +305,13 @@ private void verifyJavaInstance(InstanceConfig config, Path depsDir, String webS assertEquals(args.size(), totalArgCount); extraDepsEnv = " -Dpulsar.functions.extra.dependencies.dir=" + depsDir; classpath = classpath + ":" + depsDir + "/*"; - portArg = 31; - metricsPortArg = 33; + portArg = 37; + metricsPortArg = 39; } else { assertEquals(args.size(), totalArgCount-1); extraDepsEnv = ""; - portArg = 30; - metricsPortArg = 32; + portArg = 36; + metricsPortArg = 38; } if (webServiceUrl != null && config.isExposePulsarAdminClientEnabled()) { portArg += 3; @@ -328,7 +328,11 @@ private void verifyJavaInstance(InstanceConfig config, Path depsDir, String webS + "-Dpulsar.function.log.dir=" + logDirectory + "/functions/" + FunctionCommon.getFullyQualifiedName(config.getFunctionDetails()) + " -Dpulsar.function.log.file=" + config.getFunctionDetails().getName() + "-" + config.getInstanceId() + " -Dio.netty.tryReflectionSetAccessible=true" - + " --add-opens java.base/sun.net=ALL-UNNAMED" + + " -Dorg.apache.pulsar.shade.io.netty.tryReflectionSetAccessible=true" + + " -Dio.grpc.netty.shaded.io.netty.tryReflectionSetAccessible=true" + + " --add-opens java.base/java.nio=ALL-UNNAMED" + + " --add-opens java.base/jdk.internal.misc=ALL-UNNAMED" + + " --add-opens java.base/java.util.zip=ALL-UNNAMED" + " org.apache.pulsar.functions.instance.JavaInstanceMain" + " --jar " + userJarFile + " --transform_function_jar " + userJarFile From 4fc7561bcc4cb80b3b43efbaae082dd396421f4e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 17 Jun 2024 21:13:10 +0300 Subject: [PATCH 566/699] [fix][test] Fix TableViewBuilderImplTest NPE and infinite loop (#22924) (cherry picked from commit 2dc0d96fa0da696949414d86fb11a62beca7cb3f) (cherry picked from commit 9baf4b01ccc2553e5a59074cd5430dcb22d47d43) --- .../client/impl/TableViewBuilderImplTest.java | 33 ++++++++++++------- 1 file changed, 21 insertions(+), 12 deletions(-) diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TableViewBuilderImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TableViewBuilderImplTest.java index eee8ba4e8f41a..01353e47cd0cb 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TableViewBuilderImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TableViewBuilderImplTest.java @@ -18,6 +18,14 @@ */ package org.apache.pulsar.client.impl; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertNotNull; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import org.apache.pulsar.client.api.ConsumerCryptoFailureAction; import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.PulsarClientException; @@ -25,32 +33,25 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.TableView; import org.apache.pulsar.client.impl.conf.ReaderConfigurationData; +import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; - -import static org.mockito.Mockito.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; -import static org.testng.Assert.assertNotNull; - /** - * Unit tests of {@link TablewViewBuilderImpl}. + * Unit tests of {@link TableViewBuilderImpl}. */ public class TableViewBuilderImplTest { private static final String TOPIC_NAME = "testTopicName"; private PulsarClientImpl client; private TableViewBuilderImpl tableViewBuilderImpl; + private CompletableFuture readNextFuture; @BeforeClass(alwaysRun = true) public void setup() { Reader reader = mock(Reader.class); - when(reader.readNextAsync()).thenReturn(CompletableFuture.allOf()); + readNextFuture = new CompletableFuture(); + when(reader.readNextAsync()).thenReturn(readNextFuture); client = mock(PulsarClientImpl.class); ConnectionPool connectionPool = mock(ConnectionPool.class); when(client.getCnxPool()).thenReturn(connectionPool); @@ -61,6 +62,14 @@ public void setup() { tableViewBuilderImpl = new TableViewBuilderImpl(client, Schema.BYTES); } + @AfterClass(alwaysRun = true) + public void cleanup() { + if (readNextFuture != null) { + readNextFuture.completeExceptionally(new PulsarClientException.AlreadyClosedException("Closing test case")); + readNextFuture = null; + } + } + @Test public void testTableViewBuilderImpl() throws PulsarClientException { TableView tableView = tableViewBuilderImpl.topic(TOPIC_NAME) From 1dc870c844d253cec52616e31d54df972bfff964 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 18 Jun 2024 14:33:33 +0800 Subject: [PATCH 567/699] [fix] [client] Fix resource leak in Pulsar Client since HttpLookupService doesn't get closed (#22858) (cherry picked from commit bc3dc7727b132dd88aa84f6befef42ea0646ec50) (cherry picked from commit e9264a991bd724385008d66c7fa0c356ce839d36) --- .../PulsarClientImplMultiBrokersTest.java | 79 +++++++++++++++++++ .../pulsar/client/impl/PulsarClientImpl.java | 22 ++++++ 2 files changed, 101 insertions(+) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PulsarClientImplMultiBrokersTest.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PulsarClientImplMultiBrokersTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PulsarClientImplMultiBrokersTest.java new file mode 100644 index 0000000000000..29604d0440b05 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/PulsarClientImplMultiBrokersTest.java @@ -0,0 +1,79 @@ +/* + * 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.testng.Assert.fail; +import static org.testng.AssertJUnit.assertEquals; +import static org.testng.AssertJUnit.assertTrue; +import java.util.Map; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.MultiBrokerBaseTest; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.client.impl.LookupService; +import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.awaitility.reflect.WhiteboxImpl; +import org.testng.annotations.Test; + +/** + * Test multi-broker admin api. + */ +@Slf4j +@Test(groups = "broker-admin") +public class PulsarClientImplMultiBrokersTest extends MultiBrokerBaseTest { + @Override + protected int numberOfAdditionalBrokers() { + return 3; + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + this.conf.setManagedLedgerMaxEntriesPerLedger(10); + } + + @Override + protected void onCleanup() { + super.onCleanup(); + } + + @Test(timeOut = 30 * 1000) + public void testReleaseUrlLookupServices() throws Exception { + PulsarClientImpl pulsarClient = (PulsarClientImpl) additionalBrokerClients.get(0); + Map urlLookupMap = WhiteboxImpl.getInternalState(pulsarClient, "urlLookupMap"); + assertEquals(urlLookupMap.size(), 0); + for (PulsarService pulsar : additionalBrokers) { + pulsarClient.getLookup(pulsar.getBrokerServiceUrl()); + pulsarClient.getLookup(pulsar.getWebServiceAddress()); + } + assertEquals(urlLookupMap.size(), additionalBrokers.size() * 2); + // Verify: lookup services will be release. + pulsarClient.close(); + assertEquals(urlLookupMap.size(), 0); + try { + for (PulsarService pulsar : additionalBrokers) { + pulsarClient.getLookup(pulsar.getBrokerServiceUrl()); + pulsarClient.getLookup(pulsar.getWebServiceAddress()); + } + fail("Expected a error when calling pulsarClient.getLookup if getLookup was closed"); + } catch (IllegalStateException illegalArgumentException) { + assertTrue(illegalArgumentException.getMessage().contains("has been closed")); + } + assertEquals(urlLookupMap.size(), 0); + } +} 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 c4c2268270fe5..899e4e8fae7c7 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 @@ -32,6 +32,7 @@ import java.time.Duration; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -735,6 +736,21 @@ public void close() throws PulsarClientException { } } + private void closeUrlLookupMap() { + Map closedUrlLookupServices = new HashMap(urlLookupMap.size()); + urlLookupMap.entrySet().forEach(e -> { + try { + e.getValue().close(); + } catch (Exception ex) { + log.error("Error closing lookup service {}", e.getKey(), ex); + } + closedUrlLookupServices.put(e.getKey(), e.getValue()); + }); + closedUrlLookupServices.entrySet().forEach(e -> { + urlLookupMap.remove(e.getKey(), e.getValue()); + }); + } + @Override public CompletableFuture closeAsync() { log.info("Client closing. URL: {}", lookup.getServiceUrl()); @@ -745,6 +761,8 @@ public CompletableFuture closeAsync() { final CompletableFuture closeFuture = new CompletableFuture<>(); List> futures = new ArrayList<>(); + closeUrlLookupMap(); + producers.forEach(p -> futures.add(p.closeAsync().handle((__, t) -> { if (t != null) { log.error("Error closing producer {}", p, t); @@ -972,6 +990,10 @@ public CompletableFuture getConnection(final String topic, final Stri public LookupService getLookup(String serviceUrl) { return urlLookupMap.computeIfAbsent(serviceUrl, url -> { + if (isClosed()) { + throw new IllegalStateException("Pulsar client has been closed, can not build LookupService when" + + " calling get lookup with an url"); + } try { return createLookup(serviceUrl); } catch (PulsarClientException e) { From 1610af1a3ebf0d71326fb868b7c9eacbd46819da Mon Sep 17 00:00:00 2001 From: yangyijun <1012293987@qq.com> Date: Thu, 20 Jun 2024 18:43:43 +0800 Subject: [PATCH 568/699] [fix] [broker] broker log a full thread dump when a deadlock is detected in healthcheck every time (#22916) (cherry picked from commit ca6450598469f158d8fa4cc942fb51e12ed1b609) (cherry picked from commit c9de1bbefd410152c7da6f9e8d5a5d59ab9cbead) --- .../java/org/apache/pulsar/broker/admin/impl/BrokersBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 eeb65590bec8a..9289cb3e039f9 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 @@ -88,7 +88,7 @@ public class BrokersBase extends AdminResource { private static final Duration HEALTH_CHECK_READ_TIMEOUT = Duration.ofSeconds(58); private static final TimeoutException HEALTH_CHECK_TIMEOUT_EXCEPTION = FutureUtil.createTimeoutException("Timeout", BrokersBase.class, "healthCheckRecursiveReadNext(...)"); - private volatile long threadDumpLoggedTimestamp; + private static volatile long threadDumpLoggedTimestamp; @GET @Path("/{cluster}") From 7d13506820447b1e3be95e711539c98f9313bdba Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 20 Jun 2024 15:47:25 +0300 Subject: [PATCH 569/699] [fix][fn] Support compression type and crypto config for all producers in Functions and Connectors (#22950) (cherry picked from commit ada47a327e08c1866c2b6f102c844e4c83fe93f3) (cherry picked from commit 8935fef2589d73a13007fea001fe993c318a174a) --- .../functions/instance/ContextImpl.java | 65 +++---- .../instance/ProducerBuilderFactory.java | 159 ++++++++++++++++ .../pulsar/functions/sink/PulsarSink.java | 158 +++------------- .../src/main/resources/findbugsExclude.xml | 7 +- .../functions/instance/ContextImplTest.java | 4 +- .../instance/ProducerBuilderFactoryTest.java | 178 ++++++++++++++++++ .../functions/utils/FunctionConfigUtils.java | 87 +++++---- .../functions/utils/SourceConfigUtils.java | 47 +---- .../utils/FunctionConfigUtilsTest.java | 35 ++++ 9 files changed, 480 insertions(+), 260 deletions(-) create mode 100644 pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ProducerBuilderFactory.java create mode 100644 pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ProducerBuilderFactoryTest.java diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java index d03f57e97205c..075e8bc9a764c 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java @@ -27,6 +27,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.LinkedList; import java.util.List; @@ -40,14 +41,11 @@ import lombok.ToString; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.client.api.BatcherBuilder; import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.CompressionType; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; -import org.apache.pulsar.client.api.HashingScheme; import org.apache.pulsar.client.api.MessageId; -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.PulsarClientException; @@ -55,7 +53,7 @@ import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.client.impl.MultiTopicsConsumerImpl; -import org.apache.pulsar.client.impl.ProducerBuilderImpl; +import org.apache.pulsar.common.functions.ProducerConfig; import org.apache.pulsar.common.io.SinkConfig; import org.apache.pulsar.common.io.SourceConfig; import org.apache.pulsar.common.naming.TopicName; @@ -77,6 +75,7 @@ import org.apache.pulsar.functions.source.PulsarFunctionRecord; import org.apache.pulsar.functions.source.TopicSchema; import org.apache.pulsar.functions.utils.FunctionCommon; +import org.apache.pulsar.functions.utils.FunctionConfigUtils; import org.apache.pulsar.functions.utils.SinkConfigUtils; import org.apache.pulsar.functions.utils.SourceConfigUtils; import org.apache.pulsar.io.core.SinkContext; @@ -88,6 +87,8 @@ */ @ToString(exclude = {"pulsarAdmin"}) class ContextImpl implements Context, SinkContext, SourceContext, AutoCloseable { + private final ProducerBuilderFactory producerBuilderFactory; + private final Map producerProperties; private InstanceConfig config; private Logger logger; @@ -99,7 +100,6 @@ class ContextImpl implements Context, SinkContext, SourceContext, AutoCloseable private final PulsarAdmin pulsarAdmin; private Map> publishProducers; private ThreadLocal>> tlPublishProducers; - private ProducerBuilderImpl producerBuilder; private final TopicSchema topicSchema; @@ -151,27 +151,27 @@ public ContextImpl(InstanceConfig config, Logger logger, PulsarClient client, this.topicSchema = new TopicSchema(client, Thread.currentThread().getContextClassLoader()); this.statsManager = statsManager; - this.producerBuilder = (ProducerBuilderImpl) client.newProducer().blockIfQueueFull(true).enableBatching(true) - .batchingMaxPublishDelay(1, TimeUnit.MILLISECONDS); boolean useThreadLocalProducers = false; + Function.ProducerSpec producerSpec = config.getFunctionDetails().getSink().getProducerSpec(); + ProducerConfig producerConfig = null; if (producerSpec != null) { - if (producerSpec.getMaxPendingMessages() != 0) { - this.producerBuilder.maxPendingMessages(producerSpec.getMaxPendingMessages()); - } - if (producerSpec.getMaxPendingMessagesAcrossPartitions() != 0) { - this.producerBuilder - .maxPendingMessagesAcrossPartitions(producerSpec.getMaxPendingMessagesAcrossPartitions()); - } - if (producerSpec.getBatchBuilder() != null) { - if (producerSpec.getBatchBuilder().equals("KEY_BASED")) { - this.producerBuilder.batcherBuilder(BatcherBuilder.KEY_BASED); - } else { - this.producerBuilder.batcherBuilder(BatcherBuilder.DEFAULT); - } - } + producerConfig = FunctionConfigUtils.convertProducerSpecToProducerConfig(producerSpec); useThreadLocalProducers = producerSpec.getUseThreadLocalProducers(); } + producerBuilderFactory = new ProducerBuilderFactory(client, producerConfig, + Thread.currentThread().getContextClassLoader(), + // This is for backwards compatibility. The PR https://github.com/apache/pulsar/pull/19470 removed + // the default and made it configurable for the producers created in PulsarSink, but not in ContextImpl. + // This is to keep the default unchanged for the producers created in ContextImpl. + producerBuilder -> producerBuilder.compressionType(CompressionType.LZ4)); + producerProperties = Collections.unmodifiableMap(InstanceUtils.getProperties(componentType, + FunctionCommon.getFullyQualifiedName( + this.config.getFunctionDetails().getTenant(), + this.config.getFunctionDetails().getNamespace(), + this.config.getFunctionDetails().getName()), + this.config.getInstanceId())); + if (useThreadLocalProducers) { tlPublishProducers = new ThreadLocal<>(); } else { @@ -548,26 +548,9 @@ private Producer getProducer(String topicName, Schema schema) throws P } if (producer == null) { - - Producer newProducer = ((ProducerBuilderImpl) producerBuilder.clone()) - .schema(schema) - .blockIfQueueFull(true) - .enableBatching(true) - .batchingMaxPublishDelay(10, TimeUnit.MILLISECONDS) - .compressionType(CompressionType.LZ4) - .hashingScheme(HashingScheme.Murmur3_32Hash) // - .messageRoutingMode(MessageRoutingMode.CustomPartition) - .messageRouter(FunctionResultRouter.of()) - // set send timeout to be infinity to prevent potential deadlock with consumer - // that might happen when consumer is blocked due to unacked messages - .sendTimeout(0, TimeUnit.SECONDS) - .topic(topicName) - .properties(InstanceUtils.getProperties(componentType, - FunctionCommon.getFullyQualifiedName( - this.config.getFunctionDetails().getTenant(), - this.config.getFunctionDetails().getNamespace(), - this.config.getFunctionDetails().getName()), - this.config.getInstanceId())) + Producer newProducer = producerBuilderFactory + .createProducerBuilder(topicName, schema, null) + .properties(producerProperties) .create(); if (tlPublishProducers != null) { diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ProducerBuilderFactory.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ProducerBuilderFactory.java new file mode 100644 index 0000000000000..b08f7f3f2cb0f --- /dev/null +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ProducerBuilderFactory.java @@ -0,0 +1,159 @@ +/* + * 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.apache.commons.lang.StringUtils.isEmpty; +import com.google.common.annotations.VisibleForTesting; +import java.security.Security; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; +import lombok.Builder; +import lombok.Data; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.client.api.BatcherBuilder; +import org.apache.pulsar.client.api.CompressionType; +import org.apache.pulsar.client.api.CryptoKeyReader; +import org.apache.pulsar.client.api.HashingScheme; +import org.apache.pulsar.client.api.MessageRoutingMode; +import org.apache.pulsar.client.api.ProducerBuilder; +import org.apache.pulsar.client.api.ProducerCryptoFailureAction; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.functions.CryptoConfig; +import org.apache.pulsar.common.functions.ProducerConfig; +import org.apache.pulsar.functions.utils.CryptoUtils; +import org.bouncycastle.jce.provider.BouncyCastleProvider; + +/** + * This class is responsible for creating ProducerBuilders with the appropriate configurations to + * match the ProducerConfig provided. Producers are created in 2 locations in Pulsar Functions and Connectors + * and this class is used to unify the configuration of the producers without duplicating code. + */ +@Slf4j +public class ProducerBuilderFactory { + + private final PulsarClient client; + private final ProducerConfig producerConfig; + private final Consumer> defaultConfigurer; + private final Crypto crypto; + + public ProducerBuilderFactory(PulsarClient client, ProducerConfig producerConfig, ClassLoader functionClassLoader, + Consumer> defaultConfigurer) { + this.client = client; + this.producerConfig = producerConfig; + this.defaultConfigurer = defaultConfigurer; + try { + this.crypto = initializeCrypto(functionClassLoader); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Unable to initialize crypto config " + producerConfig.getCryptoConfig(), e); + } + if (crypto == null) { + log.info("crypto key reader is not provided, not enabling end to end encryption"); + } + } + + public ProducerBuilder createProducerBuilder(String topic, Schema schema, String producerName) { + ProducerBuilder builder = client.newProducer(schema); + if (defaultConfigurer != null) { + defaultConfigurer.accept(builder); + } + builder.blockIfQueueFull(true) + .enableBatching(true) + .batchingMaxPublishDelay(10, TimeUnit.MILLISECONDS) + .hashingScheme(HashingScheme.Murmur3_32Hash) // + .messageRoutingMode(MessageRoutingMode.CustomPartition) + .messageRouter(FunctionResultRouter.of()) + // set send timeout to be infinity to prevent potential deadlock with consumer + // that might happen when consumer is blocked due to unacked messages + .sendTimeout(0, TimeUnit.SECONDS) + .topic(topic); + if (producerName != null) { + builder.producerName(producerName); + } + if (producerConfig != null) { + if (producerConfig.getCompressionType() != null) { + builder.compressionType(producerConfig.getCompressionType()); + } else { + // TODO: address this inconsistency. + // PR https://github.com/apache/pulsar/pull/19470 removed the default compression type of LZ4 + // from the top level. This default is only used if producer config is provided. + builder.compressionType(CompressionType.LZ4); + } + if (producerConfig.getMaxPendingMessages() != null && producerConfig.getMaxPendingMessages() != 0) { + builder.maxPendingMessages(producerConfig.getMaxPendingMessages()); + } + if (producerConfig.getMaxPendingMessagesAcrossPartitions() != null + && producerConfig.getMaxPendingMessagesAcrossPartitions() != 0) { + builder.maxPendingMessagesAcrossPartitions(producerConfig.getMaxPendingMessagesAcrossPartitions()); + } + if (producerConfig.getCryptoConfig() != null) { + builder.cryptoKeyReader(crypto.keyReader); + builder.cryptoFailureAction(crypto.failureAction); + for (String encryptionKeyName : crypto.getEncryptionKeys()) { + builder.addEncryptionKey(encryptionKeyName); + } + } + if (producerConfig.getBatchBuilder() != null) { + if (producerConfig.getBatchBuilder().equals("KEY_BASED")) { + builder.batcherBuilder(BatcherBuilder.KEY_BASED); + } else { + builder.batcherBuilder(BatcherBuilder.DEFAULT); + } + } + } + return builder; + } + + + @SuppressWarnings("unchecked") + @VisibleForTesting + Crypto initializeCrypto(ClassLoader functionClassLoader) throws ClassNotFoundException { + if (producerConfig == null + || producerConfig.getCryptoConfig() == null + || isEmpty(producerConfig.getCryptoConfig().getCryptoKeyReaderClassName())) { + return null; + } + + CryptoConfig cryptoConfig = producerConfig.getCryptoConfig(); + + // add provider only if it's not in the JVM + if (Security.getProvider(BouncyCastleProvider.PROVIDER_NAME) == null) { + Security.addProvider(new BouncyCastleProvider()); + } + + final String[] encryptionKeys = cryptoConfig.getEncryptionKeys(); + Crypto.CryptoBuilder bldr = Crypto.builder() + .failureAction(cryptoConfig.getProducerCryptoFailureAction()) + .encryptionKeys(encryptionKeys); + + bldr.keyReader(CryptoUtils.getCryptoKeyReaderInstance( + cryptoConfig.getCryptoKeyReaderClassName(), cryptoConfig.getCryptoKeyReaderConfig(), + functionClassLoader)); + + return bldr.build(); + } + + @Data + @Builder + private static class Crypto { + private CryptoKeyReader keyReader; + private ProducerCryptoFailureAction failureAction; + private String[] encryptionKeys; + } +} diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java index 97a0ad0a2ce17..18e55e8e84de1 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java @@ -18,10 +18,8 @@ */ package org.apache.pulsar.functions.sink; -import static org.apache.commons.lang.StringUtils.isEmpty; import com.google.common.annotations.VisibleForTesting; import java.nio.charset.StandardCharsets; -import java.security.Security; import java.util.ArrayList; import java.util.Base64; import java.util.List; @@ -29,21 +27,12 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.TimeUnit; import java.util.function.Function; -import lombok.Builder; -import lombok.Data; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; -import org.apache.pulsar.client.api.BatcherBuilder; -import org.apache.pulsar.client.api.CompressionType; -import org.apache.pulsar.client.api.CryptoKeyReader; -import org.apache.pulsar.client.api.HashingScheme; import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; -import org.apache.pulsar.client.api.ProducerCryptoFailureAction; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; @@ -52,22 +41,18 @@ import org.apache.pulsar.client.api.schema.KeyValueSchema; import org.apache.pulsar.client.impl.schema.AutoConsumeSchema; import org.apache.pulsar.common.functions.ConsumerConfig; -import org.apache.pulsar.common.functions.CryptoConfig; import org.apache.pulsar.common.functions.FunctionConfig; -import org.apache.pulsar.common.functions.ProducerConfig; import org.apache.pulsar.common.schema.KeyValueEncodingType; import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.common.util.Reflections; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.functions.instance.AbstractSinkRecord; -import org.apache.pulsar.functions.instance.FunctionResultRouter; +import org.apache.pulsar.functions.instance.ProducerBuilderFactory; import org.apache.pulsar.functions.instance.stats.ComponentStatsManager; import org.apache.pulsar.functions.source.PulsarRecord; import org.apache.pulsar.functions.source.TopicSchema; -import org.apache.pulsar.functions.utils.CryptoUtils; import org.apache.pulsar.io.core.Sink; import org.apache.pulsar.io.core.SinkContext; -import org.bouncycastle.jce.provider.BouncyCastleProvider; @Slf4j public class PulsarSink implements Sink { @@ -82,6 +67,8 @@ public class PulsarSink implements Sink { PulsarSinkProcessor pulsarSinkProcessor; private final TopicSchema topicSchema; + private Schema schema; + private ProducerBuilderFactory producerBuilderFactory; private interface PulsarSinkProcessor { @@ -94,60 +81,6 @@ private interface PulsarSinkProcessor { abstract class PulsarSinkProcessorBase implements PulsarSinkProcessor { protected Map> publishProducers = new ConcurrentHashMap<>(); - protected Schema schema; - protected Crypto crypto; - - protected PulsarSinkProcessorBase(Schema schema, Crypto crypto) { - this.schema = schema; - this.crypto = crypto; - } - - public Producer createProducer(PulsarClient client, String topic, String producerName, Schema schema) - throws PulsarClientException { - ProducerBuilder builder = client.newProducer(schema) - .blockIfQueueFull(true) - .enableBatching(true) - .batchingMaxPublishDelay(10, TimeUnit.MILLISECONDS) - .hashingScheme(HashingScheme.Murmur3_32Hash) // - .messageRoutingMode(MessageRoutingMode.CustomPartition) - .messageRouter(FunctionResultRouter.of()) - // set send timeout to be infinity to prevent potential deadlock with consumer - // that might happen when consumer is blocked due to unacked messages - .sendTimeout(0, TimeUnit.SECONDS) - .topic(topic); - if (producerName != null) { - builder.producerName(producerName); - } - if (pulsarSinkConfig.getProducerConfig() != null) { - ProducerConfig producerConfig = pulsarSinkConfig.getProducerConfig(); - if (producerConfig.getCompressionType() != null) { - builder.compressionType(producerConfig.getCompressionType()); - } else { - builder.compressionType(CompressionType.LZ4); - } - if (producerConfig.getMaxPendingMessages() != 0) { - builder.maxPendingMessages(producerConfig.getMaxPendingMessages()); - } - if (producerConfig.getMaxPendingMessagesAcrossPartitions() != 0) { - builder.maxPendingMessagesAcrossPartitions(producerConfig.getMaxPendingMessagesAcrossPartitions()); - } - if (producerConfig.getCryptoConfig() != null) { - builder.cryptoKeyReader(crypto.keyReader); - builder.cryptoFailureAction(crypto.failureAction); - for (String encryptionKeyName : crypto.getEncryptionKeys()) { - builder.addEncryptionKey(encryptionKeyName); - } - } - if (producerConfig.getBatchBuilder() != null) { - if (producerConfig.getBatchBuilder().equals("KEY_BASED")) { - builder.batcherBuilder(BatcherBuilder.KEY_BASED); - } else { - builder.batcherBuilder(BatcherBuilder.DEFAULT); - } - } - } - return builder.properties(properties).create(); - } protected Producer getProducer(String destinationTopic, Schema schema) { return getProducer(destinationTopic, null, destinationTopic, schema); @@ -159,10 +92,9 @@ protected Producer getProducer(String producerId, String producerName, String log.info("Initializing producer {} on topic {} with schema {}", producerName, topicName, schema); Producer producer = createProducer( - client, topicName, - producerName, - schema != null ? schema : this.schema); + schema, producerName + ); log.info("Initialized producer {} on topic {} with schema {}: {} -> {}", producerName, topicName, schema, producerId, producer); return producer; @@ -218,13 +150,12 @@ public Function getPublishErrorHandler(AbstractSinkRecord re @VisibleForTesting class PulsarSinkAtMostOnceProcessor extends PulsarSinkProcessorBase { - public PulsarSinkAtMostOnceProcessor(Schema schema, Crypto crypto) { - super(schema, crypto); + public PulsarSinkAtMostOnceProcessor() { if (!(schema instanceof AutoConsumeSchema)) { // initialize default topic try { publishProducers.put(pulsarSinkConfig.getTopic(), - createProducer(client, pulsarSinkConfig.getTopic(), null, schema)); + createProducer(pulsarSinkConfig.getTopic(), schema, null)); } catch (PulsarClientException e) { log.error("Failed to create Producer while doing user publish", e); throw new RuntimeException(e); @@ -270,10 +201,6 @@ public void sendOutputMessage(TypedMessageBuilder msg, AbstractSinkRecord @VisibleForTesting class PulsarSinkAtLeastOnceProcessor extends PulsarSinkAtMostOnceProcessor { - public PulsarSinkAtLeastOnceProcessor(Schema schema, Crypto crypto) { - super(schema, crypto); - } - @Override public void sendOutputMessage(TypedMessageBuilder msg, AbstractSinkRecord record) { msg.sendAsync() @@ -284,11 +211,6 @@ public void sendOutputMessage(TypedMessageBuilder msg, AbstractSinkRecord @VisibleForTesting class PulsarSinkManualProcessor extends PulsarSinkAtMostOnceProcessor { - - public PulsarSinkManualProcessor(Schema schema, Crypto crypto) { - super(schema, crypto); - } - @Override public void sendOutputMessage(TypedMessageBuilder msg, AbstractSinkRecord record) { super.sendOutputMessage(msg, record); @@ -297,11 +219,6 @@ public void sendOutputMessage(TypedMessageBuilder msg, AbstractSinkRecord @VisibleForTesting class PulsarSinkEffectivelyOnceProcessor extends PulsarSinkProcessorBase { - - public PulsarSinkEffectivelyOnceProcessor(Schema schema, Crypto crypto) { - super(schema, crypto); - } - @Override public TypedMessageBuilder newMessage(AbstractSinkRecord record) { if (!record.getPartitionId().isPresent()) { @@ -359,30 +276,27 @@ public PulsarSink(PulsarClient client, PulsarSinkConfig pulsarSinkConfig, Map config, SinkContext sinkContext) throws Exception { log.info("Opening pulsar sink with config: {}", pulsarSinkConfig); - Schema schema = initializeSchema(); + schema = initializeSchema(); if (schema == null) { log.info("Since output type is null, not creating any real sink"); return; } - - Crypto crypto = initializeCrypto(); - if (crypto == null) { - log.info("crypto key reader is not provided, not enabling end to end encryption"); - } + producerBuilderFactory = + new ProducerBuilderFactory(client, pulsarSinkConfig.getProducerConfig(), functionClassLoader, null); FunctionConfig.ProcessingGuarantees processingGuarantees = this.pulsarSinkConfig.getProcessingGuarantees(); switch (processingGuarantees) { case ATMOST_ONCE: - this.pulsarSinkProcessor = new PulsarSinkAtMostOnceProcessor(schema, crypto); + this.pulsarSinkProcessor = new PulsarSinkAtMostOnceProcessor(); break; case ATLEAST_ONCE: - this.pulsarSinkProcessor = new PulsarSinkAtLeastOnceProcessor(schema, crypto); + this.pulsarSinkProcessor = new PulsarSinkAtLeastOnceProcessor(); break; case EFFECTIVELY_ONCE: - this.pulsarSinkProcessor = new PulsarSinkEffectivelyOnceProcessor(schema, crypto); + this.pulsarSinkProcessor = new PulsarSinkEffectivelyOnceProcessor(); break; case MANUAL: - this.pulsarSinkProcessor = new PulsarSinkManualProcessor(schema, crypto); + this.pulsarSinkProcessor = new PulsarSinkManualProcessor(); break; } } @@ -427,6 +341,16 @@ public void close() throws Exception { } } + Producer createProducer(String topic, Schema schema, String producerName) + throws PulsarClientException { + ProducerBuilder builder = + producerBuilderFactory.createProducerBuilder(topic, schema != null ? schema : this.schema, + producerName); + return builder + .properties(properties) + .create(); + } + @SuppressWarnings("unchecked") @VisibleForTesting Schema initializeSchema() throws ClassNotFoundException { @@ -461,39 +385,5 @@ Schema initializeSchema() throws ClassNotFoundException { } } - @SuppressWarnings("unchecked") - @VisibleForTesting - Crypto initializeCrypto() throws ClassNotFoundException { - if (pulsarSinkConfig.getProducerConfig() == null - || pulsarSinkConfig.getProducerConfig().getCryptoConfig() == null - || isEmpty(pulsarSinkConfig.getProducerConfig().getCryptoConfig().getCryptoKeyReaderClassName())) { - return null; - } - - CryptoConfig cryptoConfig = pulsarSinkConfig.getProducerConfig().getCryptoConfig(); - - // add provider only if it's not in the JVM - if (Security.getProvider(BouncyCastleProvider.PROVIDER_NAME) == null) { - Security.addProvider(new BouncyCastleProvider()); - } - - final String[] encryptionKeys = cryptoConfig.getEncryptionKeys(); - Crypto.CryptoBuilder bldr = Crypto.builder() - .failureAction(cryptoConfig.getProducerCryptoFailureAction()) - .encryptionKeys(encryptionKeys); - bldr.keyReader(CryptoUtils.getCryptoKeyReaderInstance( - cryptoConfig.getCryptoKeyReaderClassName(), cryptoConfig.getCryptoKeyReaderConfig(), - functionClassLoader)); - - return bldr.build(); - } - - @Data - @Builder - private static class Crypto { - private CryptoKeyReader keyReader; - private ProducerCryptoFailureAction failureAction; - private String[] encryptionKeys; - } } diff --git a/pulsar-functions/instance/src/main/resources/findbugsExclude.xml b/pulsar-functions/instance/src/main/resources/findbugsExclude.xml index 7fe247d2ab20a..40e3e91112328 100644 --- a/pulsar-functions/instance/src/main/resources/findbugsExclude.xml +++ b/pulsar-functions/instance/src/main/resources/findbugsExclude.xml @@ -49,7 +49,12 @@ - + + + + + + diff --git a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java index 90f7df37fa196..108d8e4b66663 100644 --- a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java +++ b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java @@ -102,7 +102,9 @@ public void setup() throws PulsarClientException { client = mock(PulsarClientImpl.class); ConnectionPool connectionPool = mock(ConnectionPool.class); when(client.getCnxPool()).thenReturn(connectionPool); - when(client.newProducer()).thenReturn(new ProducerBuilderImpl(client, Schema.BYTES)); + when(client.newProducer()).thenAnswer(invocation -> new ProducerBuilderImpl(client, Schema.BYTES)); + when(client.newProducer(any())).thenAnswer( + invocation -> new ProducerBuilderImpl(client, invocation.getArgument(0))); when(client.createProducerAsync(any(ProducerConfigurationData.class), any(), any())) .thenReturn(CompletableFuture.completedFuture(producer)); when(client.getSchema(anyString())).thenReturn(CompletableFuture.completedFuture(Optional.empty())); diff --git a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ProducerBuilderFactoryTest.java b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ProducerBuilderFactoryTest.java new file mode 100644 index 0000000000000..42940f7e2dae3 --- /dev/null +++ b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ProducerBuilderFactoryTest.java @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.functions.instance; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.pulsar.client.api.BatcherBuilder; +import org.apache.pulsar.client.api.CompressionType; +import org.apache.pulsar.client.api.CryptoKeyReader; +import org.apache.pulsar.client.api.EncryptionKeyInfo; +import org.apache.pulsar.client.api.HashingScheme; +import org.apache.pulsar.client.api.MessageRoutingMode; +import org.apache.pulsar.client.api.ProducerBuilder; +import org.apache.pulsar.client.api.ProducerCryptoFailureAction; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.functions.CryptoConfig; +import org.apache.pulsar.common.functions.ProducerConfig; +import org.mockito.internal.util.MockUtil; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class ProducerBuilderFactoryTest { + private PulsarClient pulsarClient; + private ProducerBuilder producerBuilder; + + @BeforeMethod + public void setup() { + pulsarClient = mock(PulsarClient.class); + + producerBuilder = mock(ProducerBuilder.class); + doReturn(producerBuilder).when(producerBuilder).blockIfQueueFull(anyBoolean()); + doReturn(producerBuilder).when(producerBuilder).enableBatching(anyBoolean()); + doReturn(producerBuilder).when(producerBuilder).batchingMaxPublishDelay(anyLong(), any()); + doReturn(producerBuilder).when(producerBuilder).compressionType(any()); + doReturn(producerBuilder).when(producerBuilder).hashingScheme(any()); + doReturn(producerBuilder).when(producerBuilder).messageRoutingMode(any()); + doReturn(producerBuilder).when(producerBuilder).messageRouter(any()); + doReturn(producerBuilder).when(producerBuilder).topic(anyString()); + doReturn(producerBuilder).when(producerBuilder).producerName(anyString()); + doReturn(producerBuilder).when(producerBuilder).property(anyString(), anyString()); + doReturn(producerBuilder).when(producerBuilder).properties(any()); + doReturn(producerBuilder).when(producerBuilder).sendTimeout(anyInt(), any()); + + doReturn(producerBuilder).when(pulsarClient).newProducer(); + doReturn(producerBuilder).when(pulsarClient).newProducer(any()); + } + + @AfterMethod + public void tearDown() { + MockUtil.resetMock(pulsarClient); + pulsarClient = null; + MockUtil.resetMock(producerBuilder); + producerBuilder = null; + TestCryptoKeyReader.LAST_INSTANCE = null; + } + + @Test + public void testCreateProducerBuilder() { + ProducerBuilderFactory builderFactory = new ProducerBuilderFactory(pulsarClient, null, null, null); + builderFactory.createProducerBuilder("topic", Schema.STRING, "producerName"); + verifyCommon(); + verifyNoMoreInteractions(producerBuilder); + } + + private void verifyCommon() { + verify(pulsarClient).newProducer(Schema.STRING); + verify(producerBuilder).blockIfQueueFull(true); + verify(producerBuilder).enableBatching(true); + verify(producerBuilder).batchingMaxPublishDelay(10, TimeUnit.MILLISECONDS); + verify(producerBuilder).hashingScheme(HashingScheme.Murmur3_32Hash); + verify(producerBuilder).messageRoutingMode(MessageRoutingMode.CustomPartition); + verify(producerBuilder).messageRouter(FunctionResultRouter.of()); + verify(producerBuilder).sendTimeout(0, TimeUnit.SECONDS); + verify(producerBuilder).topic("topic"); + verify(producerBuilder).producerName("producerName"); + } + + @Test + public void testCreateProducerBuilderWithDefaultConfigurer() { + ProducerBuilderFactory builderFactory = new ProducerBuilderFactory(pulsarClient, null, null, + builder -> builder.property("key", "value")); + builderFactory.createProducerBuilder("topic", Schema.STRING, "producerName"); + verifyCommon(); + verify(producerBuilder).property("key", "value"); + verifyNoMoreInteractions(producerBuilder); + } + + @Test + public void testCreateProducerBuilderWithSimpleProducerConfig() { + ProducerConfig producerConfig = new ProducerConfig(); + producerConfig.setBatchBuilder("KEY_BASED"); + ProducerBuilderFactory builderFactory = new ProducerBuilderFactory(pulsarClient, producerConfig, null, null); + builderFactory.createProducerBuilder("topic", Schema.STRING, "producerName"); + verifyCommon(); + verify(producerBuilder).compressionType(CompressionType.LZ4); + verify(producerBuilder).batcherBuilder(BatcherBuilder.KEY_BASED); + verifyNoMoreInteractions(producerBuilder); + } + + @Test + public void testCreateProducerBuilderWithAdvancedProducerConfig() { + ProducerConfig producerConfig = new ProducerConfig(); + producerConfig.setBatchBuilder("KEY_BASED"); + producerConfig.setCompressionType(CompressionType.SNAPPY); + producerConfig.setMaxPendingMessages(5000); + producerConfig.setMaxPendingMessagesAcrossPartitions(50000); + CryptoConfig cryptoConfig = new CryptoConfig(); + cryptoConfig.setProducerCryptoFailureAction(ProducerCryptoFailureAction.FAIL); + cryptoConfig.setEncryptionKeys(new String[]{"key1", "key2"}); + cryptoConfig.setCryptoKeyReaderConfig(Map.of("key", "value")); + cryptoConfig.setCryptoKeyReaderClassName(TestCryptoKeyReader.class.getName()); + producerConfig.setCryptoConfig(cryptoConfig); + ProducerBuilderFactory builderFactory = new ProducerBuilderFactory(pulsarClient, producerConfig, null, null); + builderFactory.createProducerBuilder("topic", Schema.STRING, "producerName"); + verifyCommon(); + verify(producerBuilder).compressionType(CompressionType.SNAPPY); + verify(producerBuilder).batcherBuilder(BatcherBuilder.KEY_BASED); + verify(producerBuilder).maxPendingMessages(5000); + verify(producerBuilder).maxPendingMessagesAcrossPartitions(50000); + TestCryptoKeyReader lastInstance = TestCryptoKeyReader.LAST_INSTANCE; + assertNotNull(lastInstance); + assertEquals(lastInstance.configs, cryptoConfig.getCryptoKeyReaderConfig()); + verify(producerBuilder).cryptoKeyReader(lastInstance); + verify(producerBuilder).cryptoFailureAction(ProducerCryptoFailureAction.FAIL); + verify(producerBuilder).addEncryptionKey("key1"); + verify(producerBuilder).addEncryptionKey("key2"); + verifyNoMoreInteractions(producerBuilder); + } + + public static class TestCryptoKeyReader implements CryptoKeyReader { + static TestCryptoKeyReader LAST_INSTANCE; + Map configs; + public TestCryptoKeyReader(Map configs) { + this.configs = configs; + assert LAST_INSTANCE == null; + LAST_INSTANCE = this; + } + + @Override + public EncryptionKeyInfo getPublicKey(String keyName, Map metadata) { + throw new UnsupportedOperationException(); + } + + @Override + public EncryptionKeyInfo getPrivateKey(String keyName, Map metadata) { + throw new UnsupportedOperationException(); + } + } +} \ No newline at end of file diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java index ee59317daf755..3c96837e4374e 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/FunctionConfigUtils.java @@ -250,29 +250,7 @@ public static FunctionDetails convert(FunctionConfig functionConfig, ExtractedFu sinkSpecBuilder.setTypeClassName(functionConfig.getOutputTypeClassName()); } if (functionConfig.getProducerConfig() != null) { - ProducerConfig producerConf = functionConfig.getProducerConfig(); - Function.ProducerSpec.Builder pbldr = Function.ProducerSpec.newBuilder(); - if (producerConf.getMaxPendingMessages() != null) { - pbldr.setMaxPendingMessages(producerConf.getMaxPendingMessages()); - } - if (producerConf.getMaxPendingMessagesAcrossPartitions() != null) { - pbldr.setMaxPendingMessagesAcrossPartitions(producerConf.getMaxPendingMessagesAcrossPartitions()); - } - if (producerConf.getUseThreadLocalProducers() != null) { - pbldr.setUseThreadLocalProducers(producerConf.getUseThreadLocalProducers()); - } - if (producerConf.getCryptoConfig() != null) { - pbldr.setCryptoSpec(CryptoUtils.convert(producerConf.getCryptoConfig())); - } - if (producerConf.getBatchBuilder() != null) { - pbldr.setBatchBuilder(producerConf.getBatchBuilder()); - } - if (producerConf.getCompressionType() != null) { - pbldr.setCompressionType(convertFromCompressionType(producerConf.getCompressionType())); - } else { - pbldr.setCompressionType(Function.CompressionType.LZ4); - } - sinkSpecBuilder.setProducerSpec(pbldr.build()); + sinkSpecBuilder.setProducerSpec(convertProducerConfigToProducerSpec(functionConfig.getProducerConfig())); } if (functionConfig.getBatchBuilder() != null) { Function.ProducerSpec.Builder builder = sinkSpecBuilder.getProducerSpec() != null @@ -463,23 +441,8 @@ public static FunctionConfig convertFromDetails(FunctionDetails functionDetails) functionConfig.setOutputSchemaType(functionDetails.getSink().getSchemaType()); } if (functionDetails.getSink().getProducerSpec() != null) { - Function.ProducerSpec spec = functionDetails.getSink().getProducerSpec(); - ProducerConfig producerConfig = new ProducerConfig(); - if (spec.getMaxPendingMessages() != 0) { - producerConfig.setMaxPendingMessages(spec.getMaxPendingMessages()); - } - if (spec.getMaxPendingMessagesAcrossPartitions() != 0) { - producerConfig.setMaxPendingMessagesAcrossPartitions(spec.getMaxPendingMessagesAcrossPartitions()); - } - if (spec.hasCryptoSpec()) { - producerConfig.setCryptoConfig(CryptoUtils.convertFromSpec(spec.getCryptoSpec())); - } - if (spec.getBatchBuilder() != null) { - producerConfig.setBatchBuilder(spec.getBatchBuilder()); - } - producerConfig.setUseThreadLocalProducers(spec.getUseThreadLocalProducers()); - producerConfig.setCompressionType(convertFromFunctionDetailsCompressionType(spec.getCompressionType())); - functionConfig.setProducerConfig(producerConfig); + functionConfig.setProducerConfig( + convertProducerSpecToProducerConfig(functionDetails.getSink().getProducerSpec())); } if (!isEmpty(functionDetails.getLogTopic())) { functionConfig.setLogTopic(functionDetails.getLogTopic()); @@ -544,6 +507,50 @@ public static FunctionConfig convertFromDetails(FunctionDetails functionDetails) return functionConfig; } + public static Function.ProducerSpec convertProducerConfigToProducerSpec(ProducerConfig producerConf) { + Function.ProducerSpec.Builder builder = Function.ProducerSpec.newBuilder(); + if (producerConf.getMaxPendingMessages() != null) { + builder.setMaxPendingMessages(producerConf.getMaxPendingMessages()); + } + if (producerConf.getMaxPendingMessagesAcrossPartitions() != null) { + builder.setMaxPendingMessagesAcrossPartitions(producerConf.getMaxPendingMessagesAcrossPartitions()); + } + if (producerConf.getUseThreadLocalProducers() != null) { + builder.setUseThreadLocalProducers(producerConf.getUseThreadLocalProducers()); + } + if (producerConf.getCryptoConfig() != null) { + builder.setCryptoSpec(CryptoUtils.convert(producerConf.getCryptoConfig())); + } + if (producerConf.getBatchBuilder() != null) { + builder.setBatchBuilder(producerConf.getBatchBuilder()); + } + if (producerConf.getCompressionType() != null) { + builder.setCompressionType(convertFromCompressionType(producerConf.getCompressionType())); + } else { + builder.setCompressionType(Function.CompressionType.LZ4); + } + return builder.build(); + } + + public static ProducerConfig convertProducerSpecToProducerConfig(Function.ProducerSpec spec) { + ProducerConfig producerConfig = new ProducerConfig(); + if (spec.getMaxPendingMessages() != 0) { + producerConfig.setMaxPendingMessages(spec.getMaxPendingMessages()); + } + if (spec.getMaxPendingMessagesAcrossPartitions() != 0) { + producerConfig.setMaxPendingMessagesAcrossPartitions(spec.getMaxPendingMessagesAcrossPartitions()); + } + if (spec.hasCryptoSpec()) { + producerConfig.setCryptoConfig(CryptoUtils.convertFromSpec(spec.getCryptoSpec())); + } + if (spec.getBatchBuilder() != null) { + producerConfig.setBatchBuilder(spec.getBatchBuilder()); + } + producerConfig.setUseThreadLocalProducers(spec.getUseThreadLocalProducers()); + producerConfig.setCompressionType(convertFromFunctionDetailsCompressionType(spec.getCompressionType())); + return producerConfig; + } + public static void inferMissingArguments(FunctionConfig functionConfig, boolean forwardSourceMessagePropertyEnabled) { if (StringUtils.isEmpty(functionConfig.getName())) { diff --git a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SourceConfigUtils.java b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SourceConfigUtils.java index a6430bbea4585..2239f0fcbc2b7 100644 --- a/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SourceConfigUtils.java +++ b/pulsar-functions/utils/src/main/java/org/apache/pulsar/functions/utils/SourceConfigUtils.java @@ -19,10 +19,10 @@ package org.apache.pulsar.functions.utils; import static org.apache.commons.lang3.StringUtils.isEmpty; -import static org.apache.pulsar.functions.utils.FunctionCommon.convertFromCompressionType; -import static org.apache.pulsar.functions.utils.FunctionCommon.convertFromFunctionDetailsCompressionType; import static org.apache.pulsar.functions.utils.FunctionCommon.convertProcessingGuarantee; import static org.apache.pulsar.functions.utils.FunctionCommon.getSourceType; +import static org.apache.pulsar.functions.utils.FunctionConfigUtils.convertProducerConfigToProducerSpec; +import static org.apache.pulsar.functions.utils.FunctionConfigUtils.convertProducerSpecToProducerConfig; import com.fasterxml.jackson.core.type.TypeReference; import com.google.gson.Gson; import com.google.gson.reflect.TypeToken; @@ -39,7 +39,6 @@ import net.bytebuddy.description.type.TypeDescription; import net.bytebuddy.pool.TypePool; import org.apache.commons.lang3.StringUtils; -import org.apache.pulsar.common.functions.ProducerConfig; import org.apache.pulsar.common.functions.Resources; import org.apache.pulsar.common.io.BatchSourceConfig; import org.apache.pulsar.common.io.ConnectorDefinition; @@ -149,29 +148,7 @@ public static FunctionDetails convert(SourceConfig sourceConfig, ExtractedSource } if (sourceConfig.getProducerConfig() != null) { - ProducerConfig conf = sourceConfig.getProducerConfig(); - Function.ProducerSpec.Builder pbldr = Function.ProducerSpec.newBuilder(); - if (conf.getMaxPendingMessages() != null) { - pbldr.setMaxPendingMessages(conf.getMaxPendingMessages()); - } - if (conf.getMaxPendingMessagesAcrossPartitions() != null) { - pbldr.setMaxPendingMessagesAcrossPartitions(conf.getMaxPendingMessagesAcrossPartitions()); - } - if (conf.getUseThreadLocalProducers() != null) { - pbldr.setUseThreadLocalProducers(conf.getUseThreadLocalProducers()); - } - if (conf.getCryptoConfig() != null) { - pbldr.setCryptoSpec(CryptoUtils.convert(conf.getCryptoConfig())); - } - if (conf.getBatchBuilder() != null) { - pbldr.setBatchBuilder(conf.getBatchBuilder()); - } - if (conf.getCompressionType() != null) { - pbldr.setCompressionType(convertFromCompressionType(conf.getCompressionType())); - } else { - pbldr.setCompressionType(Function.CompressionType.LZ4); - } - sinkSpecBuilder.setProducerSpec(pbldr.build()); + sinkSpecBuilder.setProducerSpec(convertProducerConfigToProducerSpec(sourceConfig.getProducerConfig())); } if (sourceConfig.getBatchBuilder() != null) { @@ -256,23 +233,7 @@ public static SourceConfig convertFromDetails(FunctionDetails functionDetails) { sourceConfig.setSerdeClassName(sinkSpec.getSerDeClassName()); } if (sinkSpec.getProducerSpec() != null) { - Function.ProducerSpec spec = sinkSpec.getProducerSpec(); - ProducerConfig producerConfig = new ProducerConfig(); - if (spec.getMaxPendingMessages() != 0) { - producerConfig.setMaxPendingMessages(spec.getMaxPendingMessages()); - } - if (spec.getMaxPendingMessagesAcrossPartitions() != 0) { - producerConfig.setMaxPendingMessagesAcrossPartitions(spec.getMaxPendingMessagesAcrossPartitions()); - } - if (spec.hasCryptoSpec()) { - producerConfig.setCryptoConfig(CryptoUtils.convertFromSpec(spec.getCryptoSpec())); - } - if (spec.getBatchBuilder() != null) { - producerConfig.setBatchBuilder(spec.getBatchBuilder()); - } - producerConfig.setUseThreadLocalProducers(spec.getUseThreadLocalProducers()); - producerConfig.setCompressionType(convertFromFunctionDetailsCompressionType(spec.getCompressionType())); - sourceConfig.setProducerConfig(producerConfig); + sourceConfig.setProducerConfig(convertProducerSpecToProducerConfig(sinkSpec.getProducerSpec())); } if (functionDetails.hasResources()) { Resources resources = new Resources(); diff --git a/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java b/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java index 954eef44a7366..cf4e7dd92a8f7 100644 --- a/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java +++ b/pulsar-functions/utils/src/test/java/org/apache/pulsar/functions/utils/FunctionConfigUtilsTest.java @@ -35,9 +35,12 @@ import java.util.Map; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.client.api.CompressionType; +import org.apache.pulsar.client.api.ConsumerCryptoFailureAction; +import org.apache.pulsar.client.api.ProducerCryptoFailureAction; import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.impl.schema.JSONSchema; import org.apache.pulsar.common.functions.ConsumerConfig; +import org.apache.pulsar.common.functions.CryptoConfig; import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.functions.ProducerConfig; import org.apache.pulsar.common.functions.Resources; @@ -667,4 +670,36 @@ public void testPoolMessages() { convertedConfig = FunctionConfigUtils.convertFromDetails(functionDetails); assertTrue(convertedConfig.getInputSpecs().get("test-input").isPoolMessages()); } + + @Test + public void testConvertProducerSpecToProducerConfigAndBackToProducerSpec() { + // given + Function.ProducerSpec producerSpec = Function.ProducerSpec.newBuilder() + .setBatchBuilder("KEY_BASED") + .setCompressionType(Function.CompressionType.ZSTD) + .setCryptoSpec(Function.CryptoSpec.newBuilder() + .addProducerEncryptionKeyName("key1") + .addProducerEncryptionKeyName("key2") + .setConsumerCryptoFailureAction(Function.CryptoSpec.FailureAction.DISCARD) + .setProducerCryptoFailureAction(Function.CryptoSpec.FailureAction.SEND) + .setCryptoKeyReaderClassName("ReaderClassName") + .setCryptoKeyReaderConfig("{\"key\":\"value\"}") + .build()) + .build(); + // when + ProducerConfig producerConfig = FunctionConfigUtils.convertProducerSpecToProducerConfig(producerSpec); + // then + assertEquals(producerConfig.getBatchBuilder(), "KEY_BASED"); + assertEquals(producerConfig.getCompressionType(), CompressionType.ZSTD); + CryptoConfig cryptoConfig = producerConfig.getCryptoConfig(); + assertEquals(cryptoConfig.getProducerCryptoFailureAction(), ProducerCryptoFailureAction.SEND); + assertEquals(cryptoConfig.getConsumerCryptoFailureAction(), ConsumerCryptoFailureAction.DISCARD); + assertEquals(cryptoConfig.getEncryptionKeys(), new String[]{"key1", "key2"}); + assertEquals(cryptoConfig.getCryptoKeyReaderClassName(), "ReaderClassName"); + // and when + // converted back to producer spec + Function.ProducerSpec producerSpec2 = FunctionConfigUtils.convertProducerConfigToProducerSpec(producerConfig); + // then + assertEquals(producerSpec2, producerSpec); + } } From 8edc5abc9538fb94d3aae965c03c754dfd32d69f Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Thu, 20 Jun 2024 21:47:27 +0800 Subject: [PATCH 570/699] [fix][broker] Check the markDeletePosition and calculate the backlog (#22947) Signed-off-by: Zixuan Liu (cherry picked from commit 82b8d98a488191d279612d5cf2b4846627863543) (cherry picked from commit c0e1bff5ace138d1830be19480da149725bb6142) --- .../bookkeeper/mledger/impl/ManagedCursorImpl.java | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 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 52d4e803f0bb9..4391fa0cb5fd9 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 @@ -1112,6 +1112,13 @@ public long getEstimatedSizeSinceMarkDeletePosition() { return ledger.estimateBacklogFromPosition(markDeletePosition); } + private long getNumberOfEntriesInBacklog() { + if (markDeletePosition.compareTo(ledger.getLastPosition()) >= 0) { + return 0; + } + return getNumberOfEntries(Range.openClosed(markDeletePosition, ledger.getLastPosition())); + } + @Override public long getNumberOfEntriesInBacklog(boolean isPrecise) { if (log.isDebugEnabled()) { @@ -1120,16 +1127,13 @@ public long getNumberOfEntriesInBacklog(boolean isPrecise) { messagesConsumedCounter, markDeletePosition, readPosition); } if (isPrecise) { - if (markDeletePosition.compareTo(ledger.getLastPosition()) >= 0) { - return 0; - } - return getNumberOfEntries(Range.openClosed(markDeletePosition, ledger.getLastPosition())); + return getNumberOfEntriesInBacklog(); } long backlog = ManagedLedgerImpl.ENTRIES_ADDED_COUNTER_UPDATER.get(ledger) - messagesConsumedCounter; if (backlog < 0) { // In some case the counters get incorrect values, fall back to the precise backlog count - backlog = getNumberOfEntries(Range.openClosed(markDeletePosition, ledger.getLastPosition())); + backlog = getNumberOfEntriesInBacklog(); } return backlog; From f11db329584f352dab7c19d4fd8e4f98eeaaee4b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Thu, 20 Jun 2024 15:18:03 +0800 Subject: [PATCH 571/699] [improve][broker] Optimize `ConcurrentOpenLongPairRangeSet` by RoaringBitmap (#22908) (cherry picked from commit 5b1f653e65ccd967fd9642e6d6959de4b1b01a63) (cherry picked from commit f99040db727bfadd879ffc4d552337f029ebbe6e) --- .../server/src/assemble/LICENSE.bin.txt | 3 +- .../shell/src/assemble/LICENSE.bin.txt | 2 + pom.xml | 2 +- pulsar-common/pom.xml | 5 + .../ConcurrentOpenLongPairRangeSet.java | 12 +- .../collections/ConcurrentRoaringBitSet.java | 439 ++++++++++++++++++ pulsar-sql/presto-distribution/LICENSE | 3 +- 7 files changed, 455 insertions(+), 11 deletions(-) create mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentRoaringBitSet.java diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 3f830e16c5895..1e7a6ba455ccf 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -505,8 +505,7 @@ The Apache Software License, Version 2.0 * RxJava - io.reactivex.rxjava3-rxjava-3.0.1.jar * RoaringBitmap - - org.roaringbitmap-RoaringBitmap-0.9.44.jar - - org.roaringbitmap-shims-0.9.44.jar + - org.roaringbitmap-RoaringBitmap-1.1.0.jar BSD 3-clause "New" or "Revised" License * Google auth library diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 71811b83f65c2..c6d6566bb9502 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -380,6 +380,8 @@ The Apache Software License, Version 2.0 - simpleclient_tracer_common-0.16.0.jar - simpleclient_tracer_otel-0.16.0.jar - simpleclient_tracer_otel_agent-0.16.0.jar + * RoaringBitmap + - RoaringBitmap-1.1.0.jar * Log4J - log4j-api-2.18.0.jar - log4j-core-2.18.0.jar diff --git a/pom.xml b/pom.xml index 31f0d67094ba8..237fecf85dee9 100644 --- a/pom.xml +++ b/pom.xml @@ -306,7 +306,7 @@ flexible messaging model and an intuitive client API. 1.3 0.4 9.1.0 - 0.9.44 + 1.1.0 1.6.1 6.4.0 diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 04ace01ceb5e5..e39dd288abf6e 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -244,6 +244,11 @@ awaitility test + + + org.roaringbitmap + RoaringBitmap + 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 72215d7296cc3..b5ad89d1695d4 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 @@ -29,6 +29,7 @@ import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.commons.lang.mutable.MutableInt; +import org.roaringbitmap.RoaringBitSet; /** * A Concurrent set comprising zero or more ranges of type {@link LongPair}. This can be alternative of @@ -44,7 +45,7 @@ public class ConcurrentOpenLongPairRangeSet> implements LongPairRangeSet { protected final NavigableMap rangeBitSetMap = new ConcurrentSkipListMap<>(); - private boolean threadSafe = true; + private final boolean threadSafe; private final int bitSetSize; private final LongPairConsumer consumer; @@ -95,9 +96,7 @@ public void addOpenClosed(long lowerKey, long lowerValueOpen, long upperKey, lon // (2) set 0th-index to upper-index in upperRange.getKey() if (isValid(upperKey, upperValue)) { BitSet rangeBitSet = rangeBitSetMap.computeIfAbsent(upperKey, (key) -> createNewBitSet()); - if (rangeBitSet != null) { - rangeBitSet.set(0, (int) upperValue + 1); - } + rangeBitSet.set(0, (int) upperValue + 1); } // No-op if values are not valid eg: if lower == LongPair.earliest or upper == LongPair.latest then nothing // to set @@ -414,7 +413,6 @@ private int getSafeEntry(long value) { } private BitSet createNewBitSet() { - return this.threadSafe ? new ConcurrentBitSet(bitSetSize) : new BitSet(bitSetSize); + return this.threadSafe ? new ConcurrentRoaringBitSet() : new RoaringBitSet(); } - -} +} \ No newline at end of file diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentRoaringBitSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentRoaringBitSet.java new file mode 100644 index 0000000000000..814e58400993b --- /dev/null +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentRoaringBitSet.java @@ -0,0 +1,439 @@ +/* + * 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 java.util.BitSet; +import java.util.concurrent.locks.StampedLock; +import java.util.stream.IntStream; +import org.roaringbitmap.RoaringBitSet; + +public class ConcurrentRoaringBitSet extends RoaringBitSet { + private final StampedLock rwLock = new StampedLock(); + + public ConcurrentRoaringBitSet() { + super(); + } + + @Override + public boolean get(int bitIndex) { + long stamp = rwLock.tryOptimisticRead(); + boolean isSet = super.get(bitIndex); + if (!rwLock.validate(stamp)) { + stamp = rwLock.readLock(); + try { + isSet = super.get(bitIndex); + } finally { + rwLock.unlockRead(stamp); + } + } + return isSet; + } + + @Override + public void set(int bitIndex) { + long stamp = rwLock.writeLock(); + try { + super.set(bitIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void clear(int bitIndex) { + long stamp = rwLock.writeLock(); + try { + super.clear(bitIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void set(int fromIndex, int toIndex) { + long stamp = rwLock.writeLock(); + try { + super.set(fromIndex, toIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void clear(int fromIndex, int toIndex) { + long stamp = rwLock.writeLock(); + try { + super.clear(fromIndex, toIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void clear() { + long stamp = rwLock.writeLock(); + try { + super.clear(); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public int nextSetBit(int fromIndex) { + long stamp = rwLock.tryOptimisticRead(); + int nextSetBit = super.nextSetBit(fromIndex); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + nextSetBit = super.nextSetBit(fromIndex); + } finally { + rwLock.unlockRead(stamp); + } + } + return nextSetBit; + } + + @Override + public int nextClearBit(int fromIndex) { + long stamp = rwLock.tryOptimisticRead(); + int nextClearBit = super.nextClearBit(fromIndex); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + nextClearBit = super.nextClearBit(fromIndex); + } finally { + rwLock.unlockRead(stamp); + } + } + return nextClearBit; + } + + @Override + public int previousSetBit(int fromIndex) { + long stamp = rwLock.tryOptimisticRead(); + int previousSetBit = super.previousSetBit(fromIndex); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + previousSetBit = super.previousSetBit(fromIndex); + } finally { + rwLock.unlockRead(stamp); + } + } + return previousSetBit; + } + + @Override + public int previousClearBit(int fromIndex) { + long stamp = rwLock.tryOptimisticRead(); + int previousClearBit = super.previousClearBit(fromIndex); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + previousClearBit = super.previousClearBit(fromIndex); + } finally { + rwLock.unlockRead(stamp); + } + } + return previousClearBit; + } + + @Override + public int length() { + long stamp = rwLock.tryOptimisticRead(); + int length = super.length(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + length = super.length(); + } finally { + rwLock.unlockRead(stamp); + } + } + return length; + } + + @Override + public boolean isEmpty() { + long stamp = rwLock.tryOptimisticRead(); + boolean isEmpty = super.isEmpty(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + isEmpty = super.isEmpty(); + } finally { + rwLock.unlockRead(stamp); + } + } + return isEmpty; + } + + @Override + public int cardinality() { + long stamp = rwLock.tryOptimisticRead(); + int cardinality = super.cardinality(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + cardinality = super.cardinality(); + } finally { + rwLock.unlockRead(stamp); + } + } + return cardinality; + } + + @Override + public int size() { + long stamp = rwLock.tryOptimisticRead(); + int size = super.size(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + size = super.size(); + } finally { + rwLock.unlockRead(stamp); + } + } + return size; + } + + @Override + public byte[] toByteArray() { + long stamp = rwLock.tryOptimisticRead(); + byte[] byteArray = super.toByteArray(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + byteArray = super.toByteArray(); + } finally { + rwLock.unlockRead(stamp); + } + } + return byteArray; + } + + @Override + public long[] toLongArray() { + long stamp = rwLock.tryOptimisticRead(); + long[] longArray = super.toLongArray(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + longArray = super.toLongArray(); + } finally { + rwLock.unlockRead(stamp); + } + } + return longArray; + } + + @Override + public void flip(int bitIndex) { + long stamp = rwLock.writeLock(); + try { + super.flip(bitIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void flip(int fromIndex, int toIndex) { + long stamp = rwLock.writeLock(); + try { + super.flip(fromIndex, toIndex); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void set(int bitIndex, boolean value) { + long stamp = rwLock.writeLock(); + try { + super.set(bitIndex, value); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void set(int fromIndex, int toIndex, boolean value) { + long stamp = rwLock.writeLock(); + try { + super.set(fromIndex, toIndex, value); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public BitSet get(int fromIndex, int toIndex) { + long stamp = rwLock.tryOptimisticRead(); + BitSet bitSet = super.get(fromIndex, toIndex); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + bitSet = super.get(fromIndex, toIndex); + } finally { + rwLock.unlockRead(stamp); + } + } + return bitSet; + } + + @Override + public boolean intersects(BitSet set) { + long stamp = rwLock.writeLock(); + try { + return super.intersects(set); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void and(BitSet set) { + long stamp = rwLock.writeLock(); + try { + super.and(set); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void or(BitSet set) { + long stamp = rwLock.writeLock(); + try { + super.or(set); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void xor(BitSet set) { + long stamp = rwLock.writeLock(); + try { + super.xor(set); + } finally { + rwLock.unlockWrite(stamp); + } + } + + @Override + public void andNot(BitSet set) { + long stamp = rwLock.writeLock(); + try { + super.andNot(set); + } finally { + rwLock.unlockWrite(stamp); + } + } + + /** + * Returns the clone of the internal wrapped {@code BitSet}. + * This won't be a clone of the {@code ConcurrentBitSet} object. + * + * @return a clone of the internal wrapped {@code BitSet} + */ + @Override + public Object clone() { + long stamp = rwLock.tryOptimisticRead(); + RoaringBitSet clone = (RoaringBitSet) super.clone(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + clone = (RoaringBitSet) super.clone(); + } finally { + rwLock.unlockRead(stamp); + } + } + return clone; + } + + @Override + public String toString() { + long stamp = rwLock.tryOptimisticRead(); + String str = super.toString(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + str = super.toString(); + } finally { + rwLock.unlockRead(stamp); + } + } + return str; + } + + /** + * This operation is not supported on {@code ConcurrentBitSet}. + */ + @Override + public IntStream stream() { + throw new UnsupportedOperationException("stream is not supported"); + } + + public boolean equals(final Object o) { + long stamp = rwLock.tryOptimisticRead(); + boolean isEqual = super.equals(o); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + isEqual = super.equals(o); + } finally { + rwLock.unlockRead(stamp); + } + } + return isEqual; + } + + public int hashCode() { + long stamp = rwLock.tryOptimisticRead(); + int hashCode = super.hashCode(); + if (!rwLock.validate(stamp)) { + // Fallback to read lock + stamp = rwLock.readLock(); + try { + hashCode = super.hashCode(); + } finally { + rwLock.unlockRead(stamp); + } + } + return hashCode; + } +} diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 90723dafab95f..7ca0a9fe76274 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -484,7 +484,8 @@ The Apache Software License, Version 2.0 - stream-2.9.5.jar * High Performance Primitive Collections for Java - hppc-0.9.1.jar - + * RoaringBitmap + - RoaringBitmap-1.1.0.jar Protocol Buffers License * Protocol Buffers From 59028a6c4b970ff72360e699f90c9d82bfb8187e Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 24 Jun 2024 15:44:09 +0300 Subject: [PATCH 572/699] [improve][misc][branch-3.2] Upgrade to Bookkeeper 4.16.6 (#22963) (cherry picked from commit 37f44c40d4ca0bb075333653f55032cdec5c168a) (cherry picked from commit 2a98aab0f7badf789352bf2d639c22c30ef39025) (cherry picked from commit 4c64eafa20c72f5d53508412a0ead1cd6801ebbf) --- .../server/src/assemble/LICENSE.bin.txt | 56 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 6 +- pom.xml | 2 +- pulsar-sql/presto-distribution/LICENSE | 30 +++++----- 4 files changed, 47 insertions(+), 47 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 1e7a6ba455ccf..6a9122e7e32a2 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -346,34 +346,34 @@ The Apache Software License, Version 2.0 - net.java.dev.jna-jna-jpms-5.12.1.jar - net.java.dev.jna-jna-platform-jpms-5.12.1.jar * BookKeeper - - org.apache.bookkeeper-bookkeeper-common-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-common-allocator-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-proto-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-server-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-tools-framework-4.16.5.jar - - org.apache.bookkeeper-circe-checksum-4.16.5.jar - - org.apache.bookkeeper-cpu-affinity-4.16.5.jar - - org.apache.bookkeeper-statelib-4.16.5.jar - - org.apache.bookkeeper-stream-storage-api-4.16.5.jar - - org.apache.bookkeeper-stream-storage-common-4.16.5.jar - - org.apache.bookkeeper-stream-storage-java-client-4.16.5.jar - - org.apache.bookkeeper-stream-storage-java-client-base-4.16.5.jar - - org.apache.bookkeeper-stream-storage-proto-4.16.5.jar - - org.apache.bookkeeper-stream-storage-server-4.16.5.jar - - org.apache.bookkeeper-stream-storage-service-api-4.16.5.jar - - org.apache.bookkeeper-stream-storage-service-impl-4.16.5.jar - - org.apache.bookkeeper.http-http-server-4.16.5.jar - - org.apache.bookkeeper.http-vertx-http-server-4.16.5.jar - - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.16.5.jar - - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.16.5.jar - - org.apache.distributedlog-distributedlog-common-4.16.5.jar - - org.apache.distributedlog-distributedlog-core-4.16.5-tests.jar - - org.apache.distributedlog-distributedlog-core-4.16.5.jar - - org.apache.distributedlog-distributedlog-protocol-4.16.5.jar - - org.apache.bookkeeper.stats-codahale-metrics-provider-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-slogger-api-4.16.5.jar - - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.16.5.jar - - org.apache.bookkeeper-native-io-4.16.5.jar + - org.apache.bookkeeper-bookkeeper-common-4.16.6.jar + - org.apache.bookkeeper-bookkeeper-common-allocator-4.16.6.jar + - org.apache.bookkeeper-bookkeeper-proto-4.16.6.jar + - org.apache.bookkeeper-bookkeeper-server-4.16.6.jar + - org.apache.bookkeeper-bookkeeper-tools-framework-4.16.6.jar + - org.apache.bookkeeper-circe-checksum-4.16.6.jar + - org.apache.bookkeeper-cpu-affinity-4.16.6.jar + - org.apache.bookkeeper-statelib-4.16.6.jar + - org.apache.bookkeeper-stream-storage-api-4.16.6.jar + - org.apache.bookkeeper-stream-storage-common-4.16.6.jar + - org.apache.bookkeeper-stream-storage-java-client-4.16.6.jar + - org.apache.bookkeeper-stream-storage-java-client-base-4.16.6.jar + - org.apache.bookkeeper-stream-storage-proto-4.16.6.jar + - org.apache.bookkeeper-stream-storage-server-4.16.6.jar + - org.apache.bookkeeper-stream-storage-service-api-4.16.6.jar + - org.apache.bookkeeper-stream-storage-service-impl-4.16.6.jar + - org.apache.bookkeeper.http-http-server-4.16.6.jar + - org.apache.bookkeeper.http-vertx-http-server-4.16.6.jar + - org.apache.bookkeeper.stats-bookkeeper-stats-api-4.16.6.jar + - org.apache.bookkeeper.stats-prometheus-metrics-provider-4.16.6.jar + - org.apache.distributedlog-distributedlog-common-4.16.6.jar + - org.apache.distributedlog-distributedlog-core-4.16.6-tests.jar + - org.apache.distributedlog-distributedlog-core-4.16.6.jar + - org.apache.distributedlog-distributedlog-protocol-4.16.6.jar + - org.apache.bookkeeper.stats-codahale-metrics-provider-4.16.6.jar + - org.apache.bookkeeper-bookkeeper-slogger-api-4.16.6.jar + - org.apache.bookkeeper-bookkeeper-slogger-slf4j-4.16.6.jar + - org.apache.bookkeeper-native-io-4.16.6.jar * Apache HTTP Client - org.apache.httpcomponents-httpclient-4.5.13.jar - org.apache.httpcomponents-httpcore-4.4.15.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index c6d6566bb9502..46400b122ad1d 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -389,9 +389,9 @@ The Apache Software License, Version 2.0 - log4j-web-2.18.0.jar * BookKeeper - - bookkeeper-common-allocator-4.16.5.jar - - cpu-affinity-4.16.5.jar - - circe-checksum-4.16.5.jar + - bookkeeper-common-allocator-4.16.6.jar + - cpu-affinity-4.16.6.jar + - circe-checksum-4.16.6.jar * AirCompressor - aircompressor-0.27.jar * AsyncHttpClient diff --git a/pom.xml b/pom.xml index 237fecf85dee9..292e02578d6f5 100644 --- a/pom.xml +++ b/pom.xml @@ -136,7 +136,7 @@ flexible messaging model and an intuitive client API. 1.26.0 - 4.16.5 + 4.16.6 3.9.2 1.5.0 1.10.0 diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 7ca0a9fe76274..c94adca4ca2de 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -432,21 +432,21 @@ The Apache Software License, Version 2.0 - async-http-client-2.12.1.jar - async-http-client-netty-utils-2.12.1.jar * Apache Bookkeeper - - bookkeeper-common-4.16.5.jar - - bookkeeper-common-allocator-4.16.5.jar - - bookkeeper-proto-4.16.5.jar - - bookkeeper-server-4.16.5.jar - - bookkeeper-stats-api-4.16.5.jar - - bookkeeper-tools-framework-4.16.5.jar - - circe-checksum-4.16.5.jar - - codahale-metrics-provider-4.16.5.jar - - cpu-affinity-4.16.5.jar - - http-server-4.16.5.jar - - prometheus-metrics-provider-4.16.5.jar - - codahale-metrics-provider-4.16.5.jar - - bookkeeper-slogger-api-4.16.5.jar - - bookkeeper-slogger-slf4j-4.16.5.jar - - native-io-4.16.5.jar + - bookkeeper-common-4.16.6.jar + - bookkeeper-common-allocator-4.16.6.jar + - bookkeeper-proto-4.16.6.jar + - bookkeeper-server-4.16.6.jar + - bookkeeper-stats-api-4.16.6.jar + - bookkeeper-tools-framework-4.16.6.jar + - circe-checksum-4.16.6.jar + - codahale-metrics-provider-4.16.6.jar + - cpu-affinity-4.16.6.jar + - http-server-4.16.6.jar + - prometheus-metrics-provider-4.16.6.jar + - codahale-metrics-provider-4.16.6.jar + - bookkeeper-slogger-api-4.16.6.jar + - bookkeeper-slogger-slf4j-4.16.6.jar + - native-io-4.16.6.jar * Apache Commons - commons-cli-1.5.0.jar - commons-codec-1.15.jar From d8ddba4af96449fb1f281630092b02135778a096 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 24 Jun 2024 18:24:17 +0300 Subject: [PATCH 573/699] [fix][ci] Replace removed macos-11 with macos-latest in GitHub Actions (#22965) (cherry picked from commit 10eeaccbc5f01e53603c625555abffa50d0dcb17) (cherry picked from commit 9853c20ea582fc7f46805961717e17e4409a17ce) --- .github/workflows/ci-maven-cache-update.yaml | 2 +- .github/workflows/pulsar-ci.yaml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/ci-maven-cache-update.yaml b/.github/workflows/ci-maven-cache-update.yaml index cb29f0237f335..bb2c6efa33c10 100644 --- a/.github/workflows/ci-maven-cache-update.yaml +++ b/.github/workflows/ci-maven-cache-update.yaml @@ -63,7 +63,7 @@ jobs: mvn_arguments: '' - name: all modules - macos - runs-on: macos-11 + runs-on: macos-latest cache_name: 'm2-dependencies-all' - name: core-modules diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 2ba58ba102bf9..783710b1029a2 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -1209,7 +1209,7 @@ jobs: macos-build: name: Build Pulsar on MacOS - runs-on: macos-11 + runs-on: macos-latest timeout-minutes: 120 needs: ['preconditions', 'integration-tests'] if: ${{ needs.preconditions.outputs.docs_only != 'true' }} From c4ab0b81c282cbbd8a94f9797febfe6c1a684d3d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 21 Jun 2024 23:06:56 +0300 Subject: [PATCH 574/699] [fix][misc] Rename netty native libraries in pulsar-client-admin-shaded (#22954) (cherry picked from commit ddb03bb6a3b67ffcc71c7e95a87b35eb302a7393) (cherry picked from commit a7d4e20bbaf898f28df57c5128d7e8f00e8771e9) --- pulsar-client-admin-shaded/pom.xml | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/pulsar-client-admin-shaded/pom.xml b/pulsar-client-admin-shaded/pom.xml index 6fe7bd3db0710..30452f4f9b8db 100644 --- a/pulsar-client-admin-shaded/pom.xml +++ b/pulsar-client-admin-shaded/pom.xml @@ -304,6 +304,31 @@ + + + + exec-maven-plugin + org.codehaus.mojo + + + rename-epoll-library + package + + exec + + + ${project.parent.basedir}/src/${rename.netty.native.libs} + + ${project.artifactId} + + + + + From 0f4c2086a838c21d769a2421a9b9d0254f702753 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 24 Jun 2024 19:54:27 +0300 Subject: [PATCH 575/699] [improve][misc] Replace rename-netty-native-libs.sh script with renaming with maven-shade-plugin (#22957) (cherry picked from commit f728b2ebb9bfe2dfe1f64643640700f762524c40) (cherry picked from commit 68d92fe937322671f90eaf8cc45f08843939dba9) --- README.md | 2 - pom.xml | 7 --- pulsar-client-admin-shaded/pom.xml | 31 ++-------- pulsar-client-all/pom.xml | 31 ++-------- pulsar-client-shaded/pom.xml | 31 ++-------- src/rename-netty-native-libs.cmd | 98 ------------------------------ src/rename-netty-native-libs.sh | 70 --------------------- 7 files changed, 18 insertions(+), 252 deletions(-) delete mode 100644 src/rename-netty-native-libs.cmd delete mode 100755 src/rename-netty-native-libs.sh diff --git a/README.md b/README.md index fdbf7c7339b1e..a9c4a2f5f4bcb 100644 --- a/README.md +++ b/README.md @@ -141,8 +141,6 @@ components in the Pulsar ecosystem, including connectors, adapters, and other la > > This project includes a [Maven Wrapper](https://maven.apache.org/wrapper/) that can be used instead of a system-installed Maven. > Use it by replacing `mvn` by `./mvnw` on Linux and `mvnw.cmd` on Windows in the commands below. -> -> It's better to use CMD rather than Powershell on Windows. Because maven will activate the `windows` profile which runs `rename-netty-native-libs.cmd`. ### Build diff --git a/pom.xml b/pom.xml index 292e02578d6f5..2d2fb8be00f4e 100644 --- a/pom.xml +++ b/pom.xml @@ -309,9 +309,6 @@ flexible messaging model and an intuitive client API. 1.1.0 1.6.1 6.4.0 - - - rename-netty-native-libs.sh @@ -2260,10 +2257,6 @@ flexible messaging model and an intuitive client API. Windows - - rename-netty-native-libs.cmd - - diff --git a/pulsar-client-admin-shaded/pom.xml b/pulsar-client-admin-shaded/pom.xml index 30452f4f9b8db..e2c082c648d7f 100644 --- a/pulsar-client-admin-shaded/pom.xml +++ b/pulsar-client-admin-shaded/pom.xml @@ -295,6 +295,12 @@ org.apache.bookkeeper org.apache.pulsar.shade.org.apache.bookkeeper + + + (META-INF/native/(lib)?)(netty.+\.(so|jnilib|dll))$ + $1org_apache_pulsar_shade_$3 + true + @@ -304,31 +310,6 @@ - - - - exec-maven-plugin - org.codehaus.mojo - - - rename-epoll-library - package - - exec - - - ${project.parent.basedir}/src/${rename.netty.native.libs} - - ${project.artifactId} - - - - - diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index c23da8ff99059..4bd7aee0c8461 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -387,6 +387,12 @@ org.tukaani org.apache.pulsar.shade.org.tukaani + + + (META-INF/native/(lib)?)(netty.+\.(so|jnilib|dll))$ + $1org_apache_pulsar_shade_$3 + true + @@ -396,31 +402,6 @@ - - - - exec-maven-plugin - org.codehaus.mojo - - - rename-epoll-library - package - - exec - - - ${project.parent.basedir}/src/${rename.netty.native.libs} - - ${project.artifactId} - - - - - org.apache.maven.plugins maven-enforcer-plugin diff --git a/pulsar-client-shaded/pom.xml b/pulsar-client-shaded/pom.xml index d9de319822e57..c044061906af6 100644 --- a/pulsar-client-shaded/pom.xml +++ b/pulsar-client-shaded/pom.xml @@ -300,6 +300,12 @@ org.apache.bookkeeper org.apache.pulsar.shade.org.apache.bookkeeper + + + (META-INF/native/(lib)?)(netty.+\.(so|jnilib|dll))$ + $1org_apache_pulsar_shade_$3 + true + @@ -323,31 +329,6 @@ - - - - exec-maven-plugin - org.codehaus.mojo - - - rename-epoll-library - package - - exec - - - ${project.parent.basedir}/src/${rename.netty.native.libs} - - ${project.artifactId} - - - - - diff --git a/src/rename-netty-native-libs.cmd b/src/rename-netty-native-libs.cmd deleted file mode 100644 index bfaa16de0812c..0000000000000 --- a/src/rename-netty-native-libs.cmd +++ /dev/null @@ -1,98 +0,0 @@ -@REM -@REM Licensed to the Apache Software Foundation (ASF) under one -@REM or more contributor license agreements. See the NOTICE file -@REM distributed with this work for additional information -@REM regarding copyright ownership. The ASF licenses this file -@REM to you under the Apache License, Version 2.0 (the -@REM "License"); you may not use this file except in compliance -@REM with the License. You may obtain a copy of the License at -@REM -@REM http://www.apache.org/licenses/LICENSE-2.0 -@REM -@REM Unless required by applicable law or agreed to in writing, -@REM software distributed under the License is distributed on an -@REM "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -@REM KIND, either express or implied. See the License for the -@REM specific language governing permissions and limitations -@REM under the License. -@REM - -@echo off - -set ARTIFACT_ID=%1 -set JAR_PATH=%cd%/target/%ARTIFACT_ID%.jar -set FILE_PREFIX=META-INF/native - -:: echo %JAR_PATH% -:: echo %FILE_PREFIX% - -ECHO. -echo ----- Renaming epoll lib in %JAR_PATH% ------ -set TMP_DIR=%temp%\tmp_pulsar - -rd %TMP_DIR% /s /q -mkdir %TMP_DIR% - -set UNZIP_CMD=unzip -q %JAR_PATH% -d %TMP_DIR% -call %UNZIP_CMD% - -:: echo %UNZIP_CMD% -:: echo %TMP_DIR% - -cd /d %TMP_DIR%/%FILE_PREFIX% - -:: Loop through the number of groups -SET Obj_Length=10 -SET Obj[0].FROM=libnetty_transport_native_epoll_x86_64.so -SET Obj[0].TO=liborg_apache_pulsar_shade_netty_transport_native_epoll_x86_64.so -SET Obj[1].FROM=libnetty_transport_native_epoll_aarch_64.so -SET Obj[1].TO=liborg_apache_pulsar_shade_netty_transport_native_epoll_aarch_64.so -SET Obj[2].FROM=libnetty_tcnative_linux_x86_64.so -SET Obj[2].TO=liborg_apache_pulsar_shade_netty_tcnative_linux_x86_64.so -SET Obj[3].FROM=libnetty_tcnative_linux_aarch_64.so -SET Obj[3].TO=liborg_apache_pulsar_shade_netty_tcnative_linux_aarch_64.so -SET Obj[4].FROM=libnetty_tcnative_osx_x86_64.jnilib -SET Obj[4].TO=liborg_apache_pulsar_shade_netty_tcnative_osx_x86_64.jnilib -SET Obj[5].FROM=libnetty_tcnative_osx_aarch_64.jnilib -SET Obj[5].TO=liborg_apache_pulsar_shade_netty_tcnative_osx_aarch_64.jnilib -SET Obj[6].FROM=libnetty_transport_native_io_uring_x86_64.so -SET Obj[6].TO=liborg_apache_pulsar_shade_netty_transport_native_io_uring_x86_64.so -SET Obj[7].FROM=libnetty_transport_native_io_uring_aarch_64.so -SET Obj[7].TO=liborg_apache_pulsar_shade_netty_transport_native_io_uring_aarch_64.so -SET Obj[8].FROM=libnetty_resolver_dns_native_macos_aarch_64.jnilib -SET Obj[8].TO=liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_aarch_64.jnilib -SET Obj[9].FROM=libnetty_resolver_dns_native_macos_x86_64.jnilib -SET Obj[9].TO=liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_x86_64.jnilib -SET Obj_Index=0 - -:LoopStart -IF %Obj_Index% EQU %Obj_Length% GOTO END - -SET Obj_Current.FROM=0 -SET Obj_Current.TO=0 - -FOR /F "usebackq delims==. tokens=1-3" %%I IN (`SET Obj[%Obj_Index%]`) DO ( - SET Obj_Current.%%J=%%K.so -) - -echo "Renaming %Obj_Current.FROM% -> %Obj_Current.TO%" -call ren %Obj_Current.FROM% %Obj_Current.TO% - -SET /A Obj_Index=%Obj_Index% + 1 - -GOTO LoopStart -:: Loop end - -:END -cd /d %TMP_DIR% - -:: Overwrite the original ZIP archive -rd %JAR_PATH% /s /q -set ZIP_CMD=zip -q -r %JAR_PATH% . -:: echo %ZIP_CMD% -call %ZIP_CMD% -:: echo %TMP_DIR% -rd %TMP_DIR% /s /q - -exit /b 0 -:: echo.&pause&goto:eof \ No newline at end of file diff --git a/src/rename-netty-native-libs.sh b/src/rename-netty-native-libs.sh deleted file mode 100755 index ea2a4c0e2421e..0000000000000 --- a/src/rename-netty-native-libs.sh +++ /dev/null @@ -1,70 +0,0 @@ -#!/usr/bin/env bash -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -# - -set -e - -ARTIFACT_ID=$1 -JAR_PATH="$PWD/target/$ARTIFACT_ID.jar" - -FILE_PREFIX='META-INF/native' - -FILES_TO_RENAME=( - 'libnetty_transport_native_epoll_x86_64.so liborg_apache_pulsar_shade_netty_transport_native_epoll_x86_64.so' - 'libnetty_transport_native_epoll_aarch_64.so liborg_apache_pulsar_shade_netty_transport_native_epoll_aarch_64.so' - 'libnetty_tcnative_linux_x86_64.so liborg_apache_pulsar_shade_netty_tcnative_linux_x86_64.so' - 'libnetty_tcnative_linux_aarch_64.so liborg_apache_pulsar_shade_netty_tcnative_linux_aarch_64.so' - 'libnetty_tcnative_osx_x86_64.jnilib liborg_apache_pulsar_shade_netty_tcnative_osx_x86_64.jnilib' - 'libnetty_tcnative_osx_aarch_64.jnilib liborg_apache_pulsar_shade_netty_tcnative_osx_aarch_64.jnilib' - 'libnetty_transport_native_io_uring_x86_64.so liborg_apache_pulsar_shade_netty_transport_native_io_uring_x86_64.so' - 'libnetty_transport_native_io_uring_aarch_64.so liborg_apache_pulsar_shade_netty_transport_native_io_uring_aarch_64.so' - 'libnetty_resolver_dns_native_macos_aarch_64.jnilib liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_aarch_64.jnilib' - 'libnetty_resolver_dns_native_macos_x86_64.jnilib liborg_apache_pulsar_shade_netty_resolver_dns_native_macos_x86_64.jnilib' -) - -echo "----- Renaming epoll lib in $JAR_PATH ------" -TMP_DIR=`mktemp -d` -CUR_DIR=$(pwd) -cd ${TMP_DIR} -# exclude `META-INF/LICENSE` -unzip -q $JAR_PATH -x "META-INF/LICENSE" -# include `META-INF/LICENSE` as LICENSE.netty. -# This approach is to get around the issue that MacOS is not able to recognize the difference between `META-INF/LICENSE` and `META-INF/license/`. -unzip -p $JAR_PATH META-INF/LICENSE > META-INF/LICENSE.netty -cd ${CUR_DIR} - -pushd $TMP_DIR - -for line in "${FILES_TO_RENAME[@]}"; do - read -r -a A <<< "$line" - FROM=${A[0]} - TO=${A[1]} - - if [ -f $FILE_PREFIX/$FROM ]; then - echo "Renaming $FROM -> $TO" - mv $FILE_PREFIX/$FROM $FILE_PREFIX/$TO - fi -done - -# Overwrite the original ZIP archive -rm $JAR_PATH -zip -q -r $JAR_PATH . -popd - -rm -rf $TMP_DIR From 717db84206c2422d83798312e7d0b6de06bf88ba Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 25 Jun 2024 08:37:43 +0300 Subject: [PATCH 576/699] [improve][fn] Make producer cache bounded and expiring in Functions/Connectors (#22945) (cherry picked from commit 6fe8100b1fd5d37a6e1bf33803a8904fa3879321) (cherry picked from commit f10708f985417e969c5a0fcd21a023eab9d6c487) (cherry picked from commit 9f5432aaf6dd3e6f629d697d5b852f39c6d137ae) --- pulsar-functions/instance/pom.xml | 5 + .../functions/instance/ContextImpl.java | 86 +++--------- .../instance/JavaInstanceRunnable.java | 8 +- .../functions/instance/ProducerCache.java | 130 +++++++++++++++++ .../pulsar/functions/sink/PulsarSink.java | 89 +++++------- .../functions/instance/ContextImplTest.java | 22 ++- .../pulsar/functions/sink/PulsarSinkTest.java | 132 ++++++++---------- 7 files changed, 267 insertions(+), 205 deletions(-) create mode 100644 pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ProducerCache.java diff --git a/pulsar-functions/instance/pom.xml b/pulsar-functions/instance/pom.xml index 1228869d3d646..dbbffe7794e6a 100644 --- a/pulsar-functions/instance/pom.xml +++ b/pulsar-functions/instance/pom.xml @@ -157,6 +157,11 @@ jcommander + + com.github.ben-manes.caffeine + caffeine + + net.jodah typetools diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java index 075e8bc9a764c..a70f4eaec7345 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ContextImpl.java @@ -29,16 +29,15 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.stream.Stream; import lombok.ToString; +import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.ClientBuilder; @@ -85,6 +84,7 @@ /** * This class implements the Context interface exposed to the user. */ +@Slf4j @ToString(exclude = {"pulsarAdmin"}) class ContextImpl implements Context, SinkContext, SourceContext, AutoCloseable { private final ProducerBuilderFactory producerBuilderFactory; @@ -98,8 +98,6 @@ class ContextImpl implements Context, SinkContext, SourceContext, AutoCloseable private final ClientBuilder clientBuilder; private final PulsarClient client; private final PulsarAdmin pulsarAdmin; - private Map> publishProducers; - private ThreadLocal>> tlPublishProducers; private final TopicSchema topicSchema; @@ -137,12 +135,15 @@ class ContextImpl implements Context, SinkContext, SourceContext, AutoCloseable private final Function.FunctionDetails.ComponentType componentType; + private final ProducerCache producerCache; + private final boolean useThreadLocalProducers; + public ContextImpl(InstanceConfig config, Logger logger, PulsarClient client, SecretsProvider secretsProvider, FunctionCollectorRegistry collectorRegistry, String[] metricsLabels, Function.FunctionDetails.ComponentType componentType, ComponentStatsManager statsManager, - StateManager stateManager, PulsarAdmin pulsarAdmin, ClientBuilder clientBuilder) - throws PulsarClientException { + StateManager stateManager, PulsarAdmin pulsarAdmin, ClientBuilder clientBuilder, + ProducerCache producerCache) throws PulsarClientException { this.config = config; this.logger = logger; this.clientBuilder = clientBuilder; @@ -151,14 +152,17 @@ public ContextImpl(InstanceConfig config, Logger logger, PulsarClient client, this.topicSchema = new TopicSchema(client, Thread.currentThread().getContextClassLoader()); this.statsManager = statsManager; - boolean useThreadLocalProducers = false; + this.producerCache = producerCache; Function.ProducerSpec producerSpec = config.getFunctionDetails().getSink().getProducerSpec(); ProducerConfig producerConfig = null; if (producerSpec != null) { producerConfig = FunctionConfigUtils.convertProducerSpecToProducerConfig(producerSpec); useThreadLocalProducers = producerSpec.getUseThreadLocalProducers(); + } else { + useThreadLocalProducers = false; } + producerBuilderFactory = new ProducerBuilderFactory(client, producerConfig, Thread.currentThread().getContextClassLoader(), // This is for backwards compatibility. The PR https://github.com/apache/pulsar/pull/19470 removed @@ -172,12 +176,6 @@ public ContextImpl(InstanceConfig config, Logger logger, PulsarClient client, this.config.getFunctionDetails().getName()), this.config.getInstanceId())); - if (useThreadLocalProducers) { - tlPublishProducers = new ThreadLocal<>(); - } else { - publishProducers = new ConcurrentHashMap<>(); - } - if (config.getFunctionDetails().getUserConfig().isEmpty()) { userConfigs = new HashMap<>(); } else { @@ -535,39 +533,15 @@ public ClientBuilder getPulsarClientBuilder() { } private Producer getProducer(String topicName, Schema schema) throws PulsarClientException { - Producer producer; - if (tlPublishProducers != null) { - Map> producerMap = tlPublishProducers.get(); - if (producerMap == null) { - producerMap = new HashMap<>(); - tlPublishProducers.set(producerMap); - } - producer = (Producer) producerMap.get(topicName); - } else { - producer = (Producer) publishProducers.get(topicName); - } - - if (producer == null) { - Producer newProducer = producerBuilderFactory - .createProducerBuilder(topicName, schema, null) - .properties(producerProperties) - .create(); - - if (tlPublishProducers != null) { - tlPublishProducers.get().put(topicName, newProducer); - } else { - Producer existingProducer = (Producer) publishProducers.putIfAbsent(topicName, newProducer); - - if (existingProducer != null) { - // The value in the map was not updated after the concurrent put - newProducer.close(); - producer = existingProducer; - } else { - producer = newProducer; - } - } - } - return producer; + Long additionalCacheKey = useThreadLocalProducers ? Thread.currentThread().getId() : null; + return producerCache.getOrCreateProducer(ProducerCache.CacheArea.CONTEXT_CACHE, + topicName, additionalCacheKey, () -> { + log.info("Initializing producer on topic {} with schema {}", topicName, schema); + return producerBuilderFactory + .createProducerBuilder(topicName, schema, null) + .properties(producerProperties) + .create(); + }); } public Map getAndResetMetrics() { @@ -706,29 +680,9 @@ public void setUnderlyingBuilder(TypedMessageBuilder underlyingBuilder) { @Override public void close() { - List futures = new LinkedList<>(); - - if (publishProducers != null) { - for (Producer producer : publishProducers.values()) { - futures.add(producer.closeAsync()); - } - } - - if (tlPublishProducers != null) { - for (Producer producer : tlPublishProducers.get().values()) { - futures.add(producer.closeAsync()); - } - } - if (pulsarAdmin != null) { pulsarAdmin.close(); } - - try { - CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).get(); - } catch (InterruptedException | ExecutionException e) { - logger.warn("Failed to close producers", e); - } } @Override 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 7459f553efa73..a3718f68197c6 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 @@ -168,6 +168,8 @@ public class JavaInstanceRunnable implements AutoCloseable, Runnable { private final AtomicReference> sinkSchema = new AtomicReference<>(); private SinkSchemaInfoProvider sinkSchemaInfoProvider = null; + private final ProducerCache producerCache = new ProducerCache(); + public JavaInstanceRunnable(InstanceConfig instanceConfig, ClientBuilder clientBuilder, PulsarClient pulsarClient, @@ -287,7 +289,7 @@ ContextImpl setupContext() throws PulsarClientException { Thread.currentThread().setContextClassLoader(functionClassLoader); return new ContextImpl(instanceConfig, instanceLog, client, secretsProvider, collectorRegistry, metricsLabels, this.componentType, this.stats, stateManager, - pulsarAdmin, clientBuilder); + pulsarAdmin, clientBuilder, producerCache); } finally { Thread.currentThread().setContextClassLoader(clsLoader); } @@ -583,6 +585,8 @@ public synchronized void close() { instanceCache = null; + producerCache.close(); + if (logAppender != null) { removeLogTopicAppender(LoggerContext.getContext()); removeLogTopicAppender(LoggerContext.getContext(false)); @@ -1001,7 +1005,7 @@ private void setupOutput(ContextImpl contextImpl) throws Exception { } object = new PulsarSink(this.client, pulsarSinkConfig, this.properties, this.stats, - this.functionClassLoader); + this.functionClassLoader, this.producerCache); } } else { object = Reflections.createInstance( 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 new file mode 100644 index 0000000000000..f68c4e9589558 --- /dev/null +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/instance/ProducerCache.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.functions.instance; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.Scheduler; +import com.google.common.annotations.VisibleForTesting; +import java.io.Closeable; +import java.time.Duration; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.common.util.FutureUtil; + +@Slf4j +public class ProducerCache implements Closeable { + // allow tuning the cache timeout with PRODUCER_CACHE_TIMEOUT_SECONDS env variable + private static final int PRODUCER_CACHE_TIMEOUT_SECONDS = + Integer.parseInt(System.getenv().getOrDefault("PRODUCER_CACHE_TIMEOUT_SECONDS", "300")); + // allow tuning the cache size with PRODUCER_CACHE_MAX_SIZE env variable + private static final int PRODUCER_CACHE_MAX_SIZE = + Integer.parseInt(System.getenv().getOrDefault("PRODUCER_CACHE_MAX_SIZE", "10000")); + private static final int FLUSH_OR_CLOSE_TIMEOUT_SECONDS = 60; + + // prevents the different producers created in different code locations from mixing up + public enum CacheArea { + // producers created by calling Context, SinkContext, SourceContext methods + CONTEXT_CACHE, + // producers created in Pulsar Sources, multiple topics are possible by returning destination topics + // by SinkRecord.getDestinationTopic call + SINK_RECORD_CACHE, + } + + record ProducerCacheKey(CacheArea cacheArea, String topic, Object additionalKey) { + } + + private final Cache> cache; + private final AtomicBoolean closed = new AtomicBoolean(false); + private final CopyOnWriteArrayList> closeFutures = new CopyOnWriteArrayList<>(); + + public ProducerCache() { + Caffeine builder = Caffeine.newBuilder() + .scheduler(Scheduler.systemScheduler()) + .removalListener((key, producer, cause) -> { + log.info("Closing producer for topic {}, cause {}", key.topic(), cause); + CompletableFuture closeFuture = + producer.flushAsync() + .orTimeout(FLUSH_OR_CLOSE_TIMEOUT_SECONDS, TimeUnit.SECONDS) + .exceptionally(ex -> { + log.error("Error flushing producer for topic {}", key.topic(), ex); + 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; + })); + if (closed.get()) { + closeFutures.add(closeFuture); + } + }) + .weigher((key, producer) -> Math.max(producer.getNumOfPartitions(), 1)) + .maximumWeight(PRODUCER_CACHE_MAX_SIZE); + if (PRODUCER_CACHE_TIMEOUT_SECONDS > 0) { + builder.expireAfterAccess(Duration.ofSeconds(PRODUCER_CACHE_TIMEOUT_SECONDS)); + } + cache = builder.build(); + } + + public Producer getOrCreateProducer(CacheArea cacheArea, String topicName, Object additionalCacheKey, + Callable> supplier) { + if (closed.get()) { + throw new IllegalStateException("ProducerCache is already closed"); + } + return (Producer) cache.get(new ProducerCacheKey(cacheArea, topicName, additionalCacheKey), key -> { + try { + return supplier.call(); + } catch (RuntimeException e) { + throw e; + } catch (Exception e) { + throw new RuntimeException("Unable to create producer for topic '" + topicName + "'", e); + } + }); + } + + public void close() { + if (closed.compareAndSet(false, true)) { + cache.invalidateAll(); + try { + FutureUtil.waitForAll(closeFutures).get(); + } catch (InterruptedException | ExecutionException e) { + log.warn("Failed to close producers", e); + } + } + } + + @VisibleForTesting + public boolean containsKey(CacheArea cacheArea, String topic) { + return containsKey(cacheArea, topic, null); + } + + @VisibleForTesting + public boolean containsKey(CacheArea cacheArea, String topic, Object additionalCacheKey) { + return cache.getIfPresent(new ProducerCacheKey(cacheArea, topic, additionalCacheKey)) != null; + } +} diff --git a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java index 18e55e8e84de1..da6b8006eb987 100644 --- a/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java +++ b/pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java @@ -20,19 +20,15 @@ import com.google.common.annotations.VisibleForTesting; import java.nio.charset.StandardCharsets; -import java.util.ArrayList; import java.util.Base64; -import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; import java.util.function.Function; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; @@ -48,6 +44,7 @@ import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.functions.instance.AbstractSinkRecord; import org.apache.pulsar.functions.instance.ProducerBuilderFactory; +import org.apache.pulsar.functions.instance.ProducerCache; import org.apache.pulsar.functions.instance.stats.ComponentStatsManager; import org.apache.pulsar.functions.source.PulsarRecord; import org.apache.pulsar.functions.source.TopicSchema; @@ -62,6 +59,7 @@ public class PulsarSink implements Sink { private final Map properties; private final ClassLoader functionClassLoader; private ComponentStatsManager stats; + private final ProducerCache producerCache; @VisibleForTesting PulsarSinkProcessor pulsarSinkProcessor; @@ -80,43 +78,25 @@ private interface PulsarSinkProcessor { } abstract class PulsarSinkProcessorBase implements PulsarSinkProcessor { - protected Map> publishProducers = new ConcurrentHashMap<>(); - protected Producer getProducer(String destinationTopic, Schema schema) { - return getProducer(destinationTopic, null, destinationTopic, schema); + return getProducer(destinationTopic, schema, null, null); } - protected Producer getProducer(String producerId, String producerName, String topicName, Schema schema) { - return publishProducers.computeIfAbsent(producerId, s -> { - try { - log.info("Initializing producer {} on topic {} with schema {}", - producerName, topicName, schema); - Producer producer = createProducer( - topicName, - schema, producerName - ); - log.info("Initialized producer {} on topic {} with schema {}: {} -> {}", - producerName, topicName, schema, producerId, producer); - return producer; - } catch (PulsarClientException e) { - log.error("Failed to create Producer while doing user publish", e); - throw new RuntimeException(e); - } - }); + protected Producer getProducer(String topicName, Schema schema, String producerName, String partitionId) { + return producerCache.getOrCreateProducer(ProducerCache.CacheArea.SINK_RECORD_CACHE, topicName, partitionId, + () -> { + Producer producer = createProducer(topicName, schema, producerName); + log.info( + "Initialized producer with name '{}' on topic '{}' with schema {} partitionId {} " + + "-> {}", + producerName, topicName, schema, partitionId, producer); + return producer; + }); } @Override public void close() throws Exception { - List> closeFutures = new ArrayList<>(publishProducers.size()); - for (Map.Entry> entry : publishProducers.entrySet()) { - Producer producer = entry.getValue(); - closeFutures.add(producer.closeAsync()); - } - try { - org.apache.pulsar.common.util.FutureUtil.waitForAll(closeFutures); - } catch (Exception e) { - log.warn("Failed to close all the producers", e); - } + // no op } public Function getPublishErrorHandler(AbstractSinkRecord record, boolean failSource) { @@ -153,13 +133,7 @@ class PulsarSinkAtMostOnceProcessor extends PulsarSinkProcessorBase { public PulsarSinkAtMostOnceProcessor() { if (!(schema instanceof AutoConsumeSchema)) { // initialize default topic - try { - publishProducers.put(pulsarSinkConfig.getTopic(), - createProducer(pulsarSinkConfig.getTopic(), schema, null)); - } catch (PulsarClientException e) { - log.error("Failed to create Producer while doing user publish", e); - throw new RuntimeException(e); - } + getProducer(pulsarSinkConfig.getTopic(), schema); } else { if (log.isDebugEnabled()) { log.debug("The Pulsar producer is not initialized until the first record is" @@ -232,13 +206,10 @@ public TypedMessageBuilder newMessage(AbstractSinkRecord record) { // we must use the destination topic schema schemaToWrite = schema; } - Producer producer = getProducer( - String.format("%s-%s", record.getDestinationTopic().orElse(pulsarSinkConfig.getTopic()), - record.getPartitionId().get()), - record.getPartitionId().get(), - record.getDestinationTopic().orElse(pulsarSinkConfig.getTopic()), - schemaToWrite - ); + String topicName = record.getDestinationTopic().orElse(pulsarSinkConfig.getTopic()); + String partitionId = record.getPartitionId().get(); + String producerName = partitionId; + Producer producer = getProducer(topicName, schemaToWrite, producerName, partitionId); if (schemaToWrite != null) { return producer.newMessage(schemaToWrite); } else { @@ -263,13 +234,14 @@ public void sendOutputMessage(TypedMessageBuilder msg, AbstractSinkRecord } public PulsarSink(PulsarClient client, PulsarSinkConfig pulsarSinkConfig, Map properties, - ComponentStatsManager stats, ClassLoader functionClassLoader) { + ComponentStatsManager stats, ClassLoader functionClassLoader, ProducerCache producerCache) { this.client = client; this.pulsarSinkConfig = pulsarSinkConfig; this.topicSchema = new TopicSchema(client, functionClassLoader); this.properties = properties; this.stats = stats; this.functionClassLoader = functionClassLoader; + this.producerCache = producerCache; } @Override @@ -341,14 +313,17 @@ public void close() throws Exception { } } - Producer createProducer(String topic, Schema schema, String producerName) - throws PulsarClientException { - ProducerBuilder builder = - producerBuilderFactory.createProducerBuilder(topic, schema != null ? schema : this.schema, - producerName); - return builder - .properties(properties) - .create(); + Producer createProducer(String topicName, Schema schema, String producerName) { + Schema schemaToUse = schema != null ? schema : this.schema; + try { + log.info("Initializing producer {} on topic {} with schema {}", producerName, topicName, schemaToUse); + return producerBuilderFactory.createProducerBuilder(topicName, schemaToUse, producerName) + .properties(properties) + .create(); + } catch (PulsarClientException e) { + throw new RuntimeException("Failed to create Producer for topic " + topicName + + " producerName " + producerName + " schema " + schemaToUse, e); + } } @SuppressWarnings("unchecked") diff --git a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java index 108d8e4b66663..e62838ed3d209 100644 --- a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java +++ b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/instance/ContextImplTest.java @@ -71,6 +71,7 @@ import org.mockito.Mockito; import org.slf4j.Logger; import org.testng.Assert; +import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -86,6 +87,7 @@ public class ContextImplTest { private PulsarAdmin pulsarAdmin; private ContextImpl context; private Producer producer; + private ProducerCache producerCache; @BeforeMethod(alwaysRun = true) public void setup() throws PulsarClientException { @@ -116,16 +118,24 @@ public void setup() throws PulsarClientException { TypedMessageBuilder messageBuilder = spy(new TypedMessageBuilderImpl(mock(ProducerBase.class), Schema.STRING)); doReturn(new CompletableFuture<>()).when(messageBuilder).sendAsync(); when(producer.newMessage()).thenReturn(messageBuilder); + doReturn(CompletableFuture.completedFuture(null)).when(producer).flushAsync(); + producerCache = new ProducerCache(); context = new ContextImpl( config, logger, client, new EnvironmentBasedSecretsProvider(), FunctionCollectorRegistry.getDefaultImplementation(), new String[0], FunctionDetails.ComponentType.FUNCTION, null, new InstanceStateManager(), - pulsarAdmin, clientBuilder); + pulsarAdmin, clientBuilder, producerCache); context.setCurrentMessageContext((Record) () -> null); } + @AfterMethod(alwaysRun = true) + public void tearDown() { + producerCache.close(); + producerCache = null; + } + @Test(expectedExceptions = IllegalStateException.class) public void testIncrCounterStateDisabled() { context.incrCounter("test-key", 10); @@ -236,7 +246,7 @@ public void testGetPulsarAdminWithExposePulsarAdminDisabled() throws PulsarClien new EnvironmentBasedSecretsProvider(), FunctionCollectorRegistry.getDefaultImplementation(), new String[0], FunctionDetails.ComponentType.FUNCTION, null, new InstanceStateManager(), - pulsarAdmin, clientBuilder); + pulsarAdmin, clientBuilder, producerCache); context.getPulsarAdmin(); } @@ -250,7 +260,7 @@ public void testUnsupportedExtendedSinkContext() throws PulsarClientException { new EnvironmentBasedSecretsProvider(), FunctionCollectorRegistry.getDefaultImplementation(), new String[0], FunctionDetails.ComponentType.FUNCTION, null, new InstanceStateManager(), - pulsarAdmin, clientBuilder); + pulsarAdmin, clientBuilder, producerCache); try { context.seek("z", 0, Mockito.mock(MessageId.class)); Assert.fail("Expected exception"); @@ -281,7 +291,7 @@ public void testExtendedSinkContext() throws PulsarClientException { new EnvironmentBasedSecretsProvider(), FunctionCollectorRegistry.getDefaultImplementation(), new String[0], FunctionDetails.ComponentType.FUNCTION, null, new InstanceStateManager(), - pulsarAdmin, clientBuilder); + pulsarAdmin, clientBuilder, producerCache); Consumer mockConsumer = Mockito.mock(Consumer.class); when(mockConsumer.getTopic()).thenReturn(TopicName.get("z").toString()); context.setInputConsumers(Lists.newArrayList(mockConsumer)); @@ -313,7 +323,7 @@ public void testGetConsumer() throws PulsarClientException { new EnvironmentBasedSecretsProvider(), FunctionCollectorRegistry.getDefaultImplementation(), new String[0], FunctionDetails.ComponentType.FUNCTION, null, new InstanceStateManager(), - pulsarAdmin, clientBuilder); + pulsarAdmin, clientBuilder, producerCache); Consumer mockConsumer = Mockito.mock(Consumer.class); when(mockConsumer.getTopic()).thenReturn(TopicName.get("z").toString()); context.setInputConsumers(Lists.newArrayList(mockConsumer)); @@ -337,7 +347,7 @@ public void testGetConsumerMultiTopic() throws PulsarClientException { new EnvironmentBasedSecretsProvider(), FunctionCollectorRegistry.getDefaultImplementation(), new String[0], FunctionDetails.ComponentType.FUNCTION, null, new InstanceStateManager(), - pulsarAdmin, clientBuilder); + pulsarAdmin, clientBuilder, producerCache); ConsumerImpl consumer1 = Mockito.mock(ConsumerImpl.class); when(consumer1.getTopic()).thenReturn(TopicName.get("first").toString()); ConsumerImpl consumer2 = Mockito.mock(ConsumerImpl.class); diff --git a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/sink/PulsarSinkTest.java b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/sink/PulsarSinkTest.java index 799bad839a451..8a946a3f7571b 100644 --- a/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/sink/PulsarSinkTest.java +++ b/pulsar-functions/instance/src/test/java/org/apache/pulsar/functions/sink/PulsarSinkTest.java @@ -18,13 +18,13 @@ */ package org.apache.pulsar.functions.sink; +import static org.apache.pulsar.functions.instance.ProducerCache.CacheArea.SINK_RECORD_CACHE; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyList; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -37,7 +37,6 @@ import static org.testng.Assert.fail; import java.io.IOException; import java.util.HashMap; -import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; import lombok.Getter; @@ -65,12 +64,14 @@ import org.apache.pulsar.common.schema.SchemaType; import org.apache.pulsar.functions.api.Record; import org.apache.pulsar.functions.api.SerDe; +import org.apache.pulsar.functions.instance.ProducerCache; import org.apache.pulsar.functions.instance.SinkRecord; import org.apache.pulsar.functions.instance.stats.ComponentStatsManager; import org.apache.pulsar.functions.sink.PulsarSink.PulsarSinkProcessorBase; import org.apache.pulsar.functions.source.TopicSchema; import org.apache.pulsar.io.core.SinkContext; import org.testng.Assert; +import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -132,6 +133,7 @@ private static PulsarClientImpl getPulsarClient() throws PulsarClientException { doReturn(producer).when(producerBuilder).create(); doReturn(typedMessageBuilder).when(producer).newMessage(); doReturn(typedMessageBuilder).when(producer).newMessage(any(Schema.class)); + doReturn(CompletableFuture.completedFuture(null)).when(producer).flushAsync(); doReturn(producerBuilder).when(pulsarClient).newProducer(); doReturn(producerBuilder).when(pulsarClient).newProducer(any()); @@ -139,9 +141,17 @@ private static PulsarClientImpl getPulsarClient() throws PulsarClientException { return pulsarClient; } - @BeforeMethod + ProducerCache producerCache; + + @BeforeMethod(alwaysRun = true) public void setup() { + producerCache = new ProducerCache(); + } + @AfterMethod(alwaysRun = true) + public void tearDown() { + producerCache.close(); + producerCache = null; } private static PulsarSinkConfig getPulsarConfigs() { @@ -182,7 +192,7 @@ public void testVoidOutputClasses() throws Exception { pulsarConfig.setTypeClassName(Void.class.getName()); PulsarSink pulsarSink = new PulsarSink(getPulsarClient(), pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); try { Schema schema = pulsarSink.initializeSchema(); @@ -202,7 +212,7 @@ public void testInconsistentOutputType() throws IOException { pulsarConfig.setSerdeClassName(TestSerDe.class.getName()); PulsarSink pulsarSink = new PulsarSink(getPulsarClient(), pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); try { pulsarSink.initializeSchema(); fail("Should fail constructing java instance if function type is inconsistent with serde type"); @@ -227,7 +237,7 @@ public void testDefaultSerDe() throws PulsarClientException { pulsarConfig.setTypeClassName(String.class.getName()); PulsarSink pulsarSink = new PulsarSink(getPulsarClient(), pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); try { pulsarSink.initializeSchema(); @@ -248,7 +258,7 @@ public void testExplicitDefaultSerDe() throws PulsarClientException { pulsarConfig.setSerdeClassName(TopicSchema.DEFAULT_SERDE); PulsarSink pulsarSink = new PulsarSink(getPulsarClient(), pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); try { pulsarSink.initializeSchema(); @@ -266,7 +276,7 @@ public void testComplexOuputType() throws PulsarClientException { pulsarConfig.setSerdeClassName(ComplexSerDe.class.getName()); PulsarSink pulsarSink = new PulsarSink(getPulsarClient(), pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); try { pulsarSink.initializeSchema(); @@ -286,7 +296,7 @@ public void testInitializeSchema() throws Exception { pulsarSinkConfig.setTypeClassName(GenericRecord.class.getName()); PulsarSink sink = new PulsarSink( pulsarClient, pulsarSinkConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); Schema schema = sink.initializeSchema(); assertTrue(schema instanceof AutoConsumeSchema); @@ -295,7 +305,7 @@ public void testInitializeSchema() throws Exception { pulsarSinkConfig.setTypeClassName(GenericRecord.class.getName()); sink = new PulsarSink( pulsarClient, pulsarSinkConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); schema = sink.initializeSchema(); assertTrue(schema instanceof AutoConsumeSchema); @@ -306,7 +316,7 @@ public void testInitializeSchema() throws Exception { pulsarSinkConfig.setTypeClassName(GenericRecord.class.getName()); sink = new PulsarSink( pulsarClient, pulsarSinkConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); schema = sink.initializeSchema(); assertTrue(schema instanceof AutoConsumeSchema); @@ -317,7 +327,7 @@ public void testInitializeSchema() throws Exception { pulsarSinkConfig.setTypeClassName(GenericRecord.class.getName()); sink = new PulsarSink( pulsarClient, pulsarSinkConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); schema = sink.initializeSchema(); assertTrue(schema instanceof AutoConsumeSchema); @@ -327,7 +337,7 @@ public void testInitializeSchema() throws Exception { pulsarSinkConfig.setTypeClassName(GenericRecord.class.getName()); sink = new PulsarSink( pulsarClient, pulsarSinkConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); schema = sink.initializeSchema(); assertTrue(schema instanceof AutoConsumeSchema); } @@ -344,9 +354,12 @@ public void testSinkAndMessageRouting() throws Exception { /** test MANUAL **/ pulsarClient = getPulsarClient(); pulsarConfig.setProcessingGuarantees(ProcessingGuarantees.MANUAL); - PulsarSink pulsarSink = new PulsarSink(pulsarClient, pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), Thread.currentThread().getContextClassLoader()); + PulsarSink pulsarSink = + new PulsarSink(pulsarClient, pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), + Thread.currentThread().getContextClassLoader(), producerCache); pulsarSink.open(new HashMap<>(), mock(SinkContext.class)); + verify(pulsarClient.newProducer(), times(1)).topic(defaultTopic); for (String topic : topics) { @@ -370,23 +383,19 @@ public Optional getDestinationTopic() { PulsarSink.PulsarSinkManualProcessor pulsarSinkManualProcessor = (PulsarSink.PulsarSinkManualProcessor) pulsarSink.pulsarSinkProcessor; if (topic != null) { - Assert.assertTrue(pulsarSinkManualProcessor.publishProducers.containsKey(topic)); + Assert.assertTrue(producerCache.containsKey(SINK_RECORD_CACHE, topic)); } else { - Assert.assertTrue(pulsarSinkManualProcessor.publishProducers.containsKey(defaultTopic)); + Assert.assertTrue(producerCache.containsKey(SINK_RECORD_CACHE, defaultTopic)); } - verify(pulsarClient.newProducer(), times(1)).topic(argThat(otherTopic -> { - if (topic != null) { - return topic.equals(otherTopic); - } else { - return defaultTopic.equals(otherTopic); - } - })); + String actualTopic = topic != null ? topic : defaultTopic; + verify(pulsarClient.newProducer(), times(1)).topic(actualTopic); } /** test At-least-once **/ pulsarClient = getPulsarClient(); pulsarConfig.setProcessingGuarantees(ProcessingGuarantees.ATLEAST_ONCE); - pulsarSink = new PulsarSink(pulsarClient, pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), Thread.currentThread().getContextClassLoader()); + pulsarSink = new PulsarSink(pulsarClient, pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), + Thread.currentThread().getContextClassLoader(), producerCache); pulsarSink.open(new HashMap<>(), mock(SinkContext.class)); @@ -410,24 +419,17 @@ public Optional getDestinationTopic() { PulsarSink.PulsarSinkAtLeastOnceProcessor pulsarSinkAtLeastOnceProcessor = (PulsarSink.PulsarSinkAtLeastOnceProcessor) pulsarSink.pulsarSinkProcessor; if (topic != null) { - Assert.assertTrue(pulsarSinkAtLeastOnceProcessor.publishProducers.containsKey(topic)); + Assert.assertTrue(producerCache.containsKey(SINK_RECORD_CACHE, topic)); } else { - Assert.assertTrue(pulsarSinkAtLeastOnceProcessor.publishProducers.containsKey(defaultTopic)); + Assert.assertTrue(producerCache.containsKey(SINK_RECORD_CACHE, defaultTopic)); } - verify(pulsarClient.newProducer(), times(1)).topic(argThat(otherTopic -> { - if (topic != null) { - return topic.equals(otherTopic); - } else { - return defaultTopic.equals(otherTopic); - } - })); } /** test At-most-once **/ pulsarClient = getPulsarClient(); pulsarConfig.setProcessingGuarantees(ProcessingGuarantees.ATMOST_ONCE); pulsarSink = new PulsarSink(pulsarClient, pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); pulsarSink.open(new HashMap<>(), mock(SinkContext.class)); @@ -457,20 +459,17 @@ public Optional getDestinationTopic() { PulsarSink.PulsarSinkAtMostOnceProcessor pulsarSinkAtLeastOnceProcessor = (PulsarSink.PulsarSinkAtMostOnceProcessor) pulsarSink.pulsarSinkProcessor; if (topic != null) { - Assert.assertTrue(pulsarSinkAtLeastOnceProcessor.publishProducers.containsKey(topic)); + Assert.assertTrue(producerCache.containsKey(SINK_RECORD_CACHE, topic)); } else { - Assert.assertTrue(pulsarSinkAtLeastOnceProcessor.publishProducers.containsKey(defaultTopic)); + Assert.assertTrue(producerCache.containsKey(SINK_RECORD_CACHE, defaultTopic)); } - verify(pulsarClient.newProducer(), times(1)).topic(argThat(o -> { - return getTopicEquals(o, topic, defaultTopic); - })); } /** test Effectively-once **/ pulsarClient = getPulsarClient(); pulsarConfig.setProcessingGuarantees(FunctionConfig.ProcessingGuarantees.EFFECTIVELY_ONCE); pulsarSink = new PulsarSink(pulsarClient, pulsarConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); pulsarSink.open(new HashMap<>(), mock(SinkContext.class)); @@ -520,23 +519,19 @@ public Optional getRecordSequence() { PulsarSink.PulsarSinkEffectivelyOnceProcessor pulsarSinkEffectivelyOnceProcessor = (PulsarSink.PulsarSinkEffectivelyOnceProcessor) pulsarSink.pulsarSinkProcessor; if (topic != null) { - Assert.assertTrue(pulsarSinkEffectivelyOnceProcessor.publishProducers - .containsKey(String.format("%s-%s-id-1", topic, topic))); + Assert.assertTrue(producerCache + .containsKey(SINK_RECORD_CACHE, topic, String.format("%s-id-1", topic))); } else { - Assert.assertTrue(pulsarSinkEffectivelyOnceProcessor.publishProducers - .containsKey(String.format("%s-%s-id-1", defaultTopic, defaultTopic))); + Assert.assertTrue(producerCache + .containsKey(SINK_RECORD_CACHE, + defaultTopic, String.format("%s-id-1", defaultTopic) + )); } - verify(pulsarClient.newProducer(), times(1)).topic(argThat(o -> { - return getTopicEquals(o, topic, defaultTopic); - })); - verify(pulsarClient.newProducer(), times(1)).producerName(argThat(o -> { - if (topic != null) { - return String.format("%s-id-1", topic).equals(o); - } else { - return String.format("%s-id-1", defaultTopic).equals(o); - } - })); + String expectedTopicName = topic != null ? topic : defaultTopic; + verify(pulsarClient.newProducer(), times(1)).topic(expectedTopicName); + String expectedProducerName = String.format("%s-id-1", expectedTopicName); + verify(pulsarClient.newProducer(), times(1)).producerName(expectedProducerName); } } @@ -566,7 +561,7 @@ private void testWriteGenericRecords(ProcessingGuarantees guarantees) throws Exc PulsarClient client = getPulsarClient(); PulsarSink pulsarSink = new PulsarSink( client, sinkConfig, new HashMap<>(), mock(ComponentStatsManager.class), - Thread.currentThread().getContextClassLoader()); + Thread.currentThread().getContextClassLoader(), producerCache); pulsarSink.open(new HashMap<>(), mock(SinkContext.class)); @@ -578,7 +573,7 @@ private void testWriteGenericRecords(ProcessingGuarantees guarantees) throws Exc assertTrue(pulsarSink.pulsarSinkProcessor instanceof PulsarSink.PulsarSinkEffectivelyOnceProcessor); } PulsarSinkProcessorBase processor = (PulsarSinkProcessorBase) pulsarSink.pulsarSinkProcessor; - assertFalse(processor.publishProducers.containsKey(defaultTopic)); + assertFalse(producerCache.containsKey(SINK_RECORD_CACHE, defaultTopic)); String[] topics = {"topic-1", "topic-2", "topic-3"}; for (String topic : topics) { @@ -625,17 +620,15 @@ public Optional getRecordSequence() { pulsarSink.write(record); if (ProcessingGuarantees.EFFECTIVELY_ONCE == guarantees) { - assertTrue(processor.publishProducers.containsKey(String.format("%s-%s-id-1", topic, topic))); + assertTrue(producerCache.containsKey(SINK_RECORD_CACHE, + topic, String.format("%s-id-1", topic) + )); } else { - assertTrue(processor.publishProducers.containsKey(topic)); + assertTrue(producerCache.containsKey(SINK_RECORD_CACHE, topic)); } - verify(client.newProducer(), times(1)) - .topic(argThat( - otherTopic -> topic != null ? topic.equals(otherTopic) : defaultTopic.equals(otherTopic))); - - verify(client, times(1)) - .newProducer(argThat( - otherSchema -> Objects.equals(otherSchema, schema))); + String expectedTopicName = topic != null ? topic : defaultTopic; + verify(client.newProducer(), times(1)).topic(expectedTopicName); + verify(client, times(1)).newProducer(schema); } } @@ -646,13 +639,4 @@ private Optional getTopicOptional(String topic) { return Optional.empty(); } } - - private boolean getTopicEquals(Object o, String topic, String defaultTopic) { - if (topic != null) { - return topic.equals(o); - } else { - return defaultTopic.equals(o); - } - } - } From 659896ed9b40378a97e7c81d016f377fb6b89557 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 21 Jun 2024 23:06:56 +0300 Subject: [PATCH 577/699] [fix][misc] Rename netty native libraries in pulsar-client-admin-shaded (#22954) (cherry picked from commit ddb03bb6a3b67ffcc71c7e95a87b35eb302a7393) (cherry picked from commit 07df550ce051ec14f15eb3354138855d6879d2c9) --- pulsar-client-admin-shaded/pom.xml | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/pulsar-client-admin-shaded/pom.xml b/pulsar-client-admin-shaded/pom.xml index e2c082c648d7f..4605b784e77ca 100644 --- a/pulsar-client-admin-shaded/pom.xml +++ b/pulsar-client-admin-shaded/pom.xml @@ -310,6 +310,31 @@ + + + + exec-maven-plugin + org.codehaus.mojo + + + rename-epoll-library + package + + exec + + + ${project.parent.basedir}/src/${rename.netty.native.libs} + + ${project.artifactId} + + + + + From c404e19d0ab2165e070cc770b20b709bf551eb8b Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Mon, 24 Jun 2024 17:09:51 +0800 Subject: [PATCH 578/699] [improve] [broker] make system topic distribute evenly. (#22953) (cherry picked from commit 263c6948fb3dd10480f39a9202c6fcc4a7d55d8e) (cherry picked from commit 2c1fb16fb3962ae2ba1efb793b14262560a2ded6) --- .../broker/loadbalance/impl/ModularLoadManagerImpl.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 320c273a2d9b7..d6c079ab85ce7 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 @@ -907,7 +907,9 @@ Optional selectBroker(final ServiceUnitId serviceUnit) { brokerToNamespaceToBundleRange, brokerToFailureDomainMap); // distribute bundles evenly to candidate-brokers if enable - if (conf.isLoadBalancerDistributeBundlesEvenlyEnabled()) { + // or system-namespace bundles + if (conf.isLoadBalancerDistributeBundlesEvenlyEnabled() + || serviceUnit.getNamespaceObject().equals(NamespaceName.SYSTEM_NAMESPACE)) { LoadManagerShared.removeMostServicingBrokersForNamespace(bundle, brokerCandidateCache, brokerToNamespaceToBundleRange); From 245b19783e925ed27bd390250aac256f44af15e9 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 24 Jun 2024 19:34:14 +0300 Subject: [PATCH 579/699] Revert "[improve][broker] Optimize `ConcurrentOpenLongPairRangeSet` by RoaringBitmap (#22908)" This reverts commit f99040db727bfadd879ffc4d552337f029ebbe6e. (cherry picked from commit 266f98c7c3d1097a4efac876dc54bfbfb623107b) --- .../server/src/assemble/LICENSE.bin.txt | 3 +- .../shell/src/assemble/LICENSE.bin.txt | 2 - pom.xml | 2 +- pulsar-common/pom.xml | 5 - .../ConcurrentOpenLongPairRangeSet.java | 12 +- .../collections/ConcurrentRoaringBitSet.java | 439 ------------------ pulsar-sql/presto-distribution/LICENSE | 3 +- 7 files changed, 11 insertions(+), 455 deletions(-) delete mode 100644 pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentRoaringBitSet.java diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 6a9122e7e32a2..c5e20faf13ad5 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -505,7 +505,8 @@ The Apache Software License, Version 2.0 * RxJava - io.reactivex.rxjava3-rxjava-3.0.1.jar * RoaringBitmap - - org.roaringbitmap-RoaringBitmap-1.1.0.jar + - org.roaringbitmap-RoaringBitmap-0.9.44.jar + - org.roaringbitmap-shims-0.9.44.jar BSD 3-clause "New" or "Revised" License * Google auth library diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 46400b122ad1d..396bd8d0e83e3 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -380,8 +380,6 @@ The Apache Software License, Version 2.0 - simpleclient_tracer_common-0.16.0.jar - simpleclient_tracer_otel-0.16.0.jar - simpleclient_tracer_otel_agent-0.16.0.jar - * RoaringBitmap - - RoaringBitmap-1.1.0.jar * Log4J - log4j-api-2.18.0.jar - log4j-core-2.18.0.jar diff --git a/pom.xml b/pom.xml index 2d2fb8be00f4e..525ec81208167 100644 --- a/pom.xml +++ b/pom.xml @@ -306,7 +306,7 @@ flexible messaging model and an intuitive client API. 1.3 0.4 9.1.0 - 1.1.0 + 0.9.44 1.6.1 6.4.0 diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index e39dd288abf6e..04ace01ceb5e5 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -244,11 +244,6 @@ awaitility test - - - org.roaringbitmap - RoaringBitmap - 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 b5ad89d1695d4..72215d7296cc3 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 @@ -29,7 +29,6 @@ import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.commons.lang.mutable.MutableInt; -import org.roaringbitmap.RoaringBitSet; /** * A Concurrent set comprising zero or more ranges of type {@link LongPair}. This can be alternative of @@ -45,7 +44,7 @@ public class ConcurrentOpenLongPairRangeSet> implements LongPairRangeSet { protected final NavigableMap rangeBitSetMap = new ConcurrentSkipListMap<>(); - private final boolean threadSafe; + private boolean threadSafe = true; private final int bitSetSize; private final LongPairConsumer consumer; @@ -96,7 +95,9 @@ public void addOpenClosed(long lowerKey, long lowerValueOpen, long upperKey, lon // (2) set 0th-index to upper-index in upperRange.getKey() if (isValid(upperKey, upperValue)) { BitSet rangeBitSet = rangeBitSetMap.computeIfAbsent(upperKey, (key) -> createNewBitSet()); - rangeBitSet.set(0, (int) upperValue + 1); + if (rangeBitSet != null) { + rangeBitSet.set(0, (int) upperValue + 1); + } } // No-op if values are not valid eg: if lower == LongPair.earliest or upper == LongPair.latest then nothing // to set @@ -413,6 +414,7 @@ private int getSafeEntry(long value) { } private BitSet createNewBitSet() { - return this.threadSafe ? new ConcurrentRoaringBitSet() : new RoaringBitSet(); + return this.threadSafe ? new ConcurrentBitSet(bitSetSize) : new BitSet(bitSetSize); } -} \ No newline at end of file + +} diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentRoaringBitSet.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentRoaringBitSet.java deleted file mode 100644 index 814e58400993b..0000000000000 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentRoaringBitSet.java +++ /dev/null @@ -1,439 +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 java.util.BitSet; -import java.util.concurrent.locks.StampedLock; -import java.util.stream.IntStream; -import org.roaringbitmap.RoaringBitSet; - -public class ConcurrentRoaringBitSet extends RoaringBitSet { - private final StampedLock rwLock = new StampedLock(); - - public ConcurrentRoaringBitSet() { - super(); - } - - @Override - public boolean get(int bitIndex) { - long stamp = rwLock.tryOptimisticRead(); - boolean isSet = super.get(bitIndex); - if (!rwLock.validate(stamp)) { - stamp = rwLock.readLock(); - try { - isSet = super.get(bitIndex); - } finally { - rwLock.unlockRead(stamp); - } - } - return isSet; - } - - @Override - public void set(int bitIndex) { - long stamp = rwLock.writeLock(); - try { - super.set(bitIndex); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void clear(int bitIndex) { - long stamp = rwLock.writeLock(); - try { - super.clear(bitIndex); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void set(int fromIndex, int toIndex) { - long stamp = rwLock.writeLock(); - try { - super.set(fromIndex, toIndex); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void clear(int fromIndex, int toIndex) { - long stamp = rwLock.writeLock(); - try { - super.clear(fromIndex, toIndex); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void clear() { - long stamp = rwLock.writeLock(); - try { - super.clear(); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public int nextSetBit(int fromIndex) { - long stamp = rwLock.tryOptimisticRead(); - int nextSetBit = super.nextSetBit(fromIndex); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - nextSetBit = super.nextSetBit(fromIndex); - } finally { - rwLock.unlockRead(stamp); - } - } - return nextSetBit; - } - - @Override - public int nextClearBit(int fromIndex) { - long stamp = rwLock.tryOptimisticRead(); - int nextClearBit = super.nextClearBit(fromIndex); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - nextClearBit = super.nextClearBit(fromIndex); - } finally { - rwLock.unlockRead(stamp); - } - } - return nextClearBit; - } - - @Override - public int previousSetBit(int fromIndex) { - long stamp = rwLock.tryOptimisticRead(); - int previousSetBit = super.previousSetBit(fromIndex); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - previousSetBit = super.previousSetBit(fromIndex); - } finally { - rwLock.unlockRead(stamp); - } - } - return previousSetBit; - } - - @Override - public int previousClearBit(int fromIndex) { - long stamp = rwLock.tryOptimisticRead(); - int previousClearBit = super.previousClearBit(fromIndex); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - previousClearBit = super.previousClearBit(fromIndex); - } finally { - rwLock.unlockRead(stamp); - } - } - return previousClearBit; - } - - @Override - public int length() { - long stamp = rwLock.tryOptimisticRead(); - int length = super.length(); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - length = super.length(); - } finally { - rwLock.unlockRead(stamp); - } - } - return length; - } - - @Override - public boolean isEmpty() { - long stamp = rwLock.tryOptimisticRead(); - boolean isEmpty = super.isEmpty(); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - isEmpty = super.isEmpty(); - } finally { - rwLock.unlockRead(stamp); - } - } - return isEmpty; - } - - @Override - public int cardinality() { - long stamp = rwLock.tryOptimisticRead(); - int cardinality = super.cardinality(); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - cardinality = super.cardinality(); - } finally { - rwLock.unlockRead(stamp); - } - } - return cardinality; - } - - @Override - public int size() { - long stamp = rwLock.tryOptimisticRead(); - int size = super.size(); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - size = super.size(); - } finally { - rwLock.unlockRead(stamp); - } - } - return size; - } - - @Override - public byte[] toByteArray() { - long stamp = rwLock.tryOptimisticRead(); - byte[] byteArray = super.toByteArray(); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - byteArray = super.toByteArray(); - } finally { - rwLock.unlockRead(stamp); - } - } - return byteArray; - } - - @Override - public long[] toLongArray() { - long stamp = rwLock.tryOptimisticRead(); - long[] longArray = super.toLongArray(); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - longArray = super.toLongArray(); - } finally { - rwLock.unlockRead(stamp); - } - } - return longArray; - } - - @Override - public void flip(int bitIndex) { - long stamp = rwLock.writeLock(); - try { - super.flip(bitIndex); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void flip(int fromIndex, int toIndex) { - long stamp = rwLock.writeLock(); - try { - super.flip(fromIndex, toIndex); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void set(int bitIndex, boolean value) { - long stamp = rwLock.writeLock(); - try { - super.set(bitIndex, value); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void set(int fromIndex, int toIndex, boolean value) { - long stamp = rwLock.writeLock(); - try { - super.set(fromIndex, toIndex, value); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public BitSet get(int fromIndex, int toIndex) { - long stamp = rwLock.tryOptimisticRead(); - BitSet bitSet = super.get(fromIndex, toIndex); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - bitSet = super.get(fromIndex, toIndex); - } finally { - rwLock.unlockRead(stamp); - } - } - return bitSet; - } - - @Override - public boolean intersects(BitSet set) { - long stamp = rwLock.writeLock(); - try { - return super.intersects(set); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void and(BitSet set) { - long stamp = rwLock.writeLock(); - try { - super.and(set); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void or(BitSet set) { - long stamp = rwLock.writeLock(); - try { - super.or(set); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void xor(BitSet set) { - long stamp = rwLock.writeLock(); - try { - super.xor(set); - } finally { - rwLock.unlockWrite(stamp); - } - } - - @Override - public void andNot(BitSet set) { - long stamp = rwLock.writeLock(); - try { - super.andNot(set); - } finally { - rwLock.unlockWrite(stamp); - } - } - - /** - * Returns the clone of the internal wrapped {@code BitSet}. - * This won't be a clone of the {@code ConcurrentBitSet} object. - * - * @return a clone of the internal wrapped {@code BitSet} - */ - @Override - public Object clone() { - long stamp = rwLock.tryOptimisticRead(); - RoaringBitSet clone = (RoaringBitSet) super.clone(); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - clone = (RoaringBitSet) super.clone(); - } finally { - rwLock.unlockRead(stamp); - } - } - return clone; - } - - @Override - public String toString() { - long stamp = rwLock.tryOptimisticRead(); - String str = super.toString(); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - str = super.toString(); - } finally { - rwLock.unlockRead(stamp); - } - } - return str; - } - - /** - * This operation is not supported on {@code ConcurrentBitSet}. - */ - @Override - public IntStream stream() { - throw new UnsupportedOperationException("stream is not supported"); - } - - public boolean equals(final Object o) { - long stamp = rwLock.tryOptimisticRead(); - boolean isEqual = super.equals(o); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - isEqual = super.equals(o); - } finally { - rwLock.unlockRead(stamp); - } - } - return isEqual; - } - - public int hashCode() { - long stamp = rwLock.tryOptimisticRead(); - int hashCode = super.hashCode(); - if (!rwLock.validate(stamp)) { - // Fallback to read lock - stamp = rwLock.readLock(); - try { - hashCode = super.hashCode(); - } finally { - rwLock.unlockRead(stamp); - } - } - return hashCode; - } -} diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index c94adca4ca2de..ccf112a98e24e 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -484,8 +484,7 @@ The Apache Software License, Version 2.0 - stream-2.9.5.jar * High Performance Primitive Collections for Java - hppc-0.9.1.jar - * RoaringBitmap - - RoaringBitmap-1.1.0.jar + Protocol Buffers License * Protocol Buffers From 233101729b80bc2a3d4e7c85143cbc41ab7ecb13 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 24 Jun 2024 19:54:27 +0300 Subject: [PATCH 580/699] [improve][misc] Replace rename-netty-native-libs.sh script with renaming with maven-shade-plugin (#22957) (cherry picked from commit f728b2ebb9bfe2dfe1f64643640700f762524c40) (cherry picked from commit df070b6474f41f84e244175fff2ed7538159eaf8) --- pulsar-client-admin-shaded/pom.xml | 25 ------------------------- 1 file changed, 25 deletions(-) diff --git a/pulsar-client-admin-shaded/pom.xml b/pulsar-client-admin-shaded/pom.xml index 4605b784e77ca..e2c082c648d7f 100644 --- a/pulsar-client-admin-shaded/pom.xml +++ b/pulsar-client-admin-shaded/pom.xml @@ -310,31 +310,6 @@ - - - - exec-maven-plugin - org.codehaus.mojo - - - rename-epoll-library - package - - exec - - - ${project.parent.basedir}/src/${rename.netty.native.libs} - - ${project.artifactId} - - - - - From 3f48e5f4579b7d2a6888ec6050f289744964f02e Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 25 Jun 2024 11:25:43 +0800 Subject: [PATCH 581/699] [fix][client] Fix orphan consumer when reconnection and closing are concurrency executing (#22958) (cherry picked from commit 69b2739eaa2974d93e32f6b84dd777b5112b07fa) (cherry picked from commit ad4cf9d4acfda35c40cb19b20b114a6efa9e7639) --- .../main/java/org/apache/pulsar/client/impl/ConsumerImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index bcee5c38f9a31..75326336b4446 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 @@ -1038,7 +1038,7 @@ public void connectionFailed(PulsarClientException exception) { } @Override - public CompletableFuture closeAsync() { + public synchronized CompletableFuture closeAsync() { CompletableFuture closeFuture = new CompletableFuture<>(); if (getState() == State.Closing || getState() == State.Closed) { From 2049c72417c9cc06ce0fd58e1ed2a4e94abebe02 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Tue, 16 Apr 2024 00:34:59 -0700 Subject: [PATCH 582/699] [fix][broker] Fix Replicated Topic unload bug when ExtensibleLoadManager is enabled (#22496) (cherry picked from commit 203f305bf449dd335b39501177f210cfcb73d5fa) (cherry picked from commit f467f37a75cae11841161a5a7c6e4be5671100fd) (cherry picked from commit 2557db63d48a6534951d9a988c1d7afb5fa64dce) --- .../channel/ServiceUnitStateChannelImpl.java | 13 ++++++------- .../pulsar/broker/namespace/NamespaceService.java | 5 +++++ .../service/nonpersistent/NonPersistentTopic.java | 4 +++- .../broker/service/persistent/PersistentTopic.java | 3 ++- .../broker/service/ReplicatorGlobalNSTest.java | 7 +++---- 5 files changed, 19 insertions(+), 13 deletions(-) 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 f66ed2a5c9062..7cd0e22670030 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 @@ -88,7 +88,6 @@ import org.apache.pulsar.client.api.CompressionType; 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.Schema; import org.apache.pulsar.client.api.TableView; import org.apache.pulsar.common.naming.NamespaceBundle; @@ -1338,8 +1337,8 @@ private synchronized void doCleanup(String broker) { } try { - producer.flush(); - } catch (PulsarClientException e) { + producer.flushAsync().get(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS, MILLISECONDS); + } catch (Exception e) { log.error("Failed to flush the in-flight non-system bundle override messages.", e); } @@ -1362,8 +1361,8 @@ private synchronized void doCleanup(String broker) { } try { - producer.flush(); - } catch (PulsarClientException e) { + producer.flushAsync().get(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS, MILLISECONDS); + } catch (Exception e) { log.error("Failed to flush the in-flight system bundle override messages.", e); } @@ -1541,8 +1540,8 @@ protected void monitorOwnerships(List brokers) { } try { - producer.flush(); - } catch (PulsarClientException e) { + producer.flushAsync().get(OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS, MILLISECONDS); + } catch (Exception e) { log.error("Failed to flush the in-flight messages.", e); } 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 c62e9c52a768e..30e84570ce455 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 @@ -793,6 +793,11 @@ public CompletableFuture unloadNamespaceBundle(NamespaceBundle bundle, } public CompletableFuture isNamespaceBundleOwned(NamespaceBundle bundle) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { + ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); + return extensibleLoadManager.getOwnershipAsync(Optional.empty(), bundle) + .thenApply(Optional::isPresent); + } return pulsar.getLocalMetadataStore().exists(ServiceUnitUtils.path(bundle)); } 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 23d938d22fd4b..01aec2cbb8653 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 @@ -39,6 +39,7 @@ import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.Position; import org.apache.pulsar.broker.PulsarServerException; +import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.resources.NamespaceResources; import org.apache.pulsar.broker.service.AbstractReplicator; @@ -550,7 +551,8 @@ public CompletableFuture stopReplProducers() { @Override public CompletableFuture checkReplication() { TopicName name = TopicName.get(topic); - if (!name.isGlobal() || NamespaceService.isHeartbeatNamespace(name)) { + if (!name.isGlobal() || NamespaceService.isHeartbeatNamespace(name) + || ExtensibleLoadManagerImpl.isInternalTopic(topic)) { return CompletableFuture.completedFuture(null); } 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 f2b723c38cbca..28fa026e3f54a 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 @@ -1750,7 +1750,8 @@ CompletableFuture checkPersistencePolicies() { @Override public CompletableFuture checkReplication() { TopicName name = TopicName.get(topic); - if (!name.isGlobal() || NamespaceService.isHeartbeatNamespace(name)) { + if (!name.isGlobal() || NamespaceService.isHeartbeatNamespace(name) + || ExtensibleLoadManagerImpl.isInternalTopic(topic)) { return CompletableFuture.completedFuture(null); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java index 057126e6981d4..c09809d4ad873 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorGlobalNSTest.java @@ -20,14 +20,16 @@ import static org.testng.Assert.fail; import com.google.common.collect.Sets; +import java.lang.reflect.Method; import java.util.ArrayList; import java.util.List; import java.util.concurrent.Callable; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import lombok.Cleanup; -import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.client.api.MessageRoutingMode; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.impl.ConsumerImpl; @@ -40,9 +42,6 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import java.lang.reflect.Method; -import java.util.concurrent.TimeUnit; - /** * The tests in this class should be denied in a production pulsar cluster. they are very dangerous, which leads to * a lot of topic deletion and makes namespace policies being incorrect. From 93b5fcf4bfcfc271dddfabf193fe41f94bf1bf88 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Tue, 21 May 2024 15:53:26 -0700 Subject: [PATCH 583/699] [fix][broker] Immediately tombstone Deleted and Free state bundles (#22743) (cherry picked from commit 949260f190c3ff48d16f9450083c2e8c5c9ff302) (cherry picked from commit d982d3bb4d0bf03e00976b3f4536079bab50bfe8) (cherry picked from commit 44f2e98bb2e0f33a93d46308e99dfbfc94012eed) --- .../extensions/ExtensibleLoadManagerImpl.java | 5 +- .../channel/ServiceUnitStateChannelImpl.java | 17 ++++-- .../extensions/manager/SplitManager.java | 2 +- .../extensions/manager/UnloadManager.java | 14 ++++- .../broker/namespace/NamespaceService.java | 5 +- .../ExtensibleLoadManagerImplTest.java | 59 +++++++++++++++---- .../channel/ServiceUnitStateChannelTest.java | 42 +++++++------ .../extensions/manager/SplitManagerTest.java | 27 ++------- .../extensions/manager/UnloadManagerTest.java | 27 +++++++-- 9 files changed, 127 insertions(+), 71 deletions(-) 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 855e8ccfcaaf3..fd4e94ba7774d 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 @@ -623,7 +623,8 @@ public CompletableFuture> getOwnershipWithLookupDataA } public CompletableFuture unloadNamespaceBundleAsync(ServiceUnitId bundle, - Optional destinationBroker) { + Optional destinationBroker, + boolean force) { if (NamespaceService.isSLAOrHeartbeatNamespace(bundle.getNamespaceObject().toString())) { log.info("Skip unloading namespace bundle: {}.", bundle); return CompletableFuture.completedFuture(null); @@ -642,7 +643,7 @@ public CompletableFuture unloadNamespaceBundleAsync(ServiceUnitId bundle, log.warn(msg); throw new IllegalArgumentException(msg); } - Unload unload = new Unload(sourceBroker, bundle.toString(), destinationBroker, true); + Unload unload = new Unload(sourceBroker, bundle.toString(), destinationBroker, force); UnloadDecision unloadDecision = new UnloadDecision(unload, UnloadDecision.Label.Success, UnloadDecision.Reason.Admin); return unloadAsync(unloadDecision, 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 7cd0e22670030..f959e5a13b5f2 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 @@ -826,9 +826,12 @@ private void handleFreeEvent(String serviceUnit, ServiceUnitStateData data) { } if (isTargetBroker(data.sourceBroker())) { - stateChangeListeners.notifyOnCompletion( - data.force() ? closeServiceUnit(serviceUnit) - : CompletableFuture.completedFuture(0), serviceUnit, data) + // If data.force(), try closeServiceUnit and tombstone the bundle. + CompletableFuture future = + (data.force() ? closeServiceUnit(serviceUnit) + .thenCompose(__ -> tombstoneAsync(serviceUnit)) + : CompletableFuture.completedFuture(0)).thenApply(__ -> null); + stateChangeListeners.notifyOnCompletion(future, serviceUnit, data) .whenComplete((__, e) -> log(e, serviceUnit, data, null)); } else { stateChangeListeners.notify(serviceUnit, data, null); @@ -840,9 +843,13 @@ private void handleDeleteEvent(String serviceUnit, ServiceUnitStateData data) { if (getOwnerRequest != null) { getOwnerRequest.completeExceptionally(new IllegalStateException(serviceUnit + "has been deleted.")); } - stateChangeListeners.notify(serviceUnit, data, null); + if (isTargetBroker(data.sourceBroker())) { - log(null, serviceUnit, data, null); + stateChangeListeners.notifyOnCompletion( + tombstoneAsync(serviceUnit), serviceUnit, data) + .whenComplete((__, e) -> log(e, serviceUnit, data, null)); + } else { + stateChangeListeners.notify(serviceUnit, data, null); } } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/SplitManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/SplitManager.java index 71ebbc92a87db..ac21e4c624163 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/SplitManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/SplitManager.java @@ -97,7 +97,7 @@ public void handleEvent(String serviceUnit, ServiceUnitStateData data, Throwable return; } switch (state) { - case Deleted, Owned, Init -> this.complete(serviceUnit, t); + case Init -> this.complete(serviceUnit, t); default -> { if (log.isDebugEnabled()) { log.debug("Handling {} for service unit {}", data, serviceUnit); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java index bf9885b2a252e..742b23dc2d2ec 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java @@ -93,7 +93,7 @@ public CompletableFuture waitAsync(CompletableFuture eventPubFuture, public void handleEvent(String serviceUnit, ServiceUnitStateData data, Throwable t) { ServiceUnitState state = ServiceUnitStateData.state(data); - if (StringUtils.isBlank(data.sourceBroker()) && (state == Owned || state == Assigning)) { + if ((state == Owned || state == Assigning) && StringUtils.isBlank(data.sourceBroker())) { if (log.isDebugEnabled()) { log.debug("Skipping {} for service unit {} from the assignment command.", data, serviceUnit); } @@ -113,7 +113,17 @@ public void handleEvent(String serviceUnit, ServiceUnitStateData data, Throwable } switch (state) { - case Free, Owned -> this.complete(serviceUnit, t); + case Free -> { + if (!data.force()) { + complete(serviceUnit, t); + } + } + case Init -> { + if (data.force()) { + complete(serviceUnit, t); + } + } + case Owned -> complete(serviceUnit, t); default -> { if (log.isDebugEnabled()) { log.debug("Handling {} for service unit {}", data, serviceUnit); 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 30e84570ce455..8eb83eecf3ae0 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 @@ -781,7 +781,7 @@ public CompletableFuture unloadNamespaceBundle(NamespaceBundle bundle, boolean closeWithoutWaitingClientDisconnect) { if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return ExtensibleLoadManagerImpl.get(loadManager.get()) - .unloadNamespaceBundleAsync(bundle, destinationBroker); + .unloadNamespaceBundleAsync(bundle, destinationBroker, false); } // unload namespace bundle OwnedBundle ob = ownershipCache.getOwnedBundle(bundle); @@ -1229,7 +1229,8 @@ public CompletableFuture removeOwnedServiceUnitAsync(NamespaceBundle nsBun CompletableFuture future; if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); - future = extensibleLoadManager.unloadNamespaceBundleAsync(nsBundle, Optional.empty()); + future = extensibleLoadManager.unloadNamespaceBundleAsync( + nsBundle, Optional.empty(), true); } else { future = ownershipCache.removeOwnership(nsBundle); } 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 d5aaed3436824..e6f69dd862991 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 @@ -358,9 +358,12 @@ private void checkOwnershipState(String broker, NamespaceBundle bundle) @Test(timeOut = 30 * 1000) public void testSplitBundleAdminAPI() throws Exception { - String namespace = defaultTestNamespace; - String topic = "persistent://" + namespace + "/test-split"; - admin.topics().createPartitionedTopic(topic, 10); + + final String namespace = "public/testSplitBundleAdminAPI"; + admin.namespaces().createNamespace(namespace, 1); + Pair topicAndBundle = getBundleIsNotOwnByChangeEventTopic("test-split"); + TopicName topicName = topicAndBundle.getLeft(); + admin.topics().createPartitionedTopic(topicName.toString(), 10); BundlesData bundles = admin.namespaces().getBundles(namespace); int numBundles = bundles.getNumBundles(); var bundleRanges = bundles.getBoundaries().stream().map(Long::decode).sorted().toList(); @@ -370,15 +373,18 @@ public void testSplitBundleAdminAPI() throws Exception { long mid = bundleRanges.get(0) + (bundleRanges.get(1) - bundleRanges.get(0)) / 2; admin.namespaces().splitNamespaceBundle(namespace, firstBundle, true, null); - - BundlesData bundlesData = admin.namespaces().getBundles(namespace); - assertEquals(bundlesData.getNumBundles(), numBundles + 1); - String lowBundle = String.format("0x%08x", bundleRanges.get(0)); - String midBundle = String.format("0x%08x", mid); - String highBundle = String.format("0x%08x", bundleRanges.get(1)); - assertTrue(bundlesData.getBoundaries().contains(lowBundle)); - assertTrue(bundlesData.getBoundaries().contains(midBundle)); - assertTrue(bundlesData.getBoundaries().contains(highBundle)); + Awaitility.await() + .atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> { + BundlesData bundlesData = admin.namespaces().getBundles(namespace); + assertEquals(bundlesData.getNumBundles(), numBundles + 1); + String lowBundle = String.format("0x%08x", bundleRanges.get(0)); + String midBundle = String.format("0x%08x", mid); + String highBundle = String.format("0x%08x", bundleRanges.get(1)); + assertTrue(bundlesData.getBoundaries().contains(lowBundle)); + assertTrue(bundlesData.getBoundaries().contains(midBundle)); + assertTrue(bundlesData.getBoundaries().contains(highBundle)); + }); // Test split bundle with invalid bundle range. try { @@ -387,6 +393,29 @@ public void testSplitBundleAdminAPI() throws Exception { } catch (PulsarAdminException ex) { assertTrue(ex.getMessage().contains("Invalid bundle range")); } + + + // delete and retry + Awaitility.await() + .atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> { + admin.namespaces().deleteNamespace(namespace); + }); + admin.namespaces().createNamespace(namespace, 1); + admin.namespaces().splitNamespaceBundle(namespace, firstBundle, true, null); + + Awaitility.await() + .atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> { + BundlesData bundlesData = admin.namespaces().getBundles(namespace); + assertEquals(bundlesData.getNumBundles(), numBundles + 1); + String lowBundle = String.format("0x%08x", bundleRanges.get(0)); + String midBundle = String.format("0x%08x", mid); + String highBundle = String.format("0x%08x", bundleRanges.get(1)); + assertTrue(bundlesData.getBoundaries().contains(lowBundle)); + assertTrue(bundlesData.getBoundaries().contains(midBundle)); + assertTrue(bundlesData.getBoundaries().contains(highBundle)); + }); } @Test(timeOut = 30 * 1000) @@ -1202,7 +1231,11 @@ public void testTryAcquiringOwnership() NamespaceEphemeralData namespaceEphemeralData = primaryLoadManager.tryAcquiringOwnership(bundle).get(); assertTrue(Set.of(pulsar1.getBrokerServiceUrl(), pulsar2.getBrokerServiceUrl()) .contains(namespaceEphemeralData.getNativeUrl())); - admin.namespaces().deleteNamespace(namespace); + Awaitility.await() + .atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> { + admin.namespaces().deleteNamespace(namespace, true); + }); } @Test(timeOut = 30 * 1000) 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 ceb58e8d9647c..72f8827632bd3 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 @@ -573,11 +573,11 @@ public void splitAndRetryTest() throws Exception { childBundle1Range, Optional.empty(), childBundle2Range, Optional.empty())); channel1.publishSplitEventAsync(split); - waitUntilState(channel1, bundle, Deleted); - waitUntilState(channel2, bundle, Deleted); + waitUntilState(channel1, bundle, Init); + waitUntilState(channel2, bundle, Init); - validateHandlerCounters(channel1, 1, 0, 3, 0, 0, 0, 1, 0, 0, 0, 0, 0, 1, 0); - validateHandlerCounters(channel2, 1, 0, 3, 0, 0, 0, 1, 0, 0, 0, 0, 0, 1, 0); + validateHandlerCounters(channel1, 1, 0, 3, 0, 0, 0, 1, 0, 0, 0, 1, 0, 1, 0); + validateHandlerCounters(channel2, 1, 0, 3, 0, 0, 0, 1, 0, 0, 0, 1, 0, 1, 0); validateEventCounters(channel1, 1, 0, 1, 0, 0, 0); validateEventCounters(channel2, 0, 0, 0, 0, 0, 0); // Verify the retry count @@ -617,7 +617,7 @@ public void splitAndRetryTest() throws Exception { var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; validateMonitorCounters(leader, 0, - 1, + 0, 0, 0, 0, @@ -1233,15 +1233,15 @@ public void splitTestWhenProducerFails() var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; - waitUntilStateWithMonitor(leader, bundle, Deleted); - waitUntilStateWithMonitor(channel1, bundle, Deleted); - waitUntilStateWithMonitor(channel2, bundle, Deleted); + waitUntilStateWithMonitor(leader, bundle, Init); + waitUntilStateWithMonitor(channel1, bundle, Init); + waitUntilStateWithMonitor(channel2, bundle, Init); var ownerAddr1 = channel1.getOwnerAsync(bundle); var ownerAddr2 = channel2.getOwnerAsync(bundle); - assertTrue(ownerAddr1.isCompletedExceptionally()); - assertTrue(ownerAddr2.isCompletedExceptionally()); + assertTrue(ownerAddr1.get().isEmpty()); + assertTrue(ownerAddr2.get().isEmpty()); FieldUtils.writeDeclaredField(channel1, @@ -1425,13 +1425,15 @@ public void splitAndRetryFailureTest() throws Exception { var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; ((ServiceUnitStateChannelImpl) leader) .monitorOwnerships(List.of(brokerId1, brokerId2)); - waitUntilState(leader, bundle3, Deleted); - waitUntilState(channel1, bundle3, Deleted); - waitUntilState(channel2, bundle3, Deleted); + + waitUntilState(leader, bundle3, Init); + waitUntilState(channel1, bundle3, Init); + waitUntilState(channel2, bundle3, Init); - validateHandlerCounters(channel1, 1, 0, 3, 0, 0, 0, 2, 1, 0, 0, 0, 0, 1, 0); - validateHandlerCounters(channel2, 1, 0, 3, 0, 0, 0, 2, 0, 0, 0, 0, 0, 1, 0); + + validateHandlerCounters(channel1, 1, 0, 3, 0, 0, 0, 2, 1, 0, 0, 1, 0, 1, 0); + validateHandlerCounters(channel2, 1, 0, 3, 0, 0, 0, 2, 0, 0, 0, 1, 0, 1, 0); validateEventCounters(channel1, 1, 0, 1, 0, 0, 0); validateEventCounters(channel2, 0, 0, 0, 0, 0, 0); @@ -1461,7 +1463,7 @@ public void splitAndRetryFailureTest() throws Exception { validateMonitorCounters(leader, 0, - 1, + 0, 1, 0, 0, @@ -1539,7 +1541,7 @@ public void testOverrideInactiveBrokerStateData() waitUntilNewOwner(channel2, ownedBundle, brokerId2); assertEquals(Optional.empty(), channel2.getOwnerAsync(freeBundle).get()); assertTrue(channel2.getOwnerAsync(deletedBundle).isCompletedExceptionally()); - assertTrue(channel2.getOwnerAsync(splittingBundle).isCompletedExceptionally()); + assertTrue(channel2.getOwnerAsync(splittingBundle).get().isEmpty()); // clean-up FieldUtils.writeDeclaredField(leaderChannel, "maxCleanupDelayTimeInSecs", 3 * 60, true); @@ -1602,7 +1604,7 @@ public void testOverrideOrphanStateData() waitUntilNewOwner(channel2, ownedBundle, broker); assertEquals(Optional.empty(), channel2.getOwnerAsync(freeBundle).get()); assertTrue(channel2.getOwnerAsync(deletedBundle).isCompletedExceptionally()); - assertTrue(channel2.getOwnerAsync(splittingBundle).isCompletedExceptionally()); + assertTrue(channel2.getOwnerAsync(splittingBundle).get().isEmpty()); // clean-up FieldUtils.writeDeclaredField(channel1, @@ -1660,8 +1662,10 @@ public void testActiveGetOwner() throws Exception { "inFlightStateWaitingTimeInMillis", 20 * 1000, true); start = System.currentTimeMillis(); assertTrue(channel1.getOwnerAsync(bundle).get().isEmpty()); - assertTrue(System.currentTimeMillis() - start < 20_000); + waitUntilState(channel1, bundle, Init); + waitUntilState(channel2, bundle, Init); + assertTrue(System.currentTimeMillis() - start < 20_000); // simulate ownership cleanup(brokerId1 selected owner) by the leader channel overrideTableViews(bundle, new ServiceUnitStateData(Owned, broker, null, 1)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/SplitManagerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/SplitManagerTest.java index 3287306ab48ba..57b7830214b92 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/SplitManagerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/SplitManagerTest.java @@ -123,40 +123,23 @@ public void testSuccess() throws IllegalAccessException, ExecutionException, Int manager.handleEvent(bundle, new ServiceUnitStateData(ServiceUnitState.Free, dstBroker, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequests.size(), 1); - assertEquals(counter.toMetrics(null).toString(), - counterExpected.toMetrics(null).toString()); manager.handleEvent(bundle, new ServiceUnitStateData(ServiceUnitState.Deleted, dstBroker, VERSION_ID_INIT), null); - counterExpected.update(SplitDecision.Label.Success, Sessions); - assertEquals(inFlightUnloadRequests.size(), 0); - assertEquals(counter.toMetrics(null).toString(), - counterExpected.toMetrics(null).toString()); + assertEquals(inFlightUnloadRequests.size(), 1); - // Success with Init state. - future = manager.waitAsync(CompletableFuture.completedFuture(null), - bundle, decision, 5, TimeUnit.SECONDS); - inFlightUnloadRequests = getinFlightUnloadRequests(manager); + manager.handleEvent(bundle, + new ServiceUnitStateData(ServiceUnitState.Owned, dstBroker, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequests.size(), 1); + + // Success with Init state. manager.handleEvent(bundle, new ServiceUnitStateData(ServiceUnitState.Init, dstBroker, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequests.size(), 0); counterExpected.update(SplitDecision.Label.Success, Sessions); assertEquals(counter.toMetrics(null).toString(), counterExpected.toMetrics(null).toString()); - future.get(); - // Success with Owned state. - future = manager.waitAsync(CompletableFuture.completedFuture(null), - bundle, decision, 5, TimeUnit.SECONDS); - inFlightUnloadRequests = getinFlightUnloadRequests(manager); - assertEquals(inFlightUnloadRequests.size(), 1); - manager.handleEvent(bundle, - new ServiceUnitStateData(ServiceUnitState.Owned, dstBroker, VERSION_ID_INIT), null); - assertEquals(inFlightUnloadRequests.size(), 0); - counterExpected.update(SplitDecision.Label.Success, Sessions); - assertEquals(counter.toMetrics(null).toString(), - counterExpected.toMetrics(null).toString()); future.get(); } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManagerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManagerTest.java index 45b1cd9544f91..b7dae62062944 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManagerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManagerTest.java @@ -123,11 +123,15 @@ public void testSuccess() throws IllegalAccessException, ExecutionException, Int assertEquals(inFlightUnloadRequestMap.size(), 1); manager.handleEvent(bundle, - new ServiceUnitStateData(ServiceUnitState.Init, null, srcBroker, VERSION_ID_INIT), null); + new ServiceUnitStateData(ServiceUnitState.Free, null, srcBroker, true, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequestMap.size(), 1); + // Success with Init state. manager.handleEvent(bundle, - new ServiceUnitStateData(ServiceUnitState.Free, null, srcBroker, VERSION_ID_INIT), null); + new ServiceUnitStateData(ServiceUnitState.Init, null, srcBroker, false, VERSION_ID_INIT), null); + assertEquals(inFlightUnloadRequestMap.size(), 1); + manager.handleEvent(bundle, + new ServiceUnitStateData(ServiceUnitState.Init, null, srcBroker, true, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequestMap.size(), 0); future.get(); assertEquals(counter.getBreakdownCounters().get(Success).get(Admin).get(), 1); @@ -137,17 +141,30 @@ public void testSuccess() throws IllegalAccessException, ExecutionException, Int bundle, unloadDecision, 5, TimeUnit.SECONDS); inFlightUnloadRequestMap = getInFlightUnloadRequestMap(manager); assertEquals(inFlightUnloadRequestMap.size(), 1); - manager.handleEvent(bundle, new ServiceUnitStateData(ServiceUnitState.Owned, dstBroker, null, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequestMap.size(), 1); - manager.handleEvent(bundle, new ServiceUnitStateData(ServiceUnitState.Owned, dstBroker, srcBroker, VERSION_ID_INIT), null); assertEquals(inFlightUnloadRequestMap.size(), 0); - future.get(); assertEquals(counter.getBreakdownCounters().get(Success).get(Admin).get(), 2); + + // Success with Free state. + future = manager.waitAsync(CompletableFuture.completedFuture(null), + bundle, unloadDecision, 5, TimeUnit.SECONDS); + inFlightUnloadRequestMap = getInFlightUnloadRequestMap(manager); + assertEquals(inFlightUnloadRequestMap.size(), 1); + manager.handleEvent(bundle, + new ServiceUnitStateData(ServiceUnitState.Free, dstBroker, srcBroker, true, VERSION_ID_INIT), null); + assertEquals(inFlightUnloadRequestMap.size(), 1); + manager.handleEvent(bundle, + new ServiceUnitStateData(ServiceUnitState.Free, dstBroker, srcBroker, false, VERSION_ID_INIT), null); + assertEquals(inFlightUnloadRequestMap.size(), 0); + future.get(); + assertEquals(counter.getBreakdownCounters().get(Success).get(Admin).get(), 3); + + } @Test From d26f20d2f9e9c4cc534f51038bf1e70a556eb473 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 7 Jun 2024 11:09:09 +0800 Subject: [PATCH 584/699] [fix][broker] Fix NPE after publishing a tombstone to the service unit channel (#22859) (cherry picked from commit 9326a08eb173b8a7410bcb00c4ab7d3602064b4a) (cherry picked from commit 6eac7e5aa7eea37b449de97e93eac878ff11ceca) (cherry picked from commit 9ffbffc113b87cfd8171bce59a56cf73287cbc36) --- .../loadbalance/extensions/manager/UnloadManager.java | 6 +++--- .../loadbalance/extensions/manager/UnloadManagerTest.java | 6 +----- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java index 742b23dc2d2ec..a905803c95ddd 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManager.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.loadbalance.extensions.manager; +import static com.google.common.base.Preconditions.checkArgument; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigning; import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; import static org.apache.pulsar.broker.loadbalance.extensions.models.UnloadDecision.Label.Failure; @@ -119,9 +120,8 @@ public void handleEvent(String serviceUnit, ServiceUnitStateData data, Throwable } } case Init -> { - if (data.force()) { - complete(serviceUnit, t); - } + checkArgument(data == null, "Init state must be associated with null data"); + complete(serviceUnit, t); } case Owned -> complete(serviceUnit, t); default -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManagerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManagerTest.java index b7dae62062944..06cfb0d970549 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManagerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/manager/UnloadManagerTest.java @@ -127,11 +127,7 @@ public void testSuccess() throws IllegalAccessException, ExecutionException, Int assertEquals(inFlightUnloadRequestMap.size(), 1); // Success with Init state. - manager.handleEvent(bundle, - new ServiceUnitStateData(ServiceUnitState.Init, null, srcBroker, false, VERSION_ID_INIT), null); - assertEquals(inFlightUnloadRequestMap.size(), 1); - manager.handleEvent(bundle, - new ServiceUnitStateData(ServiceUnitState.Init, null, srcBroker, true, VERSION_ID_INIT), null); + manager.handleEvent(bundle, null, null); assertEquals(inFlightUnloadRequestMap.size(), 0); future.get(); assertEquals(counter.getBreakdownCounters().get(Success).get(Admin).get(), 1); From cd5b194bc8b7dbb9940cba7ce8750bb55eaaf6e1 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Thu, 13 Jun 2024 12:26:40 -0700 Subject: [PATCH 585/699] [fix][broker] Asynchronously return brokerRegistry.lookupAsync when checking if broker is active(ExtensibleLoadManagerImpl only) (#22899) (cherry picked from commit c2702e9bc46c444cbc99f4b64cb453c622b56c26) (cherry picked from commit 2cf6e51f84b226b6e8aabea2aa5ad7ebfb94f207) (cherry picked from commit 9ab3f38763c86a7830e3225efecf1788510cb042) --- .../channel/ServiceUnitStateChannelImpl.java | 84 +++++++++++-------- .../channel/ServiceUnitStateChannelTest.java | 51 +++++++++-- 2 files changed, 89 insertions(+), 46 deletions(-) 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 f959e5a13b5f2..6c3c5f35dce60 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 @@ -491,7 +491,7 @@ private CompletableFuture> getActiveOwnerAsync( String serviceUnit, ServiceUnitState state, Optional owner) { - return deferGetOwnerRequest(serviceUnit) + return dedupeGetOwnerRequest(serviceUnit) .thenCompose(newOwner -> { if (newOwner == null) { return CompletableFuture.completedFuture(null); @@ -594,7 +594,7 @@ public CompletableFuture publishAssignEventAsync(String serviceUnit, Str } EventType eventType = Assign; eventCounters.get(eventType).getTotal().incrementAndGet(); - CompletableFuture getOwnerRequest = deferGetOwnerRequest(serviceUnit); + CompletableFuture getOwnerRequest = dedupeGetOwnerRequest(serviceUnit); pubAsync(serviceUnit, new ServiceUnitStateData(Assigning, broker, getNextVersionId(serviceUnit))) .whenComplete((__, ex) -> { @@ -891,44 +891,54 @@ private boolean isTargetBroker(String broker) { return broker.equals(brokerId); } - private CompletableFuture deferGetOwnerRequest(String serviceUnit) { + private CompletableFuture deferGetOwner(String serviceUnit) { + var future = new CompletableFuture().orTimeout(inFlightStateWaitingTimeInMillis, + TimeUnit.MILLISECONDS) + .exceptionally(e -> { + var ownerAfter = getOwner(serviceUnit); + log.warn("{} failed to wait for owner for serviceUnit:{}; Trying to " + + "return the current owner:{}", + brokerId, serviceUnit, ownerAfter, e); + if (ownerAfter == null) { + throw new IllegalStateException(e); + } + return ownerAfter.orElse(null); + }); + if (debug()) { + log.info("{} is waiting for owner for serviceUnit:{}", brokerId, serviceUnit); + } + return future; + } + + private CompletableFuture dedupeGetOwnerRequest(String serviceUnit) { var requested = new MutableObject>(); try { - return getOwnerRequests - .computeIfAbsent(serviceUnit, k -> { - var ownerBefore = getOwner(serviceUnit); - if (ownerBefore != null && ownerBefore.isPresent()) { - // Here, we do a quick active check first with the computeIfAbsent lock - brokerRegistry.lookupAsync(ownerBefore.get()).getNow(Optional.empty()) - .ifPresent(__ -> requested.setValue( - CompletableFuture.completedFuture(ownerBefore.get()))); - - if (requested.getValue() != null) { - return requested.getValue(); - } - } - - - CompletableFuture future = - new CompletableFuture().orTimeout(inFlightStateWaitingTimeInMillis, - TimeUnit.MILLISECONDS) - .exceptionally(e -> { - var ownerAfter = getOwner(serviceUnit); - log.warn("{} failed to wait for owner for serviceUnit:{}; Trying to " - + "return the current owner:{}", - brokerId, serviceUnit, ownerAfter, e); - if (ownerAfter == null) { - throw new IllegalStateException(e); - } - return ownerAfter.orElse(null); - }); - if (debug()) { - log.info("{} is waiting for owner for serviceUnit:{}", brokerId, serviceUnit); - } - requested.setValue(future); - return future; - }); + return getOwnerRequests.computeIfAbsent(serviceUnit, k -> { + var ownerBefore = getOwner(serviceUnit); + if (ownerBefore != null && ownerBefore.isPresent()) { + // Here, we do the broker active check first with the computeIfAbsent lock + requested.setValue(brokerRegistry.lookupAsync(ownerBefore.get()) + .thenCompose(brokerLookupData -> { + if (brokerLookupData.isPresent()) { + // The owner broker is active. + // Immediately return the request. + return CompletableFuture.completedFuture(ownerBefore.get()); + } else { + // The owner broker is inactive. + // The leader broker should be cleaning up the orphan service units. + // Defer this request til the leader notifies the new ownerships. + return deferGetOwner(serviceUnit); + } + })); + } else { + // The owner broker has not been declared yet. + // The ownership should be in the middle of transferring or assigning. + // Defer this request til the inflight ownership change is complete. + requested.setValue(deferGetOwner(serviceUnit)); + } + return requested.getValue(); + }); } finally { var future = requested.getValue(); if (future != null) { 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 72f8827632bd3..10ba6d2832b4f 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 @@ -1617,32 +1617,63 @@ public void testOverrideOrphanStateData() @Test(priority = 19) public void testActiveGetOwner() throws Exception { - - // set the bundle owner is the broker + // case 1: the bundle owner is empty String broker = brokerId2; String bundle = "public/owned/0xfffffff0_0xffffffff"; + overrideTableViews(bundle, null); + assertEquals(Optional.empty(), channel1.getOwnerAsync(bundle).get()); + + // case 2: the bundle ownership is transferring, and the dst broker is not the channel owner + overrideTableViews(bundle, + new ServiceUnitStateData(Releasing, broker, brokerId1, 1)); + assertEquals(Optional.of(broker), channel1.getOwnerAsync(bundle).get()); + + + // case 3: the bundle ownership is transferring, and the dst broker is the channel owner + overrideTableViews(bundle, + new ServiceUnitStateData(Assigning, brokerId1, brokerId2, 1)); + assertTrue(!channel1.getOwnerAsync(bundle).isDone()); + + // case 4: the bundle ownership is found overrideTableViews(bundle, new ServiceUnitStateData(Owned, broker, null, 1)); var owner = channel1.getOwnerAsync(bundle).get(5, TimeUnit.SECONDS).get(); assertEquals(owner, broker); - // simulate the owner is inactive + // case 5: the owner lookup gets delayed var spyRegistry = spy(new BrokerRegistryImpl(pulsar)); - doReturn(CompletableFuture.completedFuture(Optional.empty())) - .when(spyRegistry).lookupAsync(eq(broker)); FieldUtils.writeDeclaredField(channel1, "brokerRegistry", spyRegistry , true); FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 1000, true); + var delayedFuture = new CompletableFuture(); + doReturn(delayedFuture).when(spyRegistry).lookupAsync(eq(broker)); + CompletableFuture.runAsync(() -> { + try { + Thread.sleep(500); + } catch (InterruptedException e) { + Thread.currentThread().interrupt();; + } + delayedFuture.complete(Optional.of(broker)); + }); - - // verify getOwnerAsync times out because the owner is inactive now. + // verify the owner eventually returns in inFlightStateWaitingTimeInMillis. long start = System.currentTimeMillis(); + assertEquals(broker, channel1.getOwnerAsync(bundle).get().get()); + long elapsed = System.currentTimeMillis() - start; + assertTrue(elapsed < 1000); + + // case 6: the owner is inactive + doReturn(CompletableFuture.completedFuture(Optional.empty())) + .when(spyRegistry).lookupAsync(eq(broker)); + + // verify getOwnerAsync times out + start = System.currentTimeMillis(); var ex = expectThrows(ExecutionException.class, () -> channel1.getOwnerAsync(bundle).get()); assertTrue(ex.getCause() instanceof IllegalStateException); assertTrue(System.currentTimeMillis() - start >= 1000); - // simulate ownership cleanup(no selected owner) by the leader channel + // case 7: the ownership cleanup(no new owner) by the leader channel doReturn(CompletableFuture.completedFuture(Optional.empty())) .when(loadManager).selectAsync(any(), any()); var leaderChannel = channel1; @@ -1666,7 +1697,8 @@ public void testActiveGetOwner() throws Exception { waitUntilState(channel2, bundle, Init); assertTrue(System.currentTimeMillis() - start < 20_000); - // simulate ownership cleanup(brokerId1 selected owner) by the leader channel + + // case 8: simulate ownership cleanup(brokerId1 as the new owner) by the leader channel overrideTableViews(bundle, new ServiceUnitStateData(Owned, broker, null, 1)); doReturn(CompletableFuture.completedFuture(Optional.of(brokerId1))) @@ -1691,6 +1723,7 @@ public void testActiveGetOwner() throws Exception { } + private static ConcurrentHashMap>> getOwnerRequests( ServiceUnitStateChannel channel) throws IllegalAccessException { return (ConcurrentHashMap>>) From eeb084f312a3b3ecae0ff29faf7ebd1d0728b160 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Tue, 18 Jun 2024 08:43:40 -0700 Subject: [PATCH 586/699] [fix][broker] Update init and shutdown time and other minor logic (ExtensibleLoadManagerImpl only) (#22930) (cherry picked from commit aa8f696b8e17a49d1a7ff6cdc25f1d86e7c4a8ed) (cherry picked from commit 9b6156ab452756ad8356f94793c785901c69de8e) (cherry picked from commit d9c51322ce5566cc706e00dfe12b4976396656f5) --- .../pulsar/PulsarClusterMetadataSetup.java | 4 ++-- .../extensions/ExtensibleLoadManagerImpl.java | 6 +++-- .../channel/ServiceUnitStateChannelImpl.java | 23 +++++++++++++------ .../store/TableViewLoadDataStoreImpl.java | 12 ++++++---- .../broker/namespace/NamespaceService.java | 5 ++-- 5 files changed, 32 insertions(+), 18 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java index 96ebadb1ff4aa..8cad6f8bb2d77 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/PulsarClusterMetadataSetup.java @@ -373,8 +373,8 @@ static void createTenantIfAbsent(PulsarResources resources, String tenant, Strin } } - static void createNamespaceIfAbsent(PulsarResources resources, NamespaceName namespaceName, - String cluster, int bundleNumber) throws IOException { + public static void createNamespaceIfAbsent(PulsarResources resources, NamespaceName namespaceName, + String cluster, int bundleNumber) throws IOException { NamespaceResources namespaceResources = resources.getNamespaceResources(); if (!namespaceResources.namespaceExists(namespaceName)) { 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 fd4e94ba7774d..1c295fe0561ca 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 @@ -624,7 +624,9 @@ public CompletableFuture> getOwnershipWithLookupDataA public CompletableFuture unloadNamespaceBundleAsync(ServiceUnitId bundle, Optional destinationBroker, - boolean force) { + boolean force, + long timeout, + TimeUnit timeoutUnit) { if (NamespaceService.isSLAOrHeartbeatNamespace(bundle.getNamespaceObject().toString())) { log.info("Skip unloading namespace bundle: {}.", bundle); return CompletableFuture.completedFuture(null); @@ -647,7 +649,7 @@ public CompletableFuture unloadNamespaceBundleAsync(ServiceUnitId bundle, UnloadDecision unloadDecision = new UnloadDecision(unload, UnloadDecision.Label.Success, UnloadDecision.Reason.Admin); return unloadAsync(unloadDecision, - conf.getNamespaceBundleUnloadingTimeoutMs(), TimeUnit.MILLISECONDS); + timeout, timeoutUnit); }); } 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 6c3c5f35dce60..0feed5e833327 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 @@ -115,7 +115,6 @@ public class ServiceUnitStateChannelImpl implements ServiceUnitStateChannel { private static final int OWNERSHIP_CLEAN_UP_MAX_WAIT_TIME_IN_MILLIS = 5000; private static final int OWNERSHIP_CLEAN_UP_WAIT_RETRY_DELAY_IN_MILLIS = 100; - private static final int OWNERSHIP_CLEAN_UP_CONVERGENCE_DELAY_IN_MILLIS = 3000; public static final long VERSION_ID_INIT = 1; // initial versionId private static final long OWNERSHIP_MONITOR_DELAY_TIME_IN_SECS = 60; public static final long MAX_CLEAN_UP_DELAY_TIME_IN_SECS = 3 * 60; // 3 mins @@ -305,7 +304,8 @@ public synchronized void start() throws PulsarServerException { } } PulsarClusterMetadataSetup.createNamespaceIfAbsent - (pulsar.getPulsarResources(), SYSTEM_NAMESPACE, config.getClusterName()); + (pulsar.getPulsarResources(), SYSTEM_NAMESPACE, config.getClusterName(), + config.getDefaultNumberOfNamespaceBundles()); ExtensibleLoadManagerImpl.createSystemTopic(pulsar, TOPIC); @@ -1315,11 +1315,6 @@ private void waitForCleanups(String broker, boolean excludeSystemTopics, int max } } if (cleaned) { - try { - MILLISECONDS.sleep(OWNERSHIP_CLEAN_UP_CONVERGENCE_DELAY_IN_MILLIS); - } catch (InterruptedException e) { - log.warn("Interrupted while gracefully waiting for the cleanup convergence."); - } break; } else { try { @@ -1330,9 +1325,23 @@ private void waitForCleanups(String broker, boolean excludeSystemTopics, int max } } } + log.info("Finished cleanup waiting for orphan broker:{}. Elapsed {} ms", brokerId, + System.currentTimeMillis() - started); } private synchronized void doCleanup(String broker) { + try { + if (getChannelOwnerAsync().get(MAX_CHANNEL_OWNER_ELECTION_WAITING_TIME_IN_SECS, TimeUnit.SECONDS) + .isEmpty()) { + log.error("Found the channel owner is empty. Skip the inactive broker:{}'s orphan bundle cleanup", + broker); + return; + } + } catch (Exception e) { + log.error("Failed to find the channel owner. Skip the inactive broker:{}'s orphan bundle cleanup", broker); + return; + } + long startTime = System.nanoTime(); log.info("Started ownership cleanup for the inactive broker:{}", broker); int orphanServiceUnitCleanupCnt = 0; 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 81cf33b4a55d2..e9289d3ccdac2 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 @@ -31,7 +31,6 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.TableView; @@ -44,6 +43,7 @@ public class TableViewLoadDataStoreImpl implements LoadDataStore { private static final long LOAD_DATA_REPORT_UPDATE_MAX_INTERVAL_MULTIPLIER_BEFORE_RESTART = 2; + private static final long INIT_TIMEOUT_IN_SECS = 5; private volatile TableView tableView; private volatile long tableViewLastUpdateTimestamp; @@ -123,10 +123,11 @@ public synchronized void start() throws LoadDataStoreException { public synchronized void startTableView() throws LoadDataStoreException { if (tableView == null) { try { - tableView = client.newTableViewBuilder(Schema.JSON(clazz)).topic(topic).create(); + tableView = client.newTableViewBuilder(Schema.JSON(clazz)).topic(topic).createAsync() + .get(INIT_TIMEOUT_IN_SECS, TimeUnit.SECONDS); tableView.forEachAndListen((k, v) -> tableViewLastUpdateTimestamp = System.currentTimeMillis()); - } catch (PulsarClientException e) { + } catch (Exception e) { tableView = null; throw new LoadDataStoreException(e); } @@ -137,8 +138,9 @@ public synchronized void startTableView() throws LoadDataStoreException { public synchronized void startProducer() throws LoadDataStoreException { if (producer == null) { try { - producer = client.newProducer(Schema.JSON(clazz)).topic(topic).create(); - } catch (PulsarClientException e) { + producer = client.newProducer(Schema.JSON(clazz)).topic(topic).createAsync() + .get(INIT_TIMEOUT_IN_SECS, TimeUnit.SECONDS); + } catch (Exception e) { producer = null; throw new LoadDataStoreException(e); } 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 8eb83eecf3ae0..8f3a30a59f073 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 @@ -781,7 +781,7 @@ public CompletableFuture unloadNamespaceBundle(NamespaceBundle bundle, boolean closeWithoutWaitingClientDisconnect) { if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { return ExtensibleLoadManagerImpl.get(loadManager.get()) - .unloadNamespaceBundleAsync(bundle, destinationBroker, false); + .unloadNamespaceBundleAsync(bundle, destinationBroker, false, timeout, timeoutUnit); } // unload namespace bundle OwnedBundle ob = ownershipCache.getOwnedBundle(bundle); @@ -1230,7 +1230,8 @@ public CompletableFuture removeOwnedServiceUnitAsync(NamespaceBundle nsBun if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { ExtensibleLoadManagerImpl extensibleLoadManager = ExtensibleLoadManagerImpl.get(loadManager.get()); future = extensibleLoadManager.unloadNamespaceBundleAsync( - nsBundle, Optional.empty(), true); + nsBundle, Optional.empty(), true, + pulsar.getConfig().getNamespaceBundleUnloadingTimeoutMs(), TimeUnit.MILLISECONDS); } else { future = ownershipCache.removeOwnership(nsBundle); } From b691da957ff56cb72f7effce96fa5719e5335558 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 26 Jun 2024 17:54:19 +0300 Subject: [PATCH 587/699] [fix][broker] Ensure that PulsarService is ready for serving incoming requests (#22977) (cherry picked from commit 53df683b0f78f5f7c12f87e6fbb4d73637ca5bd5) (cherry picked from commit ec51420ff8e7e05e37fd55feb0ab13b123c2a151) (cherry picked from commit 1a7eb540fc0c84a1be06867ec06c191faf57983c) --- .../apache/pulsar/broker/PulsarService.java | 16 ++- .../extensions/ExtensibleLoadManagerImpl.java | 131 +++++++++++------- .../broker/namespace/NamespaceService.java | 4 +- .../service/PulsarChannelInitializer.java | 7 +- .../pulsar/broker/service/ServerCnx.java | 4 + .../apache/pulsar/broker/web/WebService.java | 53 ++++++- 6 files changed, 157 insertions(+), 58 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 58018be20b642..6f68c1586dd0d 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 @@ -273,6 +273,7 @@ public class PulsarService implements AutoCloseable, ShutdownService { private TransactionPendingAckStoreProvider transactionPendingAckStoreProvider; private final ExecutorProvider transactionExecutorProvider; private String brokerId; + private final CompletableFuture readyForIncomingRequestsFuture = new CompletableFuture<>(); public enum State { Init, Started, Closing, Closed @@ -901,6 +902,9 @@ public void start() throws PulsarServerException { this.metricsGenerator = new MetricsGenerator(this); + // the broker is ready to accept incoming requests by Pulsar binary protocol and http/https + readyForIncomingRequestsFuture.complete(null); + // Initialize the message protocol handlers. // start the protocol handlers only after the broker is ready, // so that the protocol handlers can access broker service properly. @@ -949,12 +953,22 @@ public void start() throws PulsarServerException { state = State.Started; } catch (Exception e) { LOG.error("Failed to start Pulsar service: {}", e.getMessage(), e); - throw new PulsarServerException(e); + PulsarServerException startException = new PulsarServerException(e); + readyForIncomingRequestsFuture.completeExceptionally(startException); + throw startException; } finally { mutex.unlock(); } } + public void runWhenReadyForIncomingRequests(Runnable runnable) { + readyForIncomingRequestsFuture.thenRun(runnable); + } + + public void waitUntilReadyForIncomingRequests() throws ExecutionException, InterruptedException { + readyForIncomingRequestsFuture.get(); + } + protected BrokerInterceptor newBrokerInterceptor() throws IOException { return BrokerInterceptors.load(config); } 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 1c295fe0561ca..fb33ff85521ff 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 @@ -36,7 +36,6 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -164,10 +163,10 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager { private TopBundleLoadDataReporter topBundleLoadDataReporter; - private ScheduledFuture brokerLoadDataReportTask; - private ScheduledFuture topBundlesLoadDataReportTask; + private volatile ScheduledFuture brokerLoadDataReportTask; + private volatile ScheduledFuture topBundlesLoadDataReportTask; - private ScheduledFuture monitorTask; + private volatile ScheduledFuture monitorTask; private SplitScheduler splitScheduler; private UnloadManager unloadManager; @@ -190,7 +189,7 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager { private final ConcurrentHashMap>> lookupRequests = new ConcurrentHashMap<>(); - private final CountDownLatch initWaiter = new CountDownLatch(1); + private final CompletableFuture initWaiter = new CompletableFuture<>(); /** * Get all the bundles that are owned by this broker. @@ -321,12 +320,14 @@ public void start() throws PulsarServerException { pulsar.getCoordinationService(), pulsar.getBrokerId(), pulsar.getSafeWebServiceAddress(), ELECTION_ROOT, state -> { - pulsar.getLoadManagerExecutor().execute(() -> { - if (state == LeaderElectionState.Leading) { - playLeader(); - } else { - playFollower(); - } + pulsar.runWhenReadyForIncomingRequests(() -> { + pulsar.getLoadManagerExecutor().execute(() -> { + if (state == LeaderElectionState.Leading) { + playLeader(); + } else { + playFollower(); + } + }); }); }); this.serviceUnitStateChannel = ServiceUnitStateChannelImpl.newInstance(pulsar); @@ -336,7 +337,13 @@ public void start() throws PulsarServerException { this.serviceUnitStateChannel.listen(unloadManager); this.serviceUnitStateChannel.listen(splitManager); this.leaderElectionService.start(); - this.serviceUnitStateChannel.start(); + pulsar.runWhenReadyForIncomingRequests(() -> { + try { + this.serviceUnitStateChannel.start(); + } catch (Exception e) { + failStarting(e); + } + }); this.antiAffinityGroupPolicyHelper = new AntiAffinityGroupPolicyHelper(pulsar, serviceUnitStateChannel); antiAffinityGroupPolicyHelper.listenFailureDomainUpdate(); @@ -368,54 +375,72 @@ public void start() throws PulsarServerException { new TopBundleLoadDataReporter(pulsar, brokerRegistry.getBrokerId(), topBundlesLoadDataStore); this.serviceUnitStateChannel.listen(brokerLoadDataReporter); this.serviceUnitStateChannel.listen(topBundleLoadDataReporter); - var interval = conf.getLoadBalancerReportUpdateMinIntervalMillis(); - this.brokerLoadDataReportTask = this.pulsar.getLoadManagerExecutor() - .scheduleAtFixedRate(() -> { - try { - brokerLoadDataReporter.reportAsync(false); - // TODO: update broker load metrics using getLocalData - } catch (Throwable e) { - log.error("Failed to run the broker load manager executor job.", e); - } - }, - interval, - interval, TimeUnit.MILLISECONDS); - - this.topBundlesLoadDataReportTask = this.pulsar.getLoadManagerExecutor() - .scheduleAtFixedRate(() -> { - try { - // TODO: consider excluding the bundles that are in the process of split. - topBundleLoadDataReporter.reportAsync(false); - } catch (Throwable e) { - log.error("Failed to run the top bundles load manager executor job.", e); - } - }, - interval, - interval, TimeUnit.MILLISECONDS); - - this.monitorTask = this.pulsar.getLoadManagerExecutor() - .scheduleAtFixedRate(() -> { - monitor(); - }, - MONITOR_INTERVAL_IN_MILLIS, - MONITOR_INTERVAL_IN_MILLIS, TimeUnit.MILLISECONDS); this.unloadScheduler = new UnloadScheduler( pulsar, pulsar.getLoadManagerExecutor(), unloadManager, context, serviceUnitStateChannel, unloadCounter, unloadMetrics); this.splitScheduler = new SplitScheduler( pulsar, serviceUnitStateChannel, splitManager, splitCounter, splitMetrics, context); - this.splitScheduler.start(); - this.initWaiter.countDown(); - this.started = true; - log.info("Started load manager."); + + pulsar.runWhenReadyForIncomingRequests(() -> { + try { + var interval = conf.getLoadBalancerReportUpdateMinIntervalMillis(); + + this.brokerLoadDataReportTask = this.pulsar.getLoadManagerExecutor() + .scheduleAtFixedRate(() -> { + try { + brokerLoadDataReporter.reportAsync(false); + // TODO: update broker load metrics using getLocalData + } catch (Throwable e) { + log.error("Failed to run the broker load manager executor job.", e); + } + }, + interval, + interval, TimeUnit.MILLISECONDS); + + this.topBundlesLoadDataReportTask = this.pulsar.getLoadManagerExecutor() + .scheduleAtFixedRate(() -> { + try { + // TODO: consider excluding the bundles that are in the process of split. + topBundleLoadDataReporter.reportAsync(false); + } catch (Throwable e) { + log.error("Failed to run the top bundles load manager executor job.", e); + } + }, + interval, + interval, TimeUnit.MILLISECONDS); + + this.monitorTask = this.pulsar.getLoadManagerExecutor() + .scheduleAtFixedRate(() -> { + monitor(); + }, + MONITOR_INTERVAL_IN_MILLIS, + MONITOR_INTERVAL_IN_MILLIS, TimeUnit.MILLISECONDS); + + this.splitScheduler.start(); + this.initWaiter.complete(null); + this.started = true; + log.info("Started load manager."); + } catch (Exception ex) { + failStarting(ex); + } + }); } catch (Exception ex) { - log.error("Failed to start the extensible load balance and close broker registry {}.", - this.brokerRegistry, ex); - if (this.brokerRegistry != null) { + failStarting(ex); + } + } + + private void failStarting(Exception ex) { + log.error("Failed to start the extensible load balance and close broker registry {}.", + this.brokerRegistry, ex); + if (this.brokerRegistry != null) { + try { brokerRegistry.close(); + } catch (PulsarServerException e) { + // ignore } } + initWaiter.completeExceptionally(ex); } @Override @@ -772,11 +797,11 @@ synchronized void playLeader() { boolean becameFollower = false; while (!Thread.currentThread().isInterrupted()) { try { + initWaiter.get(); if (!serviceUnitStateChannel.isChannelOwner()) { becameFollower = true; break; } - initWaiter.await(); // 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). @@ -822,11 +847,11 @@ synchronized void playFollower() { boolean becameLeader = false; while (!Thread.currentThread().isInterrupted()) { try { + initWaiter.get(); if (serviceUnitStateChannel.isChannelOwner()) { becameLeader = true; break; } - initWaiter.await(); unloadScheduler.close(); serviceUnitStateChannel.cancelOwnershipMonitor(); brokerLoadDataStore.init(); @@ -885,7 +910,7 @@ public List getMetrics() { @VisibleForTesting protected void monitor() { try { - initWaiter.await(); + initWaiter.get(); // Monitor role // Periodically check the role in case ZK watcher fails. 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 8f3a30a59f073..f9ee3c846e3a9 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 @@ -1262,7 +1262,9 @@ public void addNamespaceBundleOwnershipListener(NamespaceBundleOwnershipListener bundleOwnershipListeners.add(listener); } } - getOwnedServiceUnits().forEach(bundle -> notifyNamespaceBundleOwnershipListener(bundle, listeners)); + pulsar.runWhenReadyForIncomingRequests(() -> { + getOwnedServiceUnits().forEach(bundle -> notifyNamespaceBundleOwnershipListener(bundle, listeners)); + }); } private void notifyNamespaceBundleOwnershipListener(NamespaceBundle bundle, diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarChannelInitializer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarChannelInitializer.java index 5308b3c981eb4..e276ea24fed18 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarChannelInitializer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/PulsarChannelInitializer.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.service; import com.google.common.annotations.VisibleForTesting; +import io.netty.channel.ChannelHandler; import io.netty.channel.ChannelInitializer; import io.netty.channel.socket.SocketChannel; import io.netty.handler.codec.LengthFieldBasedFrameDecoder; @@ -104,6 +105,9 @@ public PulsarChannelInitializer(PulsarService pulsar, PulsarChannelOptions opts) @Override protected void initChannel(SocketChannel ch) throws Exception { + // disable auto read explicitly so that requests aren't served until auto read is enabled + // ServerCnx must enable auto read in channelActive after PulsarService is ready to accept incoming requests + ch.config().setAutoRead(false); ch.pipeline().addLast("consolidation", new FlushConsolidationHandler(1024, true)); if (this.enableTls) { if (this.tlsEnabledWithKeyStore) { @@ -128,7 +132,8 @@ protected void initChannel(SocketChannel ch) throws Exception { // ServerCnx ends up reading higher number of messages and broker can not throttle the messages by disabling // auto-read. ch.pipeline().addLast("flowController", new FlowControlHandler()); - ServerCnx cnx = newServerCnx(pulsar, listenerName); + // using "ChannelHandler" type to workaround an IntelliJ bug that shows a false positive error + ChannelHandler cnx = newServerCnx(pulsar, listenerName); ch.pipeline().addLast("handler", cnx); } 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 2514be55fffa9..3f7c8b9c20a22 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 @@ -337,6 +337,10 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception { this.commandSender = new PulsarCommandSenderImpl(brokerInterceptor, this); this.service.getPulsarStats().recordConnectionCreate(); cnxsPerThread.get().add(this); + service.getPulsar().runWhenReadyForIncomingRequests(() -> { + // enable auto read after PulsarService is ready to accept incoming requests + ctx.channel().config().setAutoRead(true); + }); } @Override 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 4009401971c33..89c12c6771ece 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 @@ -20,12 +20,21 @@ import io.prometheus.client.CollectorRegistry; import io.prometheus.client.jetty.JettyStatisticsCollector; +import java.io.IOException; import java.util.ArrayList; import java.util.EnumSet; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.concurrent.ExecutionException; 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 lombok.Getter; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; @@ -228,6 +237,7 @@ private static class FilterInitializer { private final FilterHolder authenticationFilterHolder; FilterInitializer(PulsarService pulsarService) { ServiceConfiguration config = pulsarService.getConfiguration(); + if (config.getMaxConcurrentHttpRequests() > 0) { FilterHolder filterHolder = new FilterHolder(QoSFilter.class); filterHolder.setInitParameter("maxRequests", String.valueOf(config.getMaxConcurrentHttpRequests())); @@ -239,8 +249,11 @@ private static class FilterInitializer { new RateLimitingFilter(config.getHttpRequestsMaxPerSecond()))); } - boolean brokerInterceptorEnabled = - pulsarService.getBrokerInterceptor() != null && !config.isDisableBrokerInterceptors(); + // wait until the PulsarService is ready to serve incoming requests + filterHolders.add( + new FilterHolder(new WaitUntilPulsarServiceIsReadyForIncomingRequestsFilter(pulsarService))); + + boolean brokerInterceptorEnabled = pulsarService.getBrokerInterceptor() != null; if (brokerInterceptorEnabled) { ExceptionHandler handler = new ExceptionHandler(); // Enable PreInterceptFilter only when interceptors are enabled @@ -281,6 +294,42 @@ public void addFilters(ServletContextHandler context, boolean requiresAuthentica } } + // Filter that waits until the PulsarService is ready to serve incoming requests + private static class WaitUntilPulsarServiceIsReadyForIncomingRequestsFilter implements Filter { + private final PulsarService pulsarService; + + public WaitUntilPulsarServiceIsReadyForIncomingRequestsFilter(PulsarService pulsarService) { + this.pulsarService = pulsarService; + } + + @Override + public void init(FilterConfig filterConfig) throws ServletException { + + } + + @Override + public void doFilter(ServletRequest request, ServletResponse response, FilterChain chain) + throws IOException, ServletException { + try { + // Wait until the PulsarService is ready to serve incoming requests + pulsarService.waitUntilReadyForIncomingRequests(); + } catch (ExecutionException e) { + ((HttpServletResponse) response).sendError(HttpServletResponse.SC_SERVICE_UNAVAILABLE, + "PulsarService failed to start."); + return; + } catch (InterruptedException e) { + ((HttpServletResponse) response).sendError(HttpServletResponse.SC_SERVICE_UNAVAILABLE, + "PulsarService is not ready."); + return; + } + chain.doFilter(request, response); + } + + @Override + public void destroy() { + + } + } } public void addServlet(String path, ServletHolder servletHolder, boolean requiresAuthentication, From fb7282171aec35ea5488ca02280779af76939410 Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Thu, 18 Apr 2024 09:48:14 +0800 Subject: [PATCH 588/699] [fix][broker] Check the broker is available for the SLA monitor bundle when the ExtensibleLoadManager is enabled (#22485) (cherry picked from commit d0b9d471d53d2db600b55a04d6255688d1fd2d27) (cherry picked from commit c1a8596d1cf673b28656ff10ce42b0322d04c8ce) (cherry picked from commit f623be8a158f8954f454759fb3cf8804621b6de3) --- .../extensions/ExtensibleLoadManagerImpl.java | 39 ++++++--------- .../broker/namespace/NamespaceService.java | 47 ++++++++++++++----- .../ExtensibleLoadManagerImplTest.java | 47 +++++++++++++++++++ 3 files changed, 98 insertions(+), 35 deletions(-) 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 fb33ff85521ff..c7843a464d14b 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 @@ -461,30 +461,20 @@ public CompletableFuture> assign(Optional { + if (candidateBrokerId != null) { + return CompletableFuture.completedFuture(Optional.of(candidateBrokerId)); + } + return getOrSelectOwnerAsync(serviceUnit, bundle).thenApply(Optional::ofNullable); + }); } return getBrokerLookupData(owner, bundle); }); } - private String getHeartbeatOrSLAMonitorBrokerId(ServiceUnitId serviceUnit) { - // Check if this is Heartbeat or SLAMonitor namespace - String candidateBroker = NamespaceService.checkHeartbeatNamespace(serviceUnit); - if (candidateBroker == null) { - candidateBroker = NamespaceService.checkHeartbeatNamespaceV2(serviceUnit); - } - if (candidateBroker == null) { - candidateBroker = NamespaceService.getSLAMonitorBrokerName(serviceUnit); - } - if (candidateBroker != null) { - return candidateBroker.substring(candidateBroker.lastIndexOf('/') + 1); - } - return candidateBroker; + private CompletableFuture getHeartbeatOrSLAMonitorBrokerId(ServiceUnitId serviceUnit) { + return pulsar.getNamespaceService().getHeartbeatOrSLAMonitorBrokerId(serviceUnit, + cb -> brokerRegistry.lookupAsync(cb).thenApply(Optional::isPresent)); } private CompletableFuture getOrSelectOwnerAsync(ServiceUnitId serviceUnit, @@ -631,11 +621,12 @@ public CompletableFuture> getOwnershipAsync(Optional { + if (candidateBroker != null) { + return CompletableFuture.completedFuture(Optional.of(candidateBroker)); + } + return serviceUnitStateChannel.getOwnerAsync(bundle); + }); } public CompletableFuture> getOwnershipWithLookupDataAsync(ServiceUnitId bundleUnit) { 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 f9ee3c846e3a9..cad66ceb3ff1e 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 @@ -43,6 +43,7 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -494,6 +495,38 @@ private CompletableFuture> findBrokerServiceUrl( }); } + /** + * Check if this is Heartbeat or SLAMonitor namespace and return the broker id. + * + * @param serviceUnit the service unit + * @param isBrokerActive the function to check if the broker is active + * @return the broker id + */ + public CompletableFuture getHeartbeatOrSLAMonitorBrokerId( + ServiceUnitId serviceUnit, Function> isBrokerActive) { + String candidateBroker = NamespaceService.checkHeartbeatNamespace(serviceUnit); + if (candidateBroker != null) { + return CompletableFuture.completedFuture(candidateBroker); + } + candidateBroker = NamespaceService.checkHeartbeatNamespaceV2(serviceUnit); + if (candidateBroker != null) { + return CompletableFuture.completedFuture(candidateBroker); + } + candidateBroker = NamespaceService.getSLAMonitorBrokerName(serviceUnit); + if (candidateBroker != null) { + // Check if the broker is available + final String finalCandidateBroker = candidateBroker; + return isBrokerActive.apply(candidateBroker).thenApply(isActive -> { + if (isActive) { + return finalCandidateBroker; + } else { + return null; + } + }); + } + return CompletableFuture.completedFuture(null); + } + private void searchForCandidateBroker(NamespaceBundle bundle, CompletableFuture> lookupFuture, LookupOptions options) { @@ -521,17 +554,9 @@ private void searchForCandidateBroker(NamespaceBundle bundle, try { // check if this is Heartbeat or SLAMonitor namespace - candidateBroker = checkHeartbeatNamespace(bundle); - if (candidateBroker == null) { - candidateBroker = checkHeartbeatNamespaceV2(bundle); - } - if (candidateBroker == null) { - String broker = getSLAMonitorBrokerName(bundle); - // checking if the broker is up and running - if (broker != null && isBrokerActive(broker)) { - candidateBroker = broker; - } - } + candidateBroker = getHeartbeatOrSLAMonitorBrokerId(bundle, cb -> + CompletableFuture.completedFuture(isBrokerActive(cb))) + .get(config.getMetadataStoreOperationTimeoutSeconds(), SECONDS); if (candidateBroker == null) { Optional currentLeader = pulsar.getLeaderElectionService().getCurrentLeader(); 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 e6f69dd862991..24e1f9949db01 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 @@ -51,6 +51,7 @@ import static org.mockito.Mockito.verify; 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.assertTrue; @@ -71,6 +72,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; +import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.commons.lang3.tuple.Pair; @@ -101,6 +103,10 @@ import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerServiceException; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.impl.TableViewImpl; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.NamespaceName; @@ -663,6 +669,47 @@ public void testDeployAndRollbackLoadManager() throws Exception { pulsar.getBrokerId(), pulsar.getBrokerServiceUrl()); } } + // Check if the broker is available + var wrapper = (ExtensibleLoadManagerWrapper) pulsar4.getLoadManager().get(); + var loadManager4 = spy((ExtensibleLoadManagerImpl) + FieldUtils.readField(wrapper, "loadManager", true)); + loadManager4.getBrokerRegistry().unregister(); + + NamespaceName slaMonitorNamespace = + getSLAMonitorNamespace(pulsar4.getBrokerId(), pulsar.getConfiguration()); + String slaMonitorTopic = slaMonitorNamespace.getPersistentTopicName("test"); + String result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); + assertNotNull(result); + log.info("{} Namespace is re-owned by {}", slaMonitorTopic, result); + assertNotEquals(result, pulsar4.getBrokerServiceUrl()); + + Producer producer = pulsar.getClient().newProducer(Schema.STRING).topic(slaMonitorTopic).create(); + producer.send("t1"); + + // Test re-register broker and check the lookup result + loadManager4.getBrokerRegistry().register(); + + result = pulsar.getAdminClient().lookups().lookupTopic(slaMonitorTopic); + assertNotNull(result); + log.info("{} Namespace is re-owned by {}", slaMonitorTopic, result); + assertEquals(result, pulsar4.getBrokerServiceUrl()); + + producer.send("t2"); + Producer producer1 = pulsar.getClient().newProducer(Schema.STRING).topic(slaMonitorTopic).create(); + producer1.send("t3"); + + producer.close(); + producer1.close(); + @Cleanup + Consumer consumer = pulsar.getClient().newConsumer(Schema.STRING) + .topic(slaMonitorTopic) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscriptionName("test") + .subscribe(); + // receive message t1 t2 t3 + assertEquals(consumer.receive().getValue(), "t1"); + assertEquals(consumer.receive().getValue(), "t2"); + assertEquals(consumer.receive().getValue(), "t3"); } } } From 64682a79832245403dbc1f068d4311f2ca830d1e Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Tue, 23 Apr 2024 15:22:44 +0800 Subject: [PATCH 589/699] [fix][broker] Support lookup options for extensible load manager (#22487) (cherry picked from commit 7fe92ac43cfd2f2de5576a023498aac8b46c7ac8) (cherry picked from commit d0c075fc5b59207e7bec587326ad35985c1ce17c) (cherry picked from commit 2b84dffe3fe836eff671df65045a8668aee87510) --- .../broker/loadbalance/LoadManager.java | 3 +- .../extensions/ExtensibleLoadManager.java | 5 +- .../extensions/ExtensibleLoadManagerImpl.java | 53 ++++++------ .../ExtensibleLoadManagerWrapper.java | 15 +++- .../channel/ServiceUnitStateChannelImpl.java | 4 +- .../extensions/data/BrokerLookupData.java | 17 +++- .../broker/namespace/NamespaceService.java | 4 +- ...tiAffinityNamespaceGroupExtensionTest.java | 4 +- .../ExtensibleLoadManagerImplBaseTest.java | 4 + .../ExtensibleLoadManagerImplTest.java | 82 +++++++++++++++---- .../channel/ServiceUnitStateChannelTest.java | 14 ++-- .../extensions/data/BrokerLookupDataTest.java | 32 +++++++- .../ExtensibleLoadManagerTest.java | 3 +- 13 files changed, 175 insertions(+), 65 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java index 2cce68b60cb49..0dd5d948480ab 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/LoadManager.java @@ -31,6 +31,7 @@ import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerWrapper; import org.apache.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl; import org.apache.pulsar.broker.lookup.LookupResult; +import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.common.naming.ServiceUnitId; import org.apache.pulsar.common.stats.Metrics; import org.apache.pulsar.common.util.Reflections; @@ -63,7 +64,7 @@ public interface LoadManager { Optional getLeastLoaded(ServiceUnitId su) throws Exception; default CompletableFuture> findBrokerServiceUrl( - Optional topic, ServiceUnitId bundle) { + Optional topic, ServiceUnitId bundle, LookupOptions options) { throw new UnsupportedOperationException(); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManager.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManager.java index b7da70d1cf1de..eabf6005b439b 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManager.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManager.java @@ -60,9 +60,12 @@ public interface ExtensibleLoadManager extends Closeable { * (e.g. {@link NamespaceService#internalGetWebServiceUrl(NamespaceBundle, LookupOptions)}), * So the topic is optional. * @param serviceUnit service unit (e.g. bundle). + * @param options The lookup options. * @return The broker lookup data. */ - CompletableFuture> assign(Optional topic, ServiceUnitId serviceUnit); + CompletableFuture> assign(Optional topic, + ServiceUnitId serviceUnit, + LookupOptions options); /** * Check the incoming service unit is owned by the current broker. 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 c7843a464d14b..fba0289367e7d 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 @@ -86,6 +86,7 @@ import org.apache.pulsar.broker.loadbalance.extensions.strategy.LeastResourceUsageWithWeight; import org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared; import org.apache.pulsar.broker.loadbalance.impl.SimpleResourceAllocationPolicies; +import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.broker.namespace.NamespaceEphemeralData; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.client.admin.PulsarAdminException; @@ -451,7 +452,8 @@ public void initialize(PulsarService pulsar) { @Override public CompletableFuture> assign(Optional topic, - ServiceUnitId serviceUnit) { + ServiceUnitId serviceUnit, + LookupOptions options) { final String bundle = serviceUnit.toString(); @@ -465,7 +467,7 @@ public CompletableFuture> assign(Optional getHeartbeatOrSLAMonitorBrokerId(ServiceUnitId } private CompletableFuture getOrSelectOwnerAsync(ServiceUnitId serviceUnit, - String bundle) { + String bundle, + LookupOptions options) { return serviceUnitStateChannel.getOwnerAsync(bundle).thenCompose(broker -> { // If the bundle not assign yet, select and publish assign event to channel. if (broker.isEmpty()) { - return this.selectAsync(serviceUnit).thenCompose(brokerOpt -> { + return this.selectAsync(serviceUnit, Collections.emptySet(), options).thenCompose(brokerOpt -> { if (brokerOpt.isPresent()) { assignCounter.incrementSuccess(); log.info("Selected new owner broker: {} for bundle: {}.", brokerOpt.get(), bundle); return serviceUnitStateChannel.publishAssignEventAsync(bundle, brokerOpt.get()); } - throw new IllegalStateException( - "Failed to select the new owner broker for bundle: " + bundle); + return CompletableFuture.completedFuture(null); }); } assignCounter.incrementSkip(); @@ -503,22 +505,19 @@ private CompletableFuture> getBrokerLookupData( String bundle) { return owner.thenCompose(broker -> { if (broker.isEmpty()) { - String errorMsg = String.format( - "Failed to get or assign the owner for bundle:%s", bundle); - log.error(errorMsg); - throw new IllegalStateException(errorMsg); - } - return CompletableFuture.completedFuture(broker.get()); - }).thenCompose(broker -> this.getBrokerRegistry().lookupAsync(broker).thenCompose(brokerLookupData -> { - if (brokerLookupData.isEmpty()) { - String errorMsg = String.format( - "Failed to lookup broker:%s for bundle:%s, the broker has not been registered.", - broker, bundle); - log.error(errorMsg); - throw new IllegalStateException(errorMsg); + return CompletableFuture.completedFuture(Optional.empty()); } - return CompletableFuture.completedFuture(brokerLookupData); - })); + return this.getBrokerRegistry().lookupAsync(broker.get()).thenCompose(brokerLookupData -> { + if (brokerLookupData.isEmpty()) { + String errorMsg = String.format( + "Failed to lookup broker:%s for bundle:%s, the broker has not been registered.", + broker, bundle); + log.error(errorMsg); + throw new IllegalStateException(errorMsg); + } + return CompletableFuture.completedFuture(brokerLookupData); + }); + }); } /** @@ -531,7 +530,7 @@ private CompletableFuture> getBrokerLookupData( public CompletableFuture tryAcquiringOwnership(NamespaceBundle namespaceBundle) { log.info("Try acquiring ownership for bundle: {} - {}.", namespaceBundle, brokerRegistry.getBrokerId()); final String bundle = namespaceBundle.toString(); - return assign(Optional.empty(), namespaceBundle) + return assign(Optional.empty(), namespaceBundle, LookupOptions.builder().readOnly(false).build()) .thenApply(brokerLookupData -> { if (brokerLookupData.isEmpty()) { String errorMsg = String.format( @@ -564,12 +563,12 @@ private CompletableFuture> dedupeLookupRequest( } } - public CompletableFuture> selectAsync(ServiceUnitId bundle) { - return selectAsync(bundle, Collections.emptySet()); - } - public CompletableFuture> selectAsync(ServiceUnitId bundle, - Set excludeBrokerSet) { + Set excludeBrokerSet, + LookupOptions options) { + if (options.isReadOnly()) { + return CompletableFuture.completedFuture(Optional.empty()); + } BrokerRegistry brokerRegistry = getBrokerRegistry(); return brokerRegistry.getAvailableBrokerLookupDataAsync() .thenComposeAsync(availableBrokers -> { 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 cd1561cb70e2d..25eb27bc58d27 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 @@ -28,10 +28,11 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.loadbalance.LoadManager; import org.apache.pulsar.broker.loadbalance.ResourceUnit; -import org.apache.pulsar.broker.loadbalance.extensions.data.BrokerLookupData; import org.apache.pulsar.broker.lookup.LookupResult; +import org.apache.pulsar.broker.namespace.LookupOptions; 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.policies.data.loadbalancer.LoadManagerReport; public class ExtensibleLoadManagerWrapper implements LoadManager { @@ -62,9 +63,15 @@ public boolean isCentralized() { @Override public CompletableFuture> findBrokerServiceUrl( - Optional topic, ServiceUnitId bundle) { - return loadManager.assign(topic, bundle) - .thenApply(lookupData -> lookupData.map(BrokerLookupData::toLookupResult)); + Optional topic, ServiceUnitId bundle, LookupOptions options) { + return loadManager.assign(topic, bundle, options) + .thenApply(lookupData -> lookupData.map(data -> { + try { + return data.toLookupResult(options); + } catch (PulsarServerException ex) { + throw FutureUtil.wrapToCompletionException(ex); + } + })); } @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 0feed5e833327..03c77033b0470 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 @@ -83,6 +83,7 @@ import org.apache.pulsar.broker.loadbalance.extensions.models.Split; import org.apache.pulsar.broker.loadbalance.extensions.models.Unload; import org.apache.pulsar.broker.loadbalance.impl.LoadManagerShared; +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.api.CompressionType; @@ -1413,7 +1414,8 @@ private synchronized void doCleanup(String broker) { private Optional selectBroker(String serviceUnit, String inactiveBroker) { try { return loadManager.selectAsync( - LoadManagerShared.getNamespaceBundle(pulsar, serviceUnit), Set.of(inactiveBroker)) + LoadManagerShared.getNamespaceBundle(pulsar, serviceUnit), + Set.of(inactiveBroker), LookupOptions.builder().build()) .get(inFlightStateWaitingTimeInMillis, MILLISECONDS); } catch (Throwable e) { log.error("Failed to select a broker for serviceUnit:{}", serviceUnit); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupData.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupData.java index 41f5b18e321e8..50a2b70404039 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupData.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupData.java @@ -18,9 +18,12 @@ */ package org.apache.pulsar.broker.loadbalance.extensions.data; +import java.net.URI; import java.util.Map; import java.util.Optional; +import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.lookup.LookupResult; +import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.broker.namespace.NamespaceEphemeralData; import org.apache.pulsar.policies.data.loadbalancer.AdvertisedListener; import org.apache.pulsar.policies.data.loadbalancer.ServiceLookupData; @@ -79,7 +82,19 @@ public long getStartTimestamp() { return this.startTimestamp; } - public LookupResult toLookupResult() { + public LookupResult toLookupResult(LookupOptions options) throws PulsarServerException { + if (options.hasAdvertisedListenerName()) { + AdvertisedListener listener = advertisedListeners.get(options.getAdvertisedListenerName()); + if (listener == null) { + throw new PulsarServerException("the broker do not have " + + options.getAdvertisedListenerName() + " listener"); + } + URI url = listener.getBrokerServiceUrl(); + URI urlTls = listener.getBrokerServiceUrlTls(); + return new LookupResult(webServiceUrl, webServiceUrlTls, + url == null ? null : url.toString(), + urlTls == null ? null : urlTls.toString(), LookupResult.Type.BrokerUrl, false); + } return new LookupResult(webServiceUrl, webServiceUrlTls, pulsarServiceUrl, pulsarServiceUrlTls, LookupResult.Type.BrokerUrl, false); } 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 cad66ceb3ff1e..c60f75b1c827b 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 @@ -195,7 +195,7 @@ public CompletableFuture> getBrokerServiceUrlAsync(TopicN return CompletableFuture.completedFuture(optResult); } if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar)) { - return loadManager.get().findBrokerServiceUrl(Optional.of(topic), bundle); + return loadManager.get().findBrokerServiceUrl(Optional.of(topic), bundle, options); } else { // TODO: Add unit tests cover it. return findBrokerServiceUrl(bundle, options); @@ -311,7 +311,7 @@ private CompletableFuture> internalGetWebServiceUrl(@Nullable Serv } CompletableFuture> future = ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar) - ? loadManager.get().findBrokerServiceUrl(Optional.ofNullable(topic), bundle) : + ? loadManager.get().findBrokerServiceUrl(Optional.ofNullable(topic), bundle, options) : findBrokerServiceUrl(bundle, options); return future.thenApply(lookupResult -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/AntiAffinityNamespaceGroupExtensionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/AntiAffinityNamespaceGroupExtensionTest.java index 42fc12c2f998e..83e5738a40574 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/AntiAffinityNamespaceGroupExtensionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/AntiAffinityNamespaceGroupExtensionTest.java @@ -38,6 +38,7 @@ import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData; import org.apache.pulsar.broker.loadbalance.extensions.filter.AntiAffinityGroupPolicyFilter; import org.apache.pulsar.broker.loadbalance.extensions.policies.AntiAffinityGroupPolicyHelper; +import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; @@ -61,7 +62,8 @@ protected String getLoadManagerClassName() { protected String selectBroker(ServiceUnitId serviceUnit, Object loadManager) { try { - return ((ExtensibleLoadManagerImpl) loadManager).assign(Optional.empty(), serviceUnit).get() + return ((ExtensibleLoadManagerImpl) loadManager) + .assign(Optional.empty(), serviceUnit, LookupOptions.builder().build()).get() .get().getPulsarServiceUrl(); } catch (Throwable e) { throw new RuntimeException(e); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java index d830eaf4b5dbb..4ff391cc977d0 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/ExtensibleLoadManagerImplBaseTest.java @@ -21,6 +21,8 @@ import static org.mockito.Mockito.reset; import static org.mockito.Mockito.spy; import com.google.common.collect.Sets; + +import java.util.Optional; import java.util.concurrent.CompletableFuture; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.commons.lang3.tuple.Pair; @@ -69,6 +71,8 @@ protected ServiceConfiguration initConfig(ServiceConfiguration conf) { conf.setLoadBalancerLoadSheddingStrategy(TransferShedder.class.getName()); conf.setLoadBalancerSheddingEnabled(false); conf.setLoadBalancerDebugModeEnabled(true); + conf.setWebServicePortTls(Optional.of(0)); + conf.setBrokerServicePortTls(Optional.of(0)); return conf; } 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 24e1f9949db01..c219fafc284d1 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 @@ -141,10 +141,12 @@ public ExtensibleLoadManagerImplTest() { public void testAssignInternalTopic() throws Exception { Optional brokerLookupData1 = primaryLoadManager.assign( Optional.of(TopicName.get(ServiceUnitStateChannelImpl.TOPIC)), - getBundleAsync(pulsar1, TopicName.get(ServiceUnitStateChannelImpl.TOPIC)).get()).get(); + getBundleAsync(pulsar1, TopicName.get(ServiceUnitStateChannelImpl.TOPIC)).get(), + LookupOptions.builder().build()).get(); Optional brokerLookupData2 = secondaryLoadManager.assign( Optional.of(TopicName.get(ServiceUnitStateChannelImpl.TOPIC)), - getBundleAsync(pulsar1, TopicName.get(ServiceUnitStateChannelImpl.TOPIC)).get()).get(); + getBundleAsync(pulsar1, TopicName.get(ServiceUnitStateChannelImpl.TOPIC)).get(), + LookupOptions.builder().build()).get(); assertEquals(brokerLookupData1, brokerLookupData2); assertTrue(brokerLookupData1.isPresent()); @@ -152,22 +154,26 @@ public void testAssignInternalTopic() throws Exception { FieldUtils.readField(channel1, "leaderElectionService", true); Optional currentLeader = leaderElectionService.getCurrentLeader(); assertTrue(currentLeader.isPresent()); - assertEquals(brokerLookupData1.get().getWebServiceUrl(), currentLeader.get().getServiceUrl()); + assertEquals(brokerLookupData1.get().getWebServiceUrlTls(), currentLeader.get().getServiceUrl()); } @Test public void testAssign() throws Exception { - TopicName topicName = TopicName.get(defaultTestNamespace + "/test-assign"); - NamespaceBundle bundle = getBundleAsync(pulsar1, topicName).get(); - Optional brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle).get(); + + Pair topicAndBundle = getBundleIsNotOwnByChangeEventTopic("test-assign"); + TopicName topicName = topicAndBundle.getLeft(); + NamespaceBundle bundle = topicAndBundle.getRight(); + Optional brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle, + LookupOptions.builder().build()).get(); assertTrue(brokerLookupData.isPresent()); log.info("Assign the bundle {} to {}", bundle, brokerLookupData); // Should get owner info from channel. - Optional brokerLookupData1 = secondaryLoadManager.assign(Optional.empty(), bundle).get(); + Optional brokerLookupData1 = secondaryLoadManager.assign(Optional.empty(), bundle, + LookupOptions.builder().build()).get(); assertEquals(brokerLookupData, brokerLookupData1); Optional lookupResult = pulsar2.getNamespaceService() - .getBrokerServiceUrlAsync(topicName, null).get(); + .getBrokerServiceUrlAsync(topicName, LookupOptions.builder().build()).get(); assertTrue(lookupResult.isPresent()); assertEquals(lookupResult.get().getLookupData().getHttpUrl(), brokerLookupData.get().getWebServiceUrl()); @@ -177,6 +183,43 @@ public void testAssign() throws Exception { assertEquals(webServiceUrl.get().toString(), brokerLookupData.get().getWebServiceUrl()); } + @Test + public void testLookupOptions() throws Exception { + Pair topicAndBundle = + getBundleIsNotOwnByChangeEventTopic("test-lookup-options"); + TopicName topicName = topicAndBundle.getLeft(); + NamespaceBundle bundle = topicAndBundle.getRight(); + + admin.topics().createPartitionedTopic(topicName.toString(), 1); + + // Test LookupOptions.readOnly = true when the bundle is not owned by any broker. + Optional webServiceUrlReadOnlyTrue = pulsar1.getNamespaceService() + .getWebServiceUrl(bundle, LookupOptions.builder().readOnly(true).requestHttps(false).build()); + assertTrue(webServiceUrlReadOnlyTrue.isEmpty()); + + // Test LookupOptions.readOnly = false and the bundle assign to some broker. + Optional webServiceUrlReadOnlyFalse = pulsar1.getNamespaceService() + .getWebServiceUrl(bundle, LookupOptions.builder().readOnly(false).requestHttps(false).build()); + assertTrue(webServiceUrlReadOnlyFalse.isPresent()); + + // Test LookupOptions.requestHttps = true + Optional webServiceUrlHttps = pulsar2.getNamespaceService() + .getWebServiceUrl(bundle, LookupOptions.builder().requestHttps(true).build()); + assertTrue(webServiceUrlHttps.isPresent()); + assertTrue(webServiceUrlHttps.get().toString().startsWith("https")); + + // TODO: Support LookupOptions.loadTopicsInBundle = true + + // Test LookupOptions.advertisedListenerName = internal but the broker do not have internal listener. + try { + pulsar2.getNamespaceService() + .getWebServiceUrl(bundle, LookupOptions.builder().advertisedListenerName("internal").build()); + fail(); + } catch (Exception e) { + assertTrue(e.getMessage().contains("the broker do not have internal listener")); + } + } + @Test public void testCheckOwnershipAsync() throws Exception { TopicName topicName = TopicName.get(defaultTestNamespace + "/test-check-ownership"); @@ -186,7 +229,7 @@ public void testCheckOwnershipAsync() throws Exception { assertFalse(secondaryLoadManager.checkOwnershipAsync(Optional.empty(), bundle).get()); // 2. Assign the bundle to a broker. - Optional lookupData = primaryLoadManager.assign(Optional.empty(), bundle).get(); + Optional lookupData = primaryLoadManager.assign(Optional.empty(), bundle, LookupOptions.builder().build()).get(); assertTrue(lookupData.isPresent()); if (lookupData.get().getPulsarServiceUrl().equals(pulsar1.getBrokerServiceUrl())) { assertTrue(primaryLoadManager.checkOwnershipAsync(Optional.empty(), bundle).get()); @@ -219,7 +262,7 @@ public CompletableFuture> filterAsync(Map brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle).get(); + Optional brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle, LookupOptions.builder().build()).get(); assertTrue(brokerLookupData.isPresent()); assertEquals(brokerLookupData.get().getWebServiceUrl(), pulsar2.getWebServiceAddress()); } @@ -239,7 +282,7 @@ public CompletableFuture> filterAsync(Map brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle).get(); + Optional brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle, LookupOptions.builder().build()).get(); assertTrue(brokerLookupData.isPresent()); } @@ -248,7 +291,7 @@ public void testUnloadUponTopicLookupFailure() throws Exception { TopicName topicName = TopicName.get("public/test/testUnloadUponTopicLookupFailure"); NamespaceBundle bundle = pulsar1.getNamespaceService().getBundle(topicName); - primaryLoadManager.assign(Optional.empty(), bundle).get(); + primaryLoadManager.assign(Optional.empty(), bundle, LookupOptions.builder().build()).get(); CompletableFuture future1 = new CompletableFuture(); CompletableFuture future2 = new CompletableFuture(); @@ -538,9 +581,16 @@ public CompletableFuture> filterAsync(Map brokerLookupData = primaryLoadManager.assign(Optional.empty(), bundle).get(); - assertTrue(brokerLookupData.isPresent()); - assertEquals(brokerLookupData.get().getWebServiceUrl(), pulsar2.getWebServiceAddress()); + Awaitility.waitAtMost(5, TimeUnit.SECONDS).untilAsserted(() -> { + Optional brokerLookupData = primaryLoadManager.assign( + Optional.empty(), bundle, LookupOptions.builder().build()).get(); + assertTrue(brokerLookupData.isPresent()); + assertEquals(brokerLookupData.get().getWebServiceUrl(), pulsar2.getWebServiceAddress()); + assertEquals(brokerLookupData.get().getPulsarServiceUrl(), + pulsar1.getAdminClient().lookups().lookupTopic(topicName.toString())); + assertEquals(brokerLookupData.get().getPulsarServiceUrl(), + pulsar2.getAdminClient().lookups().lookupTopic(topicName.toString())); + }); } @Test @@ -1229,7 +1279,7 @@ public void testGetOwnedServiceUnitsAndGetOwnedNamespaceStatus() throws Exceptio String topic = "persistent://" + defaultTestNamespace + "/test-get-owned-service-units"; admin.topics().createPartitionedTopic(topic, 1); NamespaceBundle bundle = getBundleAsync(pulsar1, TopicName.get(topic)).join(); - CompletableFuture> owner = primaryLoadManager.assign(Optional.empty(), bundle); + CompletableFuture> owner = primaryLoadManager.assign(Optional.empty(), bundle, LookupOptions.builder().build()); assertFalse(owner.join().isEmpty()); BrokerLookupData brokerLookupData = owner.join().get(); 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 10ba6d2832b4f..0b7bcb34d440e 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 @@ -499,7 +499,7 @@ public void transferTestWhenDestBrokerFails() // recovered, check the monitor update state : Assigned -> Owned doReturn(CompletableFuture.completedFuture(Optional.of(brokerId1))) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); FieldUtils.writeDeclaredField(channel2, "producer", producer, true); FieldUtils.writeDeclaredField(channel1, "inFlightStateWaitingTimeInMillis", 1 , true); @@ -721,7 +721,7 @@ public void handleBrokerDeletionEventTest() var owner1 = channel1.getOwnerAsync(bundle1); var owner2 = channel2.getOwnerAsync(bundle2); doReturn(CompletableFuture.completedFuture(Optional.of(brokerId2))) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); assertTrue(owner1.get().isEmpty()); assertTrue(owner2.get().isEmpty()); @@ -1123,7 +1123,7 @@ public void assignTestWhenDestBrokerProducerFails() FieldUtils.writeDeclaredField(channel2, "inFlightStateWaitingTimeInMillis", 3 * 1000, true); doReturn(CompletableFuture.completedFuture(Optional.of(brokerId2))) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); channel1.publishAssignEventAsync(bundle, brokerId2); // channel1 is broken. the assign won't be complete. waitUntilState(channel1, bundle); @@ -1524,7 +1524,7 @@ public void testOverrideInactiveBrokerStateData() // test stable metadata state doReturn(CompletableFuture.completedFuture(Optional.of(brokerId2))) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); leaderChannel.handleMetadataSessionEvent(SessionReestablished); followerChannel.handleMetadataSessionEvent(SessionReestablished); FieldUtils.writeDeclaredField(leaderChannel, "lastMetadataSessionEventTimestamp", @@ -1589,7 +1589,7 @@ public void testOverrideOrphanStateData() // test stable metadata state doReturn(CompletableFuture.completedFuture(Optional.of(brokerId2))) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); FieldUtils.writeDeclaredField(leaderChannel, "inFlightStateWaitingTimeInMillis", -1, true); FieldUtils.writeDeclaredField(followerChannel, "inFlightStateWaitingTimeInMillis", @@ -1675,7 +1675,7 @@ public void testActiveGetOwner() throws Exception { // case 7: the ownership cleanup(no new owner) by the leader channel doReturn(CompletableFuture.completedFuture(Optional.empty())) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); var leaderChannel = channel1; String leader1 = channel1.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); String leader2 = channel2.getChannelOwnerAsync().get(2, TimeUnit.SECONDS).get(); @@ -1702,7 +1702,7 @@ public void testActiveGetOwner() throws Exception { overrideTableViews(bundle, new ServiceUnitStateData(Owned, broker, null, 1)); doReturn(CompletableFuture.completedFuture(Optional.of(brokerId1))) - .when(loadManager).selectAsync(any(), any()); + .when(loadManager).selectAsync(any(), any(), any()); leaderChannel.handleMetadataSessionEvent(SessionReestablished); FieldUtils.writeDeclaredField(leaderChannel, "lastMetadataSessionEventTimestamp", System.currentTimeMillis() - (MAX_CLEAN_UP_DELAY_TIME_IN_SECS * 1000 + 1000), true); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupDataTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupDataTest.java index 0d874e0f77117..66e8c917d1fc5 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupDataTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/extensions/data/BrokerLookupDataTest.java @@ -18,13 +18,19 @@ */ package org.apache.pulsar.broker.loadbalance.extensions.data; +import static org.testng.Assert.fail; import static org.testng.AssertJUnit.assertEquals; import static org.testng.AssertJUnit.assertTrue; + +import java.net.URI; +import java.net.URISyntaxException; import java.util.HashMap; import java.util.Map; import java.util.Optional; +import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; import org.apache.pulsar.broker.lookup.LookupResult; +import org.apache.pulsar.broker.namespace.LookupOptions; import org.apache.pulsar.policies.data.loadbalancer.AdvertisedListener; import org.testng.annotations.Test; @@ -32,12 +38,20 @@ public class BrokerLookupDataTest { @Test - public void testConstructors() { + public void testConstructors() throws PulsarServerException, URISyntaxException { String webServiceUrl = "http://localhost:8080"; String webServiceUrlTls = "https://localhoss:8081"; String pulsarServiceUrl = "pulsar://localhost:6650"; String pulsarServiceUrlTls = "pulsar+ssl://localhost:6651"; - Map advertisedListeners = new HashMap<>(); + final String listenerUrl = "pulsar://gateway:7000"; + final String listenerUrlTls = "pulsar://gateway:8000"; + final String listener = "internal"; + Map advertisedListeners = new HashMap<>(){{ + put(listener, AdvertisedListener.builder() + .brokerServiceUrl(new URI(listenerUrl)) + .brokerServiceUrlTls(new URI(listenerUrlTls)) + .build()); + }}; Map protocols = new HashMap<>(){{ put("kafka", "9092"); }}; @@ -56,10 +70,22 @@ public void testConstructors() { assertEquals("3.0", lookupData.brokerVersion()); - LookupResult lookupResult = lookupData.toLookupResult(); + LookupResult lookupResult = lookupData.toLookupResult(LookupOptions.builder().build()); assertEquals(webServiceUrl, lookupResult.getLookupData().getHttpUrl()); assertEquals(webServiceUrlTls, lookupResult.getLookupData().getHttpUrlTls()); assertEquals(pulsarServiceUrl, lookupResult.getLookupData().getBrokerUrl()); assertEquals(pulsarServiceUrlTls, lookupResult.getLookupData().getBrokerUrlTls()); + + try { + lookupData.toLookupResult(LookupOptions.builder().advertisedListenerName("others").build()); + fail(); + } catch (PulsarServerException ex) { + assertTrue(ex.getMessage().contains("the broker do not have others listener")); + } + lookupResult = lookupData.toLookupResult(LookupOptions.builder().advertisedListenerName(listener).build()); + assertEquals(listenerUrl, lookupResult.getLookupData().getBrokerUrl()); + assertEquals(listenerUrlTls, lookupResult.getLookupData().getBrokerUrlTls()); + assertEquals(webServiceUrl, lookupResult.getLookupData().getHttpUrl()); + assertEquals(webServiceUrlTls, lookupResult.getLookupData().getHttpUrlTls()); } } diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java index af14ef97f85c3..6da4c73912619 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/loadbalance/ExtensibleLoadManagerTest.java @@ -409,9 +409,10 @@ public void testIsolationPolicy() throws Exception { () -> { try { admin.lookups().lookupTopicAsync(topic).get(5, TimeUnit.SECONDS); + fail(); } catch (Exception ex) { log.error("Failed to lookup topic: ", ex); - assertThat(ex.getMessage()).contains("Failed to select the new owner broker for bundle"); + assertThat(ex.getMessage()).contains("Service Unavailable"); } } ); From 6e7943f3993d0f0bb23931ebbde14d00a24f4e98 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Tue, 25 Jun 2024 09:38:56 +0300 Subject: [PATCH 590/699] [fix][broker] Fix updatePartitionedTopic when replication at ns level and topic policy is set (#22971) (cherry picked from commit 1c44fbb8a03e583e94aa9dbef87dfa0a165e1cd8) (cherry picked from commit d93e896810287d9c7ac714326a6ca3ddc6c6c820) --- .../admin/impl/PersistentTopicsBase.java | 5 ++-- .../broker/service/OneWayReplicatorTest.java | 27 +++++++++++++++++-- .../service/OneWayReplicatorTestBase.java | 9 ++++--- 3 files changed, 32 insertions(+), 9 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 2a6b89b413371..a899eef63d57b 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 @@ -35,7 +35,6 @@ import java.util.Base64; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -469,8 +468,8 @@ protected CompletableFuture internalCreateNonPartitionedTopicAsync(boolean Set replicationClusters = policies.get().replication_clusters; TopicPolicies topicPolicies = pulsarService.getTopicPoliciesService().getTopicPoliciesIfExists(topicName); - if (topicPolicies != null) { - replicationClusters = new HashSet<>(topicPolicies.getReplicationClusters()); + if (topicPolicies != null && topicPolicies.getReplicationClusters() != null) { + replicationClusters = topicPolicies.getReplicationClustersSet(); } // Do check replicated clusters. if (replicationClusters.size() == 0) { 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 e69165fe9495c..2aee9a355c5d5 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 @@ -68,11 +68,11 @@ import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; -import org.apache.pulsar.common.policies.data.TopicStats; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; import org.mockito.Mockito; @@ -461,6 +461,29 @@ public void testPartitionedTopicLevelReplication() throws Exception { admin2.topics().deletePartitionedTopic(topicName); } + // https://github.com/apache/pulsar/issues/22967 + @Test + public void testPartitionedTopicWithTopicPolicyAndNoReplicationClusters() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + replicatedNamespace + "/tp_"); + admin1.topics().createPartitionedTopic(topicName, 2); + try { + admin1.topicPolicies().setMessageTTL(topicName, 5); + Awaitility.await().ignoreExceptions().untilAsserted(() -> { + assertEquals(admin2.topics().getPartitionedTopicMetadata(topicName).partitions, 2); + }); + admin1.topics().updatePartitionedTopic(topicName, 3, false); + Awaitility.await().ignoreExceptions().untilAsserted(() -> { + assertEquals(admin2.topics().getPartitionedTopicMetadata(topicName).partitions, 3); + }); + } finally { + // cleanup. + admin1.topics().deletePartitionedTopic(topicName, true); + if (!usingGlobalZK) { + admin2.topics().deletePartitionedTopic(topicName, true); + } + } + } + @Test public void testPartitionedTopicLevelReplicationRemoteTopicExist() throws Exception { final String topicName = BrokerTestUtil.newUniqueName("persistent://" + nonReplicatedNamespace + "/tp_"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java index ffe6147412e56..d66e666e3a055 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java @@ -266,6 +266,7 @@ protected void setConfigDefaults(ServiceConfiguration config, String clusterName config.setEnableReplicatedSubscriptions(true); config.setReplicatedSubscriptionsSnapshotFrequencyMillis(1000); config.setLoadBalancerSheddingEnabled(false); + config.setForceDeleteNamespaceAllowed(true); } @Override @@ -276,11 +277,11 @@ protected void cleanup() throws Exception { if (!usingGlobalZK) { admin2.namespaces().setNamespaceReplicationClusters(replicatedNamespace, Sets.newHashSet(cluster2)); } - admin1.namespaces().deleteNamespace(replicatedNamespace); - admin1.namespaces().deleteNamespace(nonReplicatedNamespace); + admin1.namespaces().deleteNamespace(replicatedNamespace, true); + admin1.namespaces().deleteNamespace(nonReplicatedNamespace, true); if (!usingGlobalZK) { - admin2.namespaces().deleteNamespace(replicatedNamespace); - admin2.namespaces().deleteNamespace(nonReplicatedNamespace); + admin2.namespaces().deleteNamespace(replicatedNamespace, true); + admin2.namespaces().deleteNamespace(nonReplicatedNamespace, true); } // shutdown. From 1a44c3d512e340f4ea757bc4db82c7fb17bff5ba Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 19 Jun 2024 22:29:17 +0800 Subject: [PATCH 591/699] [improve] [broker] PIP-356 Support Geo-Replication starts at earliest position (#22856) (cherry picked from commit 5fc0eafab9ea2a4ece7b87218404489c270b64e6) (cherry picked from commit ab8dba3370f0ce02bcef2bc6ae5295ffa874a7df) --- .../pulsar/broker/ServiceConfiguration.java | 6 + .../service/persistent/PersistentTopic.java | 9 +- .../broker/service/OneWayReplicatorTest.java | 103 +++++++++++++++++- .../OneWayReplicatorUsingGlobalZKTest.java | 52 +++++++++ 4 files changed, 167 insertions(+), 3 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 29d5d4b2ea30b..e6d84cd768682 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 @@ -1332,6 +1332,12 @@ The delayed message index time step(in seconds) in per bucket snapshot segment, doc = "Max number of snapshot to be cached per subscription.") private int replicatedSubscriptionsSnapshotMaxCachedPerSubscription = 10; + @FieldContext( + category = CATEGORY_SERVER, + dynamic = true, + doc = "The position that replication task start at, it can be set to earliest or latest (default).") + private String replicationStartAt = "latest"; + @FieldContext( category = CATEGORY_SERVER, dynamic = true, 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 28fa026e3f54a..db27e24709883 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 @@ -1919,7 +1919,14 @@ CompletableFuture startReplicator(String remoteCluster) { final CompletableFuture future = new CompletableFuture<>(); String name = PersistentReplicator.getReplicatorName(replicatorPrefix, remoteCluster); - ledger.asyncOpenCursor(name, new OpenCursorCallback() { + final InitialPosition initialPosition; + if (MessageId.earliest.toString() + .equalsIgnoreCase(getBrokerService().getPulsar().getConfiguration().getReplicationStartAt())) { + initialPosition = InitialPosition.Earliest; + } else { + initialPosition = InitialPosition.Latest; + } + ledger.asyncOpenCursor(name, initialPosition, new OpenCursorCallback() { @Override public void openCursorComplete(ManagedCursor cursor, Object ctx) { String localCluster = brokerService.pulsar().getConfiguration().getClusterName(); 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 2aee9a355c5d5..b7653509596a0 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 @@ -25,6 +25,7 @@ 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.assertTrue; import static org.testng.Assert.fail; @@ -34,6 +35,7 @@ import java.lang.reflect.Method; import java.time.Duration; import java.util.Arrays; +import java.util.Collections; import java.util.Optional; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -68,11 +70,12 @@ import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.TopicStats; +import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; -import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.util.FutureUtil; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; import org.mockito.Mockito; @@ -903,4 +906,100 @@ public void testReloadWithTopicLevelGeoReplication(ReplicationLevel replicationL }); } } + + protected void enableReplication(String topic) throws Exception { + admin1.topics().setReplicationClusters(topic, Arrays.asList(cluster1, cluster2)); + } + + protected void disableReplication(String topic) throws Exception { + admin1.topics().setReplicationClusters(topic, Arrays.asList(cluster1, cluster2)); + } + + @Test + public void testConfigReplicationStartAt() throws Exception { + // Initialize. + String ns1 = defaultTenant + "/ns_" + UUID.randomUUID().toString().replace("-", ""); + String subscription1 = "s1"; + admin1.namespaces().createNamespace(ns1); + if (!usingGlobalZK) { + admin2.namespaces().createNamespace(ns1); + } + + RetentionPolicies retentionPolicies = new RetentionPolicies(60 * 24, 1024); + admin1.namespaces().setRetention(ns1, retentionPolicies); + admin2.namespaces().setRetention(ns1, retentionPolicies); + + // 1. default config. + // Enable replication for topic1. + final String topic1 = BrokerTestUtil.newUniqueName("persistent://" + ns1 + "/tp_"); + admin1.topics().createNonPartitionedTopicAsync(topic1); + admin1.topics().createSubscription(topic1, subscription1, MessageId.earliest); + Producer p1 = client1.newProducer(Schema.STRING).topic(topic1).create(); + p1.send("msg-1"); + p1.close(); + enableReplication(topic1); + // Verify: since the replication was started at latest, there is no message to consume. + Consumer c1 = client2.newConsumer(Schema.STRING).topic(topic1).subscriptionName(subscription1) + .subscribe(); + Message msg1 = c1.receive(2, TimeUnit.SECONDS); + assertNull(msg1); + c1.close(); + disableReplication(topic1); + + // 2.Update config: start at "earliest". + admin1.brokers().updateDynamicConfiguration("replicationStartAt", MessageId.earliest.toString()); + Awaitility.await().untilAsserted(() -> { + pulsar1.getConfiguration().getReplicationStartAt().equalsIgnoreCase("earliest"); + }); + + final String topic2 = BrokerTestUtil.newUniqueName("persistent://" + ns1 + "/tp_"); + admin1.topics().createNonPartitionedTopicAsync(topic2); + admin1.topics().createSubscription(topic2, subscription1, MessageId.earliest); + Producer p2 = client1.newProducer(Schema.STRING).topic(topic2).create(); + p2.send("msg-1"); + p2.close(); + enableReplication(topic2); + // Verify: since the replication was started at earliest, there is one message to consume. + Consumer c2 = client2.newConsumer(Schema.STRING).topic(topic2).subscriptionName(subscription1) + .subscribe(); + Message msg2 = c2.receive(2, TimeUnit.SECONDS); + assertNotNull(msg2); + assertEquals(msg2.getValue(), "msg-1"); + c2.close(); + disableReplication(topic2); + + // 2.Update config: start at "latest". + admin1.brokers().updateDynamicConfiguration("replicationStartAt", MessageId.latest.toString()); + Awaitility.await().untilAsserted(() -> { + pulsar1.getConfiguration().getReplicationStartAt().equalsIgnoreCase("latest"); + }); + + final String topic3 = BrokerTestUtil.newUniqueName("persistent://" + ns1 + "/tp_"); + admin1.topics().createNonPartitionedTopicAsync(topic3); + admin1.topics().createSubscription(topic3, subscription1, MessageId.earliest); + Producer p3 = client1.newProducer(Schema.STRING).topic(topic3).create(); + p3.send("msg-1"); + p3.close(); + enableReplication(topic3); + // Verify: since the replication was started at latest, there is no message to consume. + Consumer c3 = client2.newConsumer(Schema.STRING).topic(topic3).subscriptionName(subscription1) + .subscribe(); + Message msg3 = c3.receive(2, TimeUnit.SECONDS); + assertNull(msg3); + c3.close(); + disableReplication(topic3); + + // cleanup. + // There is no good way to delete topics when using global ZK, skip cleanup. + admin1.namespaces().setNamespaceReplicationClusters(ns1, Collections.singleton(cluster1)); + admin1.namespaces().unload(ns1); + admin2.namespaces().setNamespaceReplicationClusters(ns1, Collections.singleton(cluster2)); + admin2.namespaces().unload(ns1); + admin1.topics().delete(topic1, false); + admin2.topics().delete(topic1, false); + admin1.topics().delete(topic2, false); + admin2.topics().delete(topic2, false); + admin1.topics().delete(topic3, false); + admin2.topics().delete(topic3, false); + } } 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 b8f8edce2477e..31e94f435f0f6 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 @@ -18,7 +18,19 @@ */ package org.apache.pulsar.broker.service; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import java.util.Arrays; +import java.util.HashSet; +import java.util.UUID; +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.Schema; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.awaitility.Awaitility; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -109,4 +121,44 @@ public void testExpandTopicPartitionsOnNamespaceLevelReplication() throws Except public void testReloadWithTopicLevelGeoReplication(ReplicationLevel replicationLevel) throws Exception { super.testReloadWithTopicLevelGeoReplication(replicationLevel); } + + @Test + @Override + public void testConfigReplicationStartAt() throws Exception { + // Initialize. + String ns1 = defaultTenant + "/ns_" + UUID.randomUUID().toString().replace("-", ""); + String subscription1 = "s1"; + admin1.namespaces().createNamespace(ns1); + RetentionPolicies retentionPolicies = new RetentionPolicies(60 * 24, 1024); + admin1.namespaces().setRetention(ns1, retentionPolicies); + admin2.namespaces().setRetention(ns1, retentionPolicies); + + // Update config: start at "earliest". + admin1.brokers().updateDynamicConfiguration("replicationStartAt", MessageId.earliest.toString()); + Awaitility.await().untilAsserted(() -> { + pulsar1.getConfiguration().getReplicationStartAt().equalsIgnoreCase("earliest"); + }); + + // Verify: since the replication was started at earliest, there is one message to consume. + final String topic1 = BrokerTestUtil.newUniqueName("persistent://" + ns1 + "/tp_"); + admin1.topics().createNonPartitionedTopicAsync(topic1); + admin1.topics().createSubscription(topic1, subscription1, MessageId.earliest); + org.apache.pulsar.client.api.Producer p1 = client1.newProducer(Schema.STRING).topic(topic1).create(); + p1.send("msg-1"); + p1.close(); + + admin1.namespaces().setNamespaceReplicationClusters(ns1, new HashSet<>(Arrays.asList(cluster1, cluster2))); + org.apache.pulsar.client.api.Consumer c1 = client2.newConsumer(Schema.STRING).topic(topic1) + .subscriptionName(subscription1).subscribe(); + Message msg2 = c1.receive(2, TimeUnit.SECONDS); + assertNotNull(msg2); + assertEquals(msg2.getValue(), "msg-1"); + c1.close(); + + // cleanup. + admin1.brokers().updateDynamicConfiguration("replicationStartAt", MessageId.latest.toString()); + Awaitility.await().untilAsserted(() -> { + pulsar1.getConfiguration().getReplicationStartAt().equalsIgnoreCase("latest"); + }); + } } From 3d16cf7357dd17cc0d12b3eff1d5d65e3aee3e7a Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Mon, 1 Jul 2024 10:01:10 +0800 Subject: [PATCH 592/699] [feat][broker][branch-3.0] PIP-321 Introduce allowed-cluster at the namespace level (#22378) (#22960) (cherry-picked from commit https://github.com/apache/pulsar/commit/36bae695fb07f3ee790bee603149c4c2712187e0) Co-authored-by: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> (cherry picked from commit 7b2e72464b83507c0cf17ff0b364a4883d682d1f) --- .../pulsar/broker/admin/AdminResource.java | 44 +++--- .../broker/admin/impl/NamespacesBase.java | 79 ++++++++++- .../pulsar/broker/admin/v2/Namespaces.java | 46 +++++++ .../service/persistent/PersistentTopic.java | 94 ++++++++----- .../pulsar/broker/web/PulsarWebResource.java | 15 ++- .../namespace/NamespaceServiceTest.java | 127 ++++++++++++++++++ .../pulsar/broker/service/ReplicatorTest.java | 56 ++++++++ .../pulsar/client/admin/Namespaces.java | 84 ++++++++++++ .../pulsar/common/policies/data/Policies.java | 5 +- .../client/admin/internal/NamespacesImpl.java | 22 +++ .../pulsar/admin/cli/PulsarAdminToolTest.java | 8 ++ .../pulsar/admin/cli/CmdNamespaces.java | 32 +++++ .../common/policies/data/PolicyName.java | 3 +- 13 files changed, 546 insertions(+), 69 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 36e2ff0a34890..bd084f84e2aa0 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 @@ -321,32 +321,28 @@ protected Policies getNamespacePolicies(NamespaceName namespaceName) { } protected CompletableFuture getNamespacePoliciesAsync(NamespaceName namespaceName) { - return namespaceResources().getPoliciesAsync(namespaceName).thenCompose(policies -> { - if (policies.isPresent()) { - return pulsar() - .getNamespaceService() - .getNamespaceBundleFactory() - .getBundlesAsync(namespaceName) - .thenCompose(bundles -> { - BundlesData bundleData = null; - try { - bundleData = bundles.getBundlesData(); - } catch (Exception e) { - log.error("[{}] Failed to get namespace policies {}", clientAppId(), namespaceName, e); - return FutureUtil.failedFuture(new RestException(e)); - } - policies.get().bundles = bundleData != null ? bundleData : policies.get().bundles; - if (policies.get().is_allow_auto_update_schema == null) { - // the type changed from boolean to Boolean. return broker value here for keeping compatibility. - policies.get().is_allow_auto_update_schema = pulsar().getConfig() - .isAllowAutoUpdateSchemaEnabled(); + CompletableFuture result = new CompletableFuture<>(); + namespaceResources().getPoliciesAsync(namespaceName) + .thenCombine(getLocalPolicies().getLocalPoliciesAsync(namespaceName), (pl, localPolicies) -> { + if (pl.isPresent()) { + Policies policies = pl.get(); + localPolicies.ifPresent(value -> policies.bundles = value.bundles); + if (policies.is_allow_auto_update_schema == null) { + // the type changed from boolean to Boolean. return + // broker value here for keeping compatibility. + policies.is_allow_auto_update_schema = pulsar().getConfig() + .isAllowAutoUpdateSchemaEnabled(); + } + result.complete(policies); + } else { + result.completeExceptionally(new RestException(Status.NOT_FOUND, "Namespace does not exist")); } - return CompletableFuture.completedFuture(policies.get()); + return null; + }).exceptionally(ex -> { + result.completeExceptionally(ex.getCause()); + return null; }); - } else { - return FutureUtil.failedFuture(new RestException(Status.NOT_FOUND, "Namespace does not exist")); - } - }); + return result; } protected BacklogQuota namespaceBacklogQuota(NamespaceName namespace, 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 1d750bee1bc95..f9170e65ca7e7 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 @@ -702,9 +702,21 @@ protected CompletableFuture internalSetNamespaceReplicationClusters(List - validateClusterForTenantAsync( - namespaceName.getTenant(), clusterId)); + .thenCompose(__ -> getNamespacePoliciesAsync(this.namespaceName) + .thenCompose(nsPolicies -> { + if (nsPolicies.allowed_clusters.isEmpty()) { + return validateClusterForTenantAsync( + namespaceName.getTenant(), clusterId); + } + if (!nsPolicies.allowed_clusters.contains(clusterId)) { + String msg = String.format("Cluster [%s] is not in the " + + "list of allowed clusters list for namespace " + + "[%s]", clusterId, namespaceName.toString()); + log.info(msg); + throw new RestException(Status.FORBIDDEN, msg); + } + return CompletableFuture.completedFuture(null); + })); }).collect(Collectors.toList()); return FutureUtil.waitForAll(futures).thenApply(__ -> replicationClusterSet); })) @@ -2695,4 +2707,65 @@ protected void internalRemoveBacklogQuota(AsyncResponse asyncResponse, BacklogQu return null; }); } + + protected CompletableFuture internalSetNamespaceAllowedClusters(List clusterIds) { + return validateNamespacePolicyOperationAsync(namespaceName, PolicyName.ALLOW_CLUSTERS, PolicyOperation.WRITE) + .thenCompose(__ -> validatePoliciesReadOnlyAccessAsync()) + // Allowed clusters in the namespace policy should be included in the allowed clusters in the tenant + // policy. + .thenCompose(__ -> FutureUtil.waitForAll(clusterIds.stream().map(clusterId -> + validateClusterForTenantAsync(namespaceName.getTenant(), clusterId)) + .collect(Collectors.toList()))) + // Allowed clusters should include all the existed replication clusters and could not contain global + // cluster. + .thenCompose(__ -> { + checkNotNull(clusterIds, "ClusterIds should not be null"); + if (clusterIds.contains("global")) { + throw new RestException(Status.PRECONDITION_FAILED, + "Cannot specify global in the list of allowed clusters"); + } + return getNamespacePoliciesAsync(this.namespaceName).thenApply(namespacePolicies -> { + namespacePolicies.replication_clusters.forEach(replicationCluster -> { + if (!clusterIds.contains(replicationCluster)) { + throw new RestException(Status.BAD_REQUEST, + String.format("Allowed clusters do not contain the replication cluster %s. " + + "Please remove the replication cluster if the cluster is not allowed " + + "for this namespace", replicationCluster)); + } + }); + return Sets.newHashSet(clusterIds); + }); + }) + // Verify the allowed clusters are valid and they do not contain the peer clusters. + .thenCompose(allowedClusters -> clustersAsync() + .thenCompose(clusters -> { + List> futures = + allowedClusters.stream().map(clusterId -> { + if (!clusters.contains(clusterId)) { + throw new RestException(Status.FORBIDDEN, + "Invalid cluster id: " + clusterId); + } + return validatePeerClusterConflictAsync(clusterId, allowedClusters); + }).collect(Collectors.toList()); + return FutureUtil.waitForAll(futures).thenApply(__ -> allowedClusters); + })) + // Update allowed clusters into policies. + .thenCompose(allowedClusterSet -> updatePoliciesAsync(namespaceName, policies -> { + policies.allowed_clusters = allowedClusterSet; + return policies; + })); + } + + protected CompletableFuture> internalGetNamespaceAllowedClustersAsync() { + return validateNamespacePolicyOperationAsync(namespaceName, PolicyName.ALLOW_CLUSTERS, PolicyOperation.READ) + .thenAccept(__ -> { + if (!namespaceName.isGlobal()) { + throw new RestException(Status.PRECONDITION_FAILED, + "Cannot get the allowed clusters for a non-global namespace"); + } + }).thenCompose(__ -> getNamespacePoliciesAsync(namespaceName)) + .thenApply(policies -> policies.allowed_clusters); + } + + } 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 259195056e326..f1f4c62ed3439 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 @@ -2976,5 +2976,51 @@ public void removeNamespaceEntryFilters(@Suspended AsyncResponse asyncResponse, }); } + @POST + @Path("/{tenant}/{namespace}/allowedClusters") + @ApiOperation(value = "Set the allowed clusters for a namespace.") + @ApiResponses(value = { + @ApiResponse(code = 400, message = "The list of allowed clusters should include all replication clusters."), + @ApiResponse(code = 403, message = "The requester does not have admin permissions."), + @ApiResponse(code = 404, message = "The specified tenant, cluster, or namespace does not exist."), + @ApiResponse(code = 409, message = "A peer-cluster cannot be part of an allowed-cluster."), + @ApiResponse(code = 412, message = "The namespace is not global or the provided cluster IDs are invalid.")}) + public void setNamespaceAllowedClusters(@Suspended AsyncResponse asyncResponse, + @PathParam("tenant") String tenant, + @PathParam("namespace") String namespace, + @ApiParam(value = "List of allowed clusters", required = true) + List clusterIds) { + validateNamespaceName(tenant, namespace); + internalSetNamespaceAllowedClusters(clusterIds) + .thenAccept(asyncResponse::resume) + .exceptionally(e -> { + log.error("[{}] Failed to set namespace allowed clusters on namespace {}", + clientAppId(), namespace, e); + resumeAsyncResponseExceptionally(asyncResponse, e); + return null; + }); + } + + @GET + @Path("/{tenant}/{namespace}/allowedClusters") + @ApiOperation(value = "Get the allowed clusters for a namespace.", + response = String.class, responseContainer = "List") + @ApiResponses(value = {@ApiResponse(code = 403, message = "Don't have admin permission"), + @ApiResponse(code = 404, message = "Tenant or cluster or namespace doesn't exist"), + @ApiResponse(code = 412, message = "Namespace is not global")}) + public void getNamespaceAllowedClusters(@Suspended AsyncResponse asyncResponse, + @PathParam("tenant") String tenant, + @PathParam("namespace") String namespace) { + validateNamespaceName(tenant, namespace); + internalGetNamespaceAllowedClustersAsync() + .thenAccept(asyncResponse::resume) + .exceptionally(e -> { + log.error("[{}] Failed to get namespace allowed clusters on namespace {}", clientAppId(), + namespace, e); + resumeAsyncResponseExceptionally(asyncResponse, e); + return null; + }); + } + private static final Logger log = LoggerFactory.getLogger(Namespaces.class); } 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 db27e24709883..e78211d5a1d2e 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 @@ -1758,52 +1758,78 @@ public CompletableFuture checkReplication() { if (log.isDebugEnabled()) { log.debug("[{}] Checking replication status", name); } - List configuredClusters = topicPolicies.getReplicationClusters().get(); if (CollectionUtils.isEmpty(configuredClusters)) { log.warn("[{}] No replication clusters configured", name); return CompletableFuture.completedFuture(null); } - int newMessageTTLInSeconds = topicPolicies.getMessageTTLInSeconds().get(); - String localCluster = brokerService.pulsar().getConfiguration().getClusterName(); - // if local cluster is removed from global namespace cluster-list : then delete topic forcefully - // because pulsar doesn't serve global topic without local repl-cluster configured. - if (TopicName.get(topic).isGlobal() && !configuredClusters.contains(localCluster)) { - log.info("Deleting topic [{}] because local cluster is not part of " - + " global namespace repl list {}", topic, configuredClusters); - return deleteForcefully(); - } - - removeTerminatedReplicators(replicators); - List> futures = new ArrayList<>(); - - // Check for missing replicators - for (String cluster : configuredClusters) { - if (cluster.equals(localCluster)) { - continue; - } - if (!replicators.containsKey(cluster)) { - futures.add(startReplicator(cluster)); - } - } - - // Check for replicators to be stopped - replicators.forEach((cluster, replicator) -> { - // Update message TTL - ((PersistentReplicator) replicator).updateMessageTTL(newMessageTTLInSeconds); - if (!cluster.equals(localCluster)) { - if (!configuredClusters.contains(cluster)) { - futures.add(removeReplicator(cluster)); + return checkAllowedCluster(localCluster).thenCompose(success -> { + if (!success) { + // if local cluster is removed from global namespace cluster-list : then delete topic forcefully + // because pulsar doesn't serve global topic without local repl-cluster configured. + return deleteForcefully(); + } + + int newMessageTTLInSeconds = topicPolicies.getMessageTTLInSeconds().get(); + + removeTerminatedReplicators(replicators); + List> futures = new ArrayList<>(); + + // The replication clusters at namespace level will get local cluster when creating a namespace. + // If there are only one cluster in the replication clusters, it means the replication is not enabled. + // If the cluster 1 and cluster 2 use the same configuration store and the namespace is created in cluster1 + // without enabling geo-replication, then the replication clusters always has cluster1. + // + // When a topic under the namespace is load in the cluster2, the `cluster1` may be identified as + // remote cluster and start geo-replication. This check is to avoid the above case. + if (!(configuredClusters.size() == 1 && replicators.isEmpty())) { + // Check for missing replicators + for (String cluster : configuredClusters) { + if (cluster.equals(localCluster)) { + continue; + } + if (!replicators.containsKey(cluster)) { + futures.add(startReplicator(cluster)); + } } + // Check for replicators to be stopped + replicators.forEach((cluster, replicator) -> { + // Update message TTL + ((PersistentReplicator) replicator).updateMessageTTL(newMessageTTLInSeconds); + if (!cluster.equals(localCluster)) { + if (!configuredClusters.contains(cluster)) { + futures.add(removeReplicator(cluster)); + } + } + }); } - }); - futures.add(checkShadowReplication()); + futures.add(checkShadowReplication()); - return FutureUtil.waitForAll(futures); + return FutureUtil.waitForAll(futures); + }); + } + + private CompletableFuture checkAllowedCluster(String localCluster) { + List replicationClusters = topicPolicies.getReplicationClusters().get(); + return brokerService.pulsar().getPulsarResources().getNamespaceResources() + .getPoliciesAsync(TopicName.get(topic).getNamespaceObject()).thenCompose(policiesOptional -> { + Set allowedClusters = Set.of(); + if (policiesOptional.isPresent()) { + allowedClusters = policiesOptional.get().allowed_clusters; + } + if (TopicName.get(topic).isGlobal() && !replicationClusters.contains(localCluster) + && !allowedClusters.contains(localCluster)) { + log.warn("Local cluster {} is not part of global namespace repl list {} and allowed list {}", + localCluster, replicationClusters, allowedClusters); + return CompletableFuture.completedFuture(false); + } else { + return CompletableFuture.completedFuture(true); + } + }); } private CompletableFuture checkShadowReplication() { 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 2f437962002a3..dafad019613a6 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 @@ -902,14 +902,16 @@ public static CompletableFuture checkLocalOrGetPeerReplicationC log.warn(msg); validationFuture.completeExceptionally(new RestException(Status.NOT_FOUND, "Namespace is deleted")); - } else if (policies.replication_clusters.isEmpty()) { + } else if (policies.replication_clusters.isEmpty() && policies.allowed_clusters.isEmpty()) { String msg = String.format( "Namespace does not have any clusters configured : local_cluster=%s ns=%s", localCluster, namespace.toString()); log.warn(msg); validationFuture.completeExceptionally(new RestException(Status.PRECONDITION_FAILED, msg)); - } else if (!policies.replication_clusters.contains(localCluster)) { - getOwnerFromPeerClusterListAsync(pulsarService, policies.replication_clusters) + } else if (!policies.replication_clusters.contains(localCluster) && !policies.allowed_clusters + .contains(localCluster)) { + getOwnerFromPeerClusterListAsync(pulsarService, policies.replication_clusters, + policies.allowed_clusters) .thenAccept(ownerPeerCluster -> { if (ownerPeerCluster != null) { // found a peer that own this namespace @@ -949,9 +951,9 @@ public static CompletableFuture checkLocalOrGetPeerReplicationC } private static CompletableFuture getOwnerFromPeerClusterListAsync(PulsarService pulsar, - Set replicationClusters) { + Set replicationClusters, Set allowedClusters) { String currentCluster = pulsar.getConfiguration().getClusterName(); - if (replicationClusters == null || replicationClusters.isEmpty() || isBlank(currentCluster)) { + if (replicationClusters.isEmpty() && allowedClusters.isEmpty() || isBlank(currentCluster)) { return CompletableFuture.completedFuture(null); } @@ -961,7 +963,8 @@ private static CompletableFuture getOwnerFromPeerClusterListAsy return CompletableFuture.completedFuture(null); } for (String peerCluster : cluster.get().getPeerClusterNames()) { - if (replicationClusters.contains(peerCluster)) { + if (replicationClusters.contains(peerCluster) + || allowedClusters.contains(peerCluster)) { return pulsar.getPulsarResources().getClusterResources().getClusterAsync(peerCluster) .thenApply(ret -> { if (!ret.isPresent()) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java index 38a60165d5606..e975fe3cfa926 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java @@ -32,6 +32,7 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; import com.github.benmanes.caffeine.cache.AsyncLoadingCache; +import com.google.common.collect.Sets; import com.google.common.hash.Hashing; import java.lang.reflect.Field; import java.lang.reflect.Method; @@ -40,6 +41,7 @@ import java.util.EnumSet; import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -64,6 +66,7 @@ import org.apache.pulsar.broker.service.BrokerTestBase; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.admin.Namespaces; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.PulsarClient; @@ -76,8 +79,11 @@ import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.BundlesData; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.LocalPolicies; import org.apache.pulsar.common.policies.data.Policies; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.apache.pulsar.metadata.api.GetResult; @@ -828,6 +834,127 @@ public void testCheckTopicExists(String topicDomain) throws Exception { }); } + @Test + public void testAllowedClustersAtNamespaceLevelShouldBeIncludedInAllowedClustersAtTenantLevel() throws Exception { + // 1. Setup + pulsar.getConfiguration().setForceDeleteNamespaceAllowed(true); + pulsar.getConfiguration().setForceDeleteTenantAllowed(true); + Set tenantAllowedClusters = Set.of("test", "r1", "r2"); + Set allowedClusters1 = Set.of("test", "r1", "r2", "r3"); + Set allowedClusters2 = Set.of("test", "r1", "r2"); + Set clusters = Set.of("r1", "r2", "r3", "r4"); + final String tenant = "my-tenant"; + final String namespace = tenant + "/testAllowedCluster"; + admin.tenants().createTenant(tenant, + new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet("test"))); + admin.namespaces().createNamespace(namespace); + pulsar.getPulsarResources().getTenantResources().updateTenantAsync(tenant, tenantInfo -> + TenantInfo.builder().allowedClusters(tenantAllowedClusters).build()); + for (String cluster : clusters) { + pulsar.getPulsarResources().getClusterResources().createCluster(cluster, ClusterData.builder().build()); + } + // 2. Verify + admin.namespaces().setNamespaceAllowedClusters(namespace, allowedClusters2); + + try { + admin.namespaces().setNamespaceAllowedClusters(namespace, allowedClusters1); + fail(); + } catch (PulsarAdminException e) { + assertEquals(e.getStatusCode(), 403); + assertEquals(e.getMessage(), + "Cluster [r3] is not in the list of allowed clusters list for tenant [my-tenant]"); + } + // 3. Clean up + admin.namespaces().deleteNamespace(namespace, true); + admin.tenants().deleteTenant(tenant, true); + for (String cluster : clusters) { + pulsar.getPulsarResources().getClusterResources().deleteCluster(cluster); + } + pulsar.getConfiguration().setForceDeleteNamespaceAllowed(false); + pulsar.getConfiguration().setForceDeleteTenantAllowed(false); + } + + /** + * Test case: + * 1. Replication clusters should be included in the allowed clusters. For compatibility, the replication + * clusters could be set before the allowed clusters are set. + * 2. Peer cluster can not be a part of the allowed clusters. + */ + @Test + public void testNewAllowedClusterAdminAPIAndItsImpactOnReplicationClusterAPI() throws Exception { + // 1. Setup + pulsar.getConfiguration().setForceDeleteNamespaceAllowed(true); + pulsar.getConfiguration().setForceDeleteTenantAllowed(true); + // Setup: Prepare cluster resource, tenant and namespace + Set replicationClusters = Set.of("test", "r1", "r2"); + Set tenantAllowedClusters = Set.of("test", "r1", "r2", "r3"); + Set allowedClusters = Set.of("test", "r1", "r2", "r3"); + Set clusters = Set.of("r1", "r2", "r3", "r4"); + final String tenant = "my-tenant"; + final String namespace = tenant + "/testAllowedCluster"; + admin.tenants().createTenant(tenant, + new TenantInfoImpl(Sets.newHashSet("appid1"), Sets.newHashSet("test"))); + admin.namespaces().createNamespace(namespace); + pulsar.getPulsarResources().getTenantResources().updateTenantAsync(tenant, tenantInfo -> + TenantInfo.builder().allowedClusters(tenantAllowedClusters).build()); + + Namespaces namespaces = admin.namespaces(); + for (String cluster : clusters) { + pulsar.getPulsarResources().getClusterResources().createCluster(cluster, ClusterData.builder().build()); + } + // 2. Verify + // 2.1 Replication clusters should be included in the allowed clusters. + + // SUCCESS + // 2.1.1. Set replication clusters without allowed clusters at namespace level. + namespaces.setNamespaceReplicationClusters(namespace, replicationClusters); + // 2..1.2 Set allowed clusters. + namespaces.setNamespaceAllowedClusters(namespace, allowedClusters); + // 2.1.3. Get allowed clusters and replication clusters. + List allowedClustersResponse = namespaces.getNamespaceAllowedClusters(namespace); + + List replicationClustersResponse = namespaces.getNamespaceReplicationClusters(namespace); + + assertEquals(replicationClustersResponse.size(), replicationClusters.size()); + assertEquals(allowedClustersResponse.size(), allowedClusters.size()); + + // FAIL + // 2.1.4. Fail: Set allowed clusters whose scope is smaller than replication clusters. + Set allowedClustersSmallScope = Set.of("r1", "r3"); + try { + namespaces.setNamespaceAllowedClusters(namespace, allowedClustersSmallScope); + fail(); + } catch (PulsarAdminException ignore) {} + // 2.1.5. Fail: Set replication clusters whose scope is excel the allowed clusters. + Set replicationClustersExcel = Set.of("r1", "r4"); + try { + namespaces.setNamespaceReplicationClusters(namespace, replicationClustersExcel); + fail(); + //Todo: The status code in the old implementation is confused. + } catch (PulsarAdminException.NotAuthorizedException ignore) {} + + // 2.2 Peer cluster can not be a part of the allowed clusters. + LinkedHashSet peerCluster = new LinkedHashSet<>(); + peerCluster.add("r2"); + pulsar.getPulsarResources().getClusterResources().deleteCluster("r1"); + pulsar.getPulsarResources().getClusterResources().createCluster("r1", + ClusterData.builder().peerClusterNames(peerCluster).build()); + try { + namespaces.setNamespaceAllowedClusters(namespace, Set.of("test", "r1", "r2", "r3")); + fail(); + } catch (PulsarAdminException.ConflictException ignore) {} + + // CleanUp: Namespace with replication clusters can not be deleted by force. + namespaces.setNamespaceReplicationClusters(namespace, Set.of(conf.getClusterName())); + admin.namespaces().deleteNamespace(namespace, true); + admin.tenants().deleteTenant(tenant, true); + for (String cluster : clusters) { + pulsar.getPulsarResources().getClusterResources().deleteCluster(cluster); + } + pulsar.getConfiguration().setForceDeleteNamespaceAllowed(false); + pulsar.getConfiguration().setForceDeleteTenantAllowed(false); + } + /** * 1. Manually trigger "LoadReportUpdaterTask" * 2. Registry another new zk-node-listener "waitForBrokerChangeNotice". 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 3cc2ca2457a4b..75ff51055fc7e 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 @@ -62,6 +62,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.commons.lang3.RandomUtils; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; @@ -1782,6 +1783,61 @@ public void testReplicatorWithTTL() throws Exception { assertEquals(result, Lists.newArrayList("V1", "V2", "V3", "V4")); } + @Test + public void testEnableReplicationWithNamespaceAllowedClustersPolices() throws Exception { + log.info("--- testEnableReplicationWithNamespaceAllowedClustersPolices ---"); + String namespace1 = "pulsar/ns" + RandomUtils.nextLong(); + admin1.namespaces().createNamespace(namespace1); + admin2.namespaces().createNamespace(namespace1 + "init_cluster_node"); + admin1.namespaces().setNamespaceAllowedClusters(namespace1, Sets.newHashSet("r1", "r2")); + final TopicName topicName = TopicName.get( + BrokerTestUtil.newUniqueName("persistent://" + namespace1 + "/testReplicatorProducerNotExceed1")); + + @Cleanup PulsarClient client1 = PulsarClient + .builder() + .serviceUrl(pulsar1.getBrokerServiceUrl()) + .build(); + @Cleanup Producer producer = client1 + .newProducer() + .topic(topicName.toString()) + .create(); + producer.newMessage().send(); + // Enable replication at the topic level in the cluster1. + admin1.topics().setReplicationClusters(topicName.toString(), List.of("r1", "r2")); + + PersistentTopic persistentTopic1 = (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName.toString(), + false) + .get() + .get(); + // Verify the replication from cluster1 to cluster2 is ready, but the replication form the cluster2 to cluster1 + // is not ready. + Awaitility.await().untilAsserted(() -> { + ConcurrentOpenHashMap replicatorMap = persistentTopic1.getReplicators(); + assertEquals(replicatorMap.size(), 1); + Replicator replicator = replicatorMap.get(replicatorMap.keys().get(0)); + assertTrue(replicator.isConnected()); + }); + + PersistentTopic persistentTopic2 = (PersistentTopic) pulsar2.getBrokerService().getTopic(topicName.toString(), + false) + .get() + .get(); + + Awaitility.await().untilAsserted(() -> { + ConcurrentOpenHashMap replicatorMap = persistentTopic2.getReplicators(); + assertEquals(replicatorMap.size(), 0); + }); + // Enable replication at the topic level in the cluster2. + admin2.topics().setReplicationClusters(topicName.toString(), List.of("r1", "r2")); + // Verify the replication between cluster1 and cluster2 is ready. + Awaitility.await().untilAsserted(() -> { + ConcurrentOpenHashMap replicatorMap = persistentTopic2.getReplicators(); + assertEquals(replicatorMap.size(), 1); + Replicator replicator = replicatorMap.get(replicatorMap.keys().get(0)); + assertTrue(replicator.isConnected()); + }); + } + private void pauseReplicator(PersistentReplicator replicator) { Awaitility.await().untilAsserted(() -> { assertTrue(replicator.isConnected()); 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 2690df658b7be..32c659dc01db5 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 @@ -4623,4 +4623,88 @@ void setIsAllowAutoUpdateSchema(String namespace, boolean isAllowAutoUpdateSchem * @return */ CompletableFuture removeNamespaceEntryFiltersAsync(String namespace); + + /** + * Get the allowed clusters for a namespace. + *

    + * Response example: + * + *

    +     * ["use", "usw", "usc"]
    +     * 
    + * + * @param namespace + * Namespace name + * @throws NotAuthorizedException + * Don't have admin permission + * @throws NotFoundException + * Namespace does not exist + * @throws PreconditionFailedException + * Namespace is not global + * @throws PulsarAdminException + * Unexpected error + */ + List getNamespaceAllowedClusters(String namespace) throws PulsarAdminException; + + /** + * Get the allowed clusters for a namespace asynchronously. + *

    + * Response example: + * + *

    +     * ["use", "usw", "usc"]
    +     * 
    + * + * @param namespace + * Namespace name + */ + CompletableFuture> getNamespaceAllowedClustersAsync(String namespace); + + /** + * Set the allowed clusters for a namespace. + *

    + * Request example: + * + *

    +     * ["us-west", "us-east", "us-cent"]
    +     * 
    + * + * @param namespace + * Namespace name + * @param clusterIds + * Pulsar Cluster Ids + * + * @throws ConflictException + * Peer-cluster cannot be part of an allowed-cluster + * @throws NotAuthorizedException + * Don't have admin permission + * @throws NotFoundException + * Namespace does not exist + * @throws PreconditionFailedException + * Namespace is not global + * @throws PreconditionFailedException + * Invalid cluster ids + * @throws PulsarAdminException + * The list of allowed clusters should include all replication clusters. + * @throws PulsarAdminException + * Unexpected error + */ + void setNamespaceAllowedClusters(String namespace, Set clusterIds) throws PulsarAdminException; + + /** + * Set the allowed clusters for a namespace asynchronously. + *

    + * Request example: + * + *

    +     * ["us-west", "us-east", "us-cent"]
    +     * 
    + * + * @param namespace + * Namespace name + * @param clusterIds + * Pulsar Cluster Ids + */ + CompletableFuture setNamespaceAllowedClustersAsync(String namespace, Set clusterIds); + } diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/Policies.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/Policies.java index 066fdf1df4f09..4e0c68bed3a88 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/Policies.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/common/policies/data/Policies.java @@ -36,6 +36,8 @@ public class Policies { public final AuthPolicies auth_policies = AuthPolicies.builder().build(); @SuppressWarnings("checkstyle:MemberName") public Set replication_clusters = new HashSet<>(); + @SuppressWarnings("checkstyle:MemberName") + public Set allowed_clusters = new HashSet<>(); public BundlesData bundles; @SuppressWarnings("checkstyle:MemberName") public Map backlog_quota_map = new HashMap<>(); @@ -135,7 +137,7 @@ public enum BundleType { @Override public int hashCode() { - return Objects.hash(auth_policies, replication_clusters, + return Objects.hash(auth_policies, replication_clusters, allowed_clusters, backlog_quota_map, publishMaxMessageRate, clusterDispatchRate, topicDispatchRate, subscriptionDispatchRate, replicatorDispatchRate, clusterSubscribeRate, deduplicationEnabled, autoTopicCreationOverride, @@ -165,6 +167,7 @@ public boolean equals(Object obj) { Policies other = (Policies) obj; return Objects.equals(auth_policies, other.auth_policies) && Objects.equals(replication_clusters, other.replication_clusters) + && Objects.equals(allowed_clusters, other.allowed_clusters) && Objects.equals(backlog_quota_map, other.backlog_quota_map) && Objects.equals(clusterDispatchRate, other.clusterDispatchRate) && Objects.equals(topicDispatchRate, other.topicDispatchRate) 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 59f0ef3b34763..792fbdc91d1ff 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 @@ -1950,4 +1950,26 @@ public CompletableFuture removeNamespaceEntryFiltersAsync(String namespace WebTarget path = namespacePath(ns, "entryFilters"); return asyncDeleteRequest(path); } + + @Override + public List getNamespaceAllowedClusters(String namespace) throws PulsarAdminException { + return sync(() -> getNamespaceAllowedClustersAsync(namespace)); + } + + @Override + public CompletableFuture> getNamespaceAllowedClustersAsync(String namespace) { + return asyncGetNamespaceParts(new FutureCallback>(){}, namespace, "allowedClusters"); + } + + @Override + public void setNamespaceAllowedClusters(String namespace, Set clusterIds) throws PulsarAdminException { + sync(() -> setNamespaceAllowedClustersAsync(namespace, clusterIds)); + } + + @Override + public CompletableFuture setNamespaceAllowedClustersAsync(String namespace, Set clusterIds) { + NamespaceName ns = NamespaceName.get(namespace); + WebTarget path = namespacePath(ns, "allowedClusters"); + return asyncPostRequest(path, Entity.entity(clusterIds, MediaType.APPLICATION_JSON)); + } } diff --git a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java index 8dc6f752c09ad..64f0f8bebb2f3 100644 --- a/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java +++ b/pulsar-client-tools-test/src/test/java/org/apache/pulsar/admin/cli/PulsarAdminToolTest.java @@ -428,6 +428,14 @@ public void namespaces() throws Exception { namespaces.run(split("get-clusters myprop/clust/ns1")); verify(mockNamespaces).getNamespaceReplicationClusters("myprop/clust/ns1"); + namespaces.run(split("set-allowed-clusters myprop/clust/ns1 -c use,usw,usc")); + verify(mockNamespaces).setNamespaceAllowedClusters("myprop/clust/ns1", + Sets.newHashSet("use", "usw", "usc")); + + namespaces.run(split("get-allowed-clusters myprop/clust/ns1")); + verify(mockNamespaces).getNamespaceAllowedClusters("myprop/clust/ns1"); + + namespaces.run(split("set-subscription-types-enabled myprop/clust/ns1 -t Shared,Failover")); verify(mockNamespaces).setSubscriptionTypesEnabled("myprop/clust/ns1", Sets.newHashSet(SubscriptionType.Shared, SubscriptionType.Failover)); 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 5f2006edeafdf..ac0b424301bb8 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 @@ -2680,6 +2680,35 @@ void run() throws PulsarAdminException { } } + @Parameters(commandDescription = "Set allowed clusters for a namespace") + private class SetAllowedClusters extends CliCommand { + @Parameter(description = "tenant/namespace", required = true) + private java.util.List params; + + @Parameter(names = { "--clusters", + "-c" }, description = "Replication Cluster Ids list (comma separated values)", required = true) + private String clusterIds; + + @Override + void run() throws PulsarAdminException { + String namespace = validateNamespace(params); + List clusters = Lists.newArrayList(clusterIds.split(",")); + getAdmin().namespaces().setNamespaceAllowedClusters(namespace, Sets.newHashSet(clusters)); + } + } + + @Parameters(commandDescription = "Get allowed clusters for a namespace") + private class GetAllowedClusters extends CliCommand { + @Parameter(description = "tenant/namespace", required = true) + private java.util.List params; + + @Override + void run() throws PulsarAdminException { + String namespace = validateNamespace(params); + print(getAdmin().namespaces().getNamespaceAllowedClusters(namespace)); + } + } + public CmdNamespaces(Supplier admin) { super("namespaces", admin); jcommander.addCommand("list", new GetNamespacesPerProperty()); @@ -2707,6 +2736,9 @@ public CmdNamespaces(Supplier admin) { jcommander.addCommand("get-subscription-types-enabled", new GetSubscriptionTypesEnabled()); jcommander.addCommand("remove-subscription-types-enabled", new RemoveSubscriptionTypesEnabled()); + jcommander.addCommand("set-allowed-clusters", new SetAllowedClusters()); + jcommander.addCommand("get-allowed-clusters", new GetAllowedClusters()); + jcommander.addCommand("get-backlog-quotas", new GetBacklogQuotaMap()); jcommander.addCommand("set-backlog-quota", new SetBacklogQuota()); jcommander.addCommand("remove-backlog-quota", new RemoveBacklogQuota()); 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 456d4b9270cd6..a01e1e90027be 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 @@ -51,5 +51,6 @@ public enum PolicyName { MAX_TOPICS, RESOURCEGROUP, ENTRY_FILTERS, - SHADOW_TOPIC + SHADOW_TOPIC, + ALLOW_CLUSTERS } From 8ced7875e325513c888e17a662d32677af130072 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 13 Jun 2024 01:24:04 +0300 Subject: [PATCH 593/699] [improve][misc] Upgrade to Netty 4.1.111.Final and switch to use grpc-netty-shaded (#22892) (cherry picked from commit 75d7e557d84bf2cca2ec791dfe8479b8a6df7875) (cherry picked from commit a982d7b2efb3aad456c9c0ec921c8f7bbcb48ab3) --- buildtools/pom.xml | 2 +- distribution/server/pom.xml | 13 ++ .../server/src/assemble/LICENSE.bin.txt | 51 +++-- .../shell/src/assemble/LICENSE.bin.txt | 40 ++-- jetcd-core-shaded/pom.xml | 187 ++++++++++++++++++ pom.xml | 60 +++++- pulsar-broker/pom.xml | 12 ++ pulsar-functions/instance/pom.xml | 9 +- pulsar-metadata/pom.xml | 11 +- .../metadata/impl/EtcdMetadataStore.java | 6 +- pulsar-sql/presto-distribution/LICENSE | 53 +++-- .../src/assembly/assembly.xml | 3 + .../src/assembly/assembly.xml | 3 + src/check-binary-license.sh | 2 +- 14 files changed, 361 insertions(+), 91 deletions(-) create mode 100644 jetcd-core-shaded/pom.xml diff --git a/buildtools/pom.xml b/buildtools/pom.xml index 1400c77e508ad..ece9e78b2d977 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -47,7 +47,7 @@ 4.1 10.14.2 3.1.2 - 4.1.108.Final + 4.1.111.Final 4.2.3 32.1.1-jre 1.10.12 diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index 818c5537c616a..ede175f6f4ae8 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -46,6 +46,19 @@ ${project.version}
    + + ${project.groupId} + pulsar-metadata + ${project.version} + + + + ${project.groupId} + jetcd-core-shaded + ${project.version} + shaded + + ${project.groupId} pulsar-proxy diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index c5e20faf13ad5..b663b839ec917 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -289,27 +289,27 @@ 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.108.Final.jar - - io.netty-netty-codec-4.1.108.Final.jar - - io.netty-netty-codec-dns-4.1.108.Final.jar - - io.netty-netty-codec-http-4.1.108.Final.jar - - io.netty-netty-codec-http2-4.1.108.Final.jar - - io.netty-netty-codec-socks-4.1.108.Final.jar - - io.netty-netty-codec-haproxy-4.1.108.Final.jar - - io.netty-netty-common-4.1.108.Final.jar - - io.netty-netty-handler-4.1.108.Final.jar - - io.netty-netty-handler-proxy-4.1.108.Final.jar - - io.netty-netty-resolver-4.1.108.Final.jar - - io.netty-netty-resolver-dns-4.1.108.Final.jar - - io.netty-netty-resolver-dns-classes-macos-4.1.108.Final.jar - - io.netty-netty-resolver-dns-native-macos-4.1.108.Final-osx-aarch_64.jar - - io.netty-netty-resolver-dns-native-macos-4.1.108.Final-osx-x86_64.jar - - io.netty-netty-transport-4.1.108.Final.jar - - io.netty-netty-transport-classes-epoll-4.1.108.Final.jar - - io.netty-netty-transport-native-epoll-4.1.108.Final-linux-aarch_64.jar - - io.netty-netty-transport-native-epoll-4.1.108.Final-linux-x86_64.jar - - io.netty-netty-transport-native-unix-common-4.1.108.Final.jar - - io.netty-netty-transport-native-unix-common-4.1.108.Final-linux-x86_64.jar + - io.netty-netty-buffer-4.1.111.Final.jar + - io.netty-netty-codec-4.1.111.Final.jar + - io.netty-netty-codec-dns-4.1.111.Final.jar + - io.netty-netty-codec-http-4.1.111.Final.jar + - io.netty-netty-codec-http2-4.1.111.Final.jar + - io.netty-netty-codec-socks-4.1.111.Final.jar + - io.netty-netty-codec-haproxy-4.1.111.Final.jar + - io.netty-netty-common-4.1.111.Final.jar + - io.netty-netty-handler-4.1.111.Final.jar + - io.netty-netty-handler-proxy-4.1.111.Final.jar + - io.netty-netty-resolver-4.1.111.Final.jar + - io.netty-netty-resolver-dns-4.1.111.Final.jar + - io.netty-netty-resolver-dns-classes-macos-4.1.111.Final.jar + - io.netty-netty-resolver-dns-native-macos-4.1.111.Final-osx-aarch_64.jar + - io.netty-netty-resolver-dns-native-macos-4.1.111.Final-osx-x86_64.jar + - io.netty-netty-transport-4.1.111.Final.jar + - io.netty-netty-transport-classes-epoll-4.1.111.Final.jar + - io.netty-netty-transport-native-epoll-4.1.111.Final-linux-aarch_64.jar + - io.netty-netty-transport-native-epoll-4.1.111.Final-linux-x86_64.jar + - io.netty-netty-transport-native-unix-common-4.1.111.Final.jar + - io.netty-netty-transport-native-unix-common-4.1.111.Final-linux-x86_64.jar - io.netty-netty-tcnative-boringssl-static-2.0.65.Final.jar - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-linux-aarch_64.jar - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-linux-x86_64.jar @@ -424,7 +424,6 @@ The Apache Software License, Version 2.0 - io.grpc-grpc-auth-1.55.3.jar - io.grpc-grpc-context-1.55.3.jar - io.grpc-grpc-core-1.55.3.jar - - io.grpc-grpc-netty-1.55.3.jar - io.grpc-grpc-protobuf-1.55.3.jar - io.grpc-grpc-protobuf-lite-1.55.3.jar - io.grpc-grpc-stub-1.55.3.jar @@ -483,7 +482,6 @@ The Apache Software License, Version 2.0 - 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-grpc-4.5.8.jar * Apache ZooKeeper - org.apache.zookeeper-zookeeper-3.9.2.jar - org.apache.zookeeper-zookeeper-jute-3.9.2.jar @@ -495,11 +493,7 @@ The Apache Software License, Version 2.0 - com.google.http-client-google-http-client-1.41.0.jar - com.google.auto.value-auto-value-annotations-1.9.jar - com.google.re2j-re2j-1.6.jar - * Jetcd - - io.etcd-jetcd-api-0.7.7.jar - - io.etcd-jetcd-common-0.7.7.jar - - io.etcd-jetcd-core-0.7.7.jar - - io.etcd-jetcd-grpc-0.7.7.jar + * Jetcd - shaded * IPAddress - com.github.seancfoley-ipaddress-5.5.0.jar * RxJava @@ -608,6 +602,7 @@ Datastax - com.datastax.oss-pulsar-transaction-common-3.1.4.3-SNAPSHOT.jar - com.datastax.oss-pulsar-transaction-coordinator-3.1.4.3-SNAPSHOT.jar - com.datastax.oss-testmocks-3.1.4.3-SNAPSHOT.jar + - com.datastax.oss-jetcd-core-shaded-3.1.4.3-SNAPSHOT-shaded.jar ------------------------ diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 396bd8d0e83e3..f8062c1383463 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -344,23 +344,23 @@ The Apache Software License, Version 2.0 - commons-text-1.10.0.jar - commons-compress-1.26.0.jar * Netty - - netty-buffer-4.1.108.Final.jar - - netty-codec-4.1.108.Final.jar - - netty-codec-dns-4.1.108.Final.jar - - netty-codec-http-4.1.108.Final.jar - - netty-codec-socks-4.1.108.Final.jar - - netty-codec-haproxy-4.1.108.Final.jar - - netty-common-4.1.108.Final.jar - - netty-handler-4.1.108.Final.jar - - netty-handler-proxy-4.1.108.Final.jar - - netty-resolver-4.1.108.Final.jar - - netty-resolver-dns-4.1.108.Final.jar - - netty-transport-4.1.108.Final.jar - - netty-transport-classes-epoll-4.1.108.Final.jar - - netty-transport-native-epoll-4.1.108.Final-linux-aarch_64.jar - - netty-transport-native-epoll-4.1.108.Final-linux-x86_64.jar - - netty-transport-native-unix-common-4.1.108.Final.jar - - netty-transport-native-unix-common-4.1.108.Final-linux-x86_64.jar + - netty-buffer-4.1.111.Final.jar + - netty-codec-4.1.111.Final.jar + - netty-codec-dns-4.1.111.Final.jar + - netty-codec-http-4.1.111.Final.jar + - netty-codec-socks-4.1.111.Final.jar + - netty-codec-haproxy-4.1.111.Final.jar + - netty-common-4.1.111.Final.jar + - netty-handler-4.1.111.Final.jar + - netty-handler-proxy-4.1.111.Final.jar + - netty-resolver-4.1.111.Final.jar + - netty-resolver-dns-4.1.111.Final.jar + - netty-transport-4.1.111.Final.jar + - netty-transport-classes-epoll-4.1.111.Final.jar + - netty-transport-native-epoll-4.1.111.Final-linux-aarch_64.jar + - netty-transport-native-epoll-4.1.111.Final-linux-x86_64.jar + - netty-transport-native-unix-common-4.1.111.Final.jar + - netty-transport-native-unix-common-4.1.111.Final-linux-x86_64.jar - netty-tcnative-boringssl-static-2.0.65.Final.jar - netty-tcnative-boringssl-static-2.0.65.Final-linux-aarch_64.jar - netty-tcnative-boringssl-static-2.0.65.Final-linux-x86_64.jar @@ -371,9 +371,9 @@ The Apache Software License, Version 2.0 - netty-incubator-transport-classes-io_uring-0.0.21.Final.jar - netty-incubator-transport-native-io_uring-0.0.21.Final-linux-aarch_64.jar - netty-incubator-transport-native-io_uring-0.0.21.Final-linux-x86_64.jar - - netty-resolver-dns-classes-macos-4.1.108.Final.jar - - netty-resolver-dns-native-macos-4.1.108.Final-osx-aarch_64.jar - - netty-resolver-dns-native-macos-4.1.108.Final-osx-x86_64.jar + - netty-resolver-dns-classes-macos-4.1.111.Final.jar + - netty-resolver-dns-native-macos-4.1.111.Final-osx-aarch_64.jar + - netty-resolver-dns-native-macos-4.1.111.Final-osx-x86_64.jar * Prometheus client - simpleclient-0.16.0.jar - simpleclient_log4j2-0.16.0.jar diff --git a/jetcd-core-shaded/pom.xml b/jetcd-core-shaded/pom.xml new file mode 100644 index 0000000000000..32042b302b6c5 --- /dev/null +++ b/jetcd-core-shaded/pom.xml @@ -0,0 +1,187 @@ + + + + 4.0.0 + + com.datastax.oss + pulsar + 3.1.4.3-SNAPSHOT + + + jetcd-core-shaded + Apache Pulsar :: jetcd-core shaded + + + + io.etcd + jetcd-core + + + io.grpc + grpc-netty + + + io.netty + * + + + + + io.grpc + grpc-netty-shaded + + + + dev.failsafe + failsafe + + + io.grpc + grpc-protobuf + + + io.grpc + grpc-stub + + + io.grpc + grpc-grpclb + + + io.grpc + grpc-util + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + true + true + false + + + io.etcd:* + io.vertx:* + + + + + + io.vertx + org.apache.pulsar.jetcd.shaded.io.vertx + + + + io.grpc.netty + io.grpc.netty.shaded.io.grpc.netty + + + + io.netty + io.grpc.netty.shaded.io.netty + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + META-INF/maven/${project.groupId}/${project.artifactId}/pom.xml + + + + + + + + META-INF/maven/${project.groupId}/${project.artifactId}/pom.xml + ${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 525ec81208167..d56af113808d9 100644 --- a/pom.xml +++ b/pom.xml @@ -143,7 +143,7 @@ flexible messaging model and an intuitive client API. 1.1.10.5 4.1.12.1 5.1.0 - 4.1.108.Final + 4.1.111.Final 0.0.21.Final 9.4.54.v20240208 2.5.2 @@ -291,6 +291,7 @@ flexible messaging model and an intuitive client API. 2.3.0 3.4.1 3.1.0 + 3.6.0 1.1.0 1.5.0 3.1.2 @@ -568,6 +569,10 @@ flexible messaging model and an intuitive client API. jose4j org.bitbucket.b_c + + io.grpc + grpc-netty + @@ -1035,12 +1040,51 @@ flexible messaging model and an intuitive client API. io.etcd jetcd-core ${jetcd.version} + + + io.grpc + grpc-netty + + - io.etcd jetcd-test ${jetcd.version} + + + io.grpc + grpc-netty + + + io.etcd + jetcd-core + + + io.etcd + jetcd-api + + + io.vertx + * + + + + + ${project.groupId} + jetcd-core-shaded + ${project.version} + shaded + + + io.etcd + * + + + io.vertx + * + + @@ -1134,6 +1178,10 @@ flexible messaging model and an intuitive client API. com.squareup.okio okio + + io.grpc + grpc-netty + @@ -2080,6 +2128,11 @@ flexible messaging model and an intuitive client API. docker-maven-plugin ${docker-maven.version} + + org.codehaus.mojo + build-helper-maven-plugin + ${build-helper-maven-plugin.version} + @@ -2319,6 +2372,7 @@ flexible messaging model and an intuitive client API. pulsar-client-messagecrypto-bc pulsar-metadata + jetcd-core-shaded jclouds-shaded @@ -2379,7 +2433,7 @@ flexible messaging model and an intuitive client API. distribution pulsar-metadata - + jetcd-core-shaded pulsar-package-management diff --git a/pulsar-broker/pom.xml b/pulsar-broker/pom.xml index 10b49561bd6f9..5cb29706478bf 100644 --- a/pulsar-broker/pom.xml +++ b/pulsar-broker/pom.xml @@ -453,6 +453,18 @@ ${project.version} + + ${project.groupId} + jetcd-core-shaded + ${project.version} + shaded + test + + + io.grpc + grpc-netty-shaded + test + io.etcd jetcd-test diff --git a/pulsar-functions/instance/pom.xml b/pulsar-functions/instance/pom.xml index dbbffe7794e6a..f3583cc9e70e4 100644 --- a/pulsar-functions/instance/pom.xml +++ b/pulsar-functions/instance/pom.xml @@ -101,7 +101,7 @@ io.grpc - grpc-all + * com.google.protobuf @@ -110,6 +110,11 @@ + + io.grpc + grpc-netty-shaded + + io.grpc grpc-stub @@ -220,7 +225,7 @@ - + diff --git a/pulsar-metadata/pom.xml b/pulsar-metadata/pom.xml index 2fff417aeec09..1fa3b9474e813 100644 --- a/pulsar-metadata/pom.xml +++ b/pulsar-metadata/pom.xml @@ -107,10 +107,15 @@ - io.etcd - jetcd-core + ${project.groupId} + jetcd-core-shaded + ${project.version} + shaded + + + io.grpc + grpc-netty-shaded - io.etcd diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java index a7fb7192cb5fe..27862cd20b5e3 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/EtcdMetadataStore.java @@ -43,10 +43,10 @@ import io.etcd.jetcd.watch.WatchResponse; import io.grpc.Status; import io.grpc.StatusRuntimeException; -import io.grpc.netty.GrpcSslContexts; +import io.grpc.netty.shaded.io.grpc.netty.GrpcSslContexts; +import io.grpc.netty.shaded.io.netty.handler.ssl.SslContext; +import io.grpc.netty.shaded.io.netty.handler.ssl.SslProvider; import io.grpc.stub.StreamObserver; -import io.netty.handler.ssl.SslContext; -import io.netty.handler.ssl.SslProvider; import java.io.File; import java.io.IOException; import java.io.InputStream; diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index ccf112a98e24e..be9967031797c 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -231,21 +231,21 @@ The Apache Software License, Version 2.0 - commons-compress-1.26.0.jar - commons-lang3-3.11.jar * Netty - - netty-buffer-4.1.108.Final.jar - - netty-codec-4.1.108.Final.jar - - netty-codec-dns-4.1.108.Final.jar - - netty-codec-http-4.1.108.Final.jar - - netty-codec-haproxy-4.1.108.Final.jar - - netty-codec-socks-4.1.108.Final.jar - - netty-handler-proxy-4.1.108.Final.jar - - netty-common-4.1.108.Final.jar - - netty-handler-4.1.108.Final.jar + - netty-buffer-4.1.111.Final.jar + - netty-codec-4.1.111.Final.jar + - netty-codec-dns-4.1.111.Final.jar + - netty-codec-http-4.1.111.Final.jar + - netty-codec-haproxy-4.1.111.Final.jar + - netty-codec-socks-4.1.111.Final.jar + - netty-handler-proxy-4.1.111.Final.jar + - netty-common-4.1.111.Final.jar + - netty-handler-4.1.111.Final.jar - netty-reactive-streams-2.0.6.jar - - netty-resolver-4.1.108.Final.jar - - netty-resolver-dns-4.1.108.Final.jar - - netty-resolver-dns-classes-macos-4.1.108.Final.jar - - netty-resolver-dns-native-macos-4.1.108.Final-osx-aarch_64.jar - - netty-resolver-dns-native-macos-4.1.108.Final-osx-x86_64.jar + - netty-resolver-4.1.111.Final.jar + - netty-resolver-dns-4.1.111.Final.jar + - netty-resolver-dns-classes-macos-4.1.111.Final.jar + - netty-resolver-dns-native-macos-4.1.111.Final-osx-aarch_64.jar + - netty-resolver-dns-native-macos-4.1.111.Final-osx-x86_64.jar - netty-tcnative-boringssl-static-2.0.65.Final.jar - netty-tcnative-boringssl-static-2.0.65.Final-linux-aarch_64.jar - netty-tcnative-boringssl-static-2.0.65.Final-linux-x86_64.jar @@ -253,13 +253,12 @@ The Apache Software License, Version 2.0 - netty-tcnative-boringssl-static-2.0.65.Final-osx-x86_64.jar - netty-tcnative-boringssl-static-2.0.65.Final-windows-x86_64.jar - netty-tcnative-classes-2.0.65.Final.jar - - netty-transport-4.1.108.Final.jar - - netty-transport-classes-epoll-4.1.108.Final.jar - - netty-transport-native-epoll-4.1.108.Final-linux-aarch_64.jar - - netty-transport-native-epoll-4.1.108.Final-linux-x86_64.jar - - netty-transport-native-unix-common-4.1.108.Final.jar - - netty-transport-native-unix-common-4.1.108.Final-linux-x86_64.jar - - netty-codec-http2-4.1.108.Final.jar + - netty-transport-4.1.111.Final.jar + - netty-transport-classes-epoll-4.1.111.Final.jar + - netty-transport-native-epoll-4.1.111.Final-linux-aarch_64.jar + - netty-transport-native-epoll-4.1.111.Final-linux-x86_64.jar + - netty-transport-native-unix-common-4.1.111.Final.jar + - netty-transport-native-unix-common-4.1.111.Final-linux-x86_64.jar - netty-incubator-transport-classes-io_uring-0.0.21.Final.jar - netty-incubator-transport-native-io_uring-0.0.21.Final-linux-x86_64.jar - netty-incubator-transport-native-io_uring-0.0.21.Final-linux-aarch_64.jar @@ -268,19 +267,13 @@ The Apache Software License, Version 2.0 - grpc-context-1.55.3.jar - grpc-core-1.55.3.jar - grpc-grpclb-1.55.3.jar - - grpc-netty-1.55.3.jar + - grpc-netty-shaded-1.55.3.jar - grpc-protobuf-1.55.3.jar - grpc-protobuf-lite-1.55.3.jar - grpc-stub-1.55.3.jar - grpc-util-1.60.0.jar - * JEtcd - - jetcd-api-0.7.7.jar - - jetcd-common-0.7.7.jar - - jetcd-core-0.7.7.jar - - jetcd-grpc-0.7.7.jar - * Vertx - - vertx-core-4.5.8.jar - - vertx-grpc-4.5.8.jar + * JEtcd - shaded + * Vertx - shaded * Joda Time - joda-time-2.10.10.jar - failsafe-3.3.2.jar diff --git a/pulsar-sql/presto-distribution/src/assembly/assembly.xml b/pulsar-sql/presto-distribution/src/assembly/assembly.xml index 96c0421c71515..64c6778bc4396 100644 --- a/pulsar-sql/presto-distribution/src/assembly/assembly.xml +++ b/pulsar-sql/presto-distribution/src/assembly/assembly.xml @@ -60,6 +60,9 @@ io.airlift:launcher:tar.gz:bin:${airlift.version} io.airlift:launcher:tar.gz:properties:${airlift.version} *:tar.gz + + org.codehaus.mojo:animal-sniffer-annotations + com.google.android:annotations diff --git a/pulsar-sql/presto-pulsar-plugin/src/assembly/assembly.xml b/pulsar-sql/presto-pulsar-plugin/src/assembly/assembly.xml index 6650abfda3fc3..ac17aaed70bdf 100644 --- a/pulsar-sql/presto-pulsar-plugin/src/assembly/assembly.xml +++ b/pulsar-sql/presto-pulsar-plugin/src/assembly/assembly.xml @@ -33,6 +33,9 @@ runtime jakarta.ws.rs:jakarta.ws.rs-api + + org.codehaus.mojo:animal-sniffer-annotations + com.google.android:annotations diff --git a/src/check-binary-license.sh b/src/check-binary-license.sh index 3a6d266345f30..c9ca41fef3571 100755 --- a/src/check-binary-license.sh +++ b/src/check-binary-license.sh @@ -96,7 +96,7 @@ done if [ "$NO_PRESTO" -ne 1 ]; then # check pulsar sql jars - JARS=$(tar -tf $TARBALL | grep '\.jar' | grep 'trino/' | grep -v pulsar-client | grep -v bouncy-castle-bc | grep -v pulsar-metadata | grep -v 'managed-ledger' | grep -v 'pulsar-client-admin' | grep -v 'pulsar-client-api' | grep -v 'pulsar-functions-api' | grep -v 'pulsar-presto-connector-original' | grep -v 'pulsar-presto-distribution' | grep -v 'pulsar-common' | grep -v 'pulsar-functions-proto' | grep -v 'pulsar-functions-utils' | grep -v 'pulsar-io-core' | grep -v 'pulsar-transaction-common' | grep -v 'pulsar-package-core' | sed 's!.*/!!' | sort) + JARS=$(tar -tf $TARBALL | grep '\.jar' | grep 'trino/' | grep -v pulsar-client | grep -v bouncy-castle-bc | grep -v pulsar-metadata | grep -v 'managed-ledger' | grep -v 'pulsar-client-admin' | grep -v 'pulsar-client-api' | grep -v 'pulsar-functions-api' | grep -v 'pulsar-presto-connector-original' | grep -v 'pulsar-presto-distribution' | grep -v 'pulsar-common' | grep -v 'pulsar-functions-proto' | grep -v 'pulsar-functions-utils' | grep -v 'pulsar-io-core' | grep -v 'pulsar-transaction-common' | grep -v 'pulsar-package-core' | grep -v jetcd-core-shaded | sed 's!.*/!!' | sort) if [ -n "$JARS" ]; then LICENSEPATH=$(tar -tf $TARBALL | awk '/^[^\/]*\/trino\/LICENSE/') LICENSE=$(tar -O -xf $TARBALL "$LICENSEPATH") From fe09af2995415b5cc199a81ca32648f395d3d728 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 20 Jun 2024 23:39:58 +0300 Subject: [PATCH 594/699] [cleanup][misc] Remove classifier from netty-transport-native-unix-common dependency (#22951) (cherry picked from commit 6692bc8e327ea6958149ea0fb207691f4bce907d) (cherry picked from commit 2c1198845b1d924ae8b3657ad762da696151d5a8) --- distribution/server/src/assemble/LICENSE.bin.txt | 1 - distribution/shell/src/assemble/LICENSE.bin.txt | 1 - pulsar-common/pom.xml | 1 - pulsar-sql/presto-distribution/LICENSE | 1 - 4 files changed, 4 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index b663b839ec917..7b644016eeb8c 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -309,7 +309,6 @@ The Apache Software License, Version 2.0 - io.netty-netty-transport-native-epoll-4.1.111.Final-linux-aarch_64.jar - io.netty-netty-transport-native-epoll-4.1.111.Final-linux-x86_64.jar - io.netty-netty-transport-native-unix-common-4.1.111.Final.jar - - io.netty-netty-transport-native-unix-common-4.1.111.Final-linux-x86_64.jar - io.netty-netty-tcnative-boringssl-static-2.0.65.Final.jar - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-linux-aarch_64.jar - io.netty-netty-tcnative-boringssl-static-2.0.65.Final-linux-x86_64.jar diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index f8062c1383463..233fc3d22b857 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -360,7 +360,6 @@ The Apache Software License, Version 2.0 - netty-transport-native-epoll-4.1.111.Final-linux-aarch_64.jar - netty-transport-native-epoll-4.1.111.Final-linux-x86_64.jar - netty-transport-native-unix-common-4.1.111.Final.jar - - netty-transport-native-unix-common-4.1.111.Final-linux-x86_64.jar - netty-tcnative-boringssl-static-2.0.65.Final.jar - netty-tcnative-boringssl-static-2.0.65.Final-linux-aarch_64.jar - netty-tcnative-boringssl-static-2.0.65.Final-linux-x86_64.jar diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 04ace01ceb5e5..eb151a8217667 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -108,7 +108,6 @@ io.netty netty-transport-native-unix-common - linux-x86_64 diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index be9967031797c..0cbc598fba0b1 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -258,7 +258,6 @@ The Apache Software License, Version 2.0 - netty-transport-native-epoll-4.1.111.Final-linux-aarch_64.jar - netty-transport-native-epoll-4.1.111.Final-linux-x86_64.jar - netty-transport-native-unix-common-4.1.111.Final.jar - - netty-transport-native-unix-common-4.1.111.Final-linux-x86_64.jar - netty-incubator-transport-classes-io_uring-0.0.21.Final.jar - netty-incubator-transport-native-io_uring-0.0.21.Final-linux-x86_64.jar - netty-incubator-transport-native-io_uring-0.0.21.Final-linux-aarch_64.jar From b453330b46db37540e5bd1211a551af8cd33c58d Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 24 Jun 2024 21:42:30 +0300 Subject: [PATCH 595/699] [fix][ci] Fix jacoco code coverage report aggregation (#22964) (cherry picked from commit aa03c0efebc9e0f46c8653629ae92206b47591c6) (cherry picked from commit 53284175b4cfc7f64686eb4bac34139715556079) --- build/pulsar_ci_tool.sh | 9 +++++---- jetcd-core-shaded/pom.xml | 11 +++++++++++ pom.xml | 2 +- 3 files changed, 17 insertions(+), 5 deletions(-) diff --git a/build/pulsar_ci_tool.sh b/build/pulsar_ci_tool.sh index d946edd395789..40b1e5d18c39c 100755 --- a/build/pulsar_ci_tool.sh +++ b/build/pulsar_ci_tool.sh @@ -389,6 +389,7 @@ _ci_upload_coverage_files() { --transform="flags=r;s|\\(/jacoco.*\\).exec$|\\1_${testtype}_${testgroup}.exec|" \ --transform="flags=r;s|\\(/tmp/jacocoDir/.*\\).exec$|\\1_${testtype}_${testgroup}.exec|" \ --exclude="*/META-INF/bundled-dependencies/*" \ + --exclude="*/META-INF/versions/*" \ $GITHUB_WORKSPACE/target/classpath_* \ $(find "$GITHUB_WORKSPACE" -path "*/target/jacoco*.exec" -printf "%p\n%h/classes\n" | sort | uniq) \ $([ -d /tmp/jacocoDir ] && echo "/tmp/jacocoDir" ) \ @@ -530,11 +531,11 @@ ci_create_test_coverage_report() { local classfilesArgs="--classfiles $({ { for classpathEntry in $(cat $completeClasspathFile | { grep -v -f $filterArtifactsFile || true; } | sort | uniq | { grep -v -E "$excludeJarsPattern" || true; }); do - if [[ -f $classpathEntry && -n "$(unzip -Z1C $classpathEntry 'META-INF/bundled-dependencies/*' 2>/dev/null)" ]]; then - # file must be processed by removing META-INF/bundled-dependencies + if [[ -f $classpathEntry && -n "$(unzip -Z1C $classpathEntry 'META-INF/bundled-dependencies/*' 'META-INF/versions/*' 2>/dev/null)" ]]; then + # file must be processed by removing META-INF/bundled-dependencies and META-INF/versions local jartempfile=$(mktemp -t jarfile.XXXX --suffix=.jar) cp $classpathEntry $jartempfile - zip -q -d $jartempfile 'META-INF/bundled-dependencies/*' &> /dev/null + zip -q -d $jartempfile 'META-INF/bundled-dependencies/*' 'META-INF/versions/*' &> /dev/null echo $jartempfile else echo $classpathEntry @@ -596,7 +597,7 @@ ci_create_inttest_coverage_report() { # remove jar file that causes duplicate classes issue rm /tmp/jacocoDir/pulsar_lib/org.apache.pulsar-bouncy-castle* || true # remove any bundled dependencies as part of .jar/.nar files - find /tmp/jacocoDir/pulsar_lib '(' -name "*.jar" -or -name "*.nar" ')' -exec echo "Processing {}" \; -exec zip -q -d {} 'META-INF/bundled-dependencies/*' \; |grep -E -v "Nothing to do|^$" || true + find /tmp/jacocoDir/pulsar_lib '(' -name "*.jar" -or -name "*.nar" ')' -exec echo "Processing {}" \; -exec zip -q -d {} 'META-INF/bundled-dependencies/*' 'META-INF/versions/*' \; |grep -E -v "Nothing to do|^$" || true fi # projects that aren't considered as production code and their own src/main/java source code shouldn't be analysed local excludeProjectsPattern="testmocks|testclient|buildtools" diff --git a/jetcd-core-shaded/pom.xml b/jetcd-core-shaded/pom.xml index 32042b302b6c5..d92f55787928d 100644 --- a/jetcd-core-shaded/pom.xml +++ b/jetcd-core-shaded/pom.xml @@ -100,6 +100,12 @@ io.vertx org.apache.pulsar.jetcd.shaded.io.vertx + + + META-INF/versions/(\d+)/io/vertx/ + META-INF/versions/$1/org/apache/pulsar/jetcd/shaded/io/vertx/ + true + io.grpc.netty @@ -123,6 +129,11 @@ + + + true + + diff --git a/pom.xml b/pom.xml index d56af113808d9..da44f75458356 100644 --- a/pom.xml +++ b/pom.xml @@ -298,7 +298,7 @@ flexible messaging model and an intuitive client API. 4.9.10 3.5.3 1.7.0 - 0.8.8 + 0.8.12 4.7.3.0 4.7.3 2.5.1 From b5b4b95da3ca40976cc7fced6f1f40602181cd79 Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Tue, 2 Jul 2024 10:27:19 +0530 Subject: [PATCH 596/699] Updated license version for Datastax --- .../server/src/assemble/LICENSE.bin.txt | 46 +++++++++---------- 1 file changed, 23 insertions(+), 23 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 7b644016eeb8c..9460cac1c2bc9 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -579,29 +579,29 @@ Bouncy Castle License - org.bouncycastle-bcutil-jdk18on-1.78.1.jar Datastax - - com.datastax.oss-managed-ledger-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-auth-oidc-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-auth-sasl-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-common-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-config-validation-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-docs-tools-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-api-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-instance-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-local-runner-original-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-proto-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-runtime-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-secrets-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-utils-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-worker-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-io-core-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-metadata-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-proxy-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-testclient-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-transaction-common-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-pulsar-transaction-coordinator-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-testmocks-3.1.4.3-SNAPSHOT.jar - - com.datastax.oss-jetcd-core-shaded-3.1.4.3-SNAPSHOT-shaded.jar + - com.datastax.oss-managed-ledger-3.1.4.3.jar + - com.datastax.oss-pulsar-broker-3.1.4.3.jar + - com.datastax.oss-pulsar-broker-auth-oidc-3.1.4.3.jar + - com.datastax.oss-pulsar-broker-auth-sasl-3.1.4.3.jar + - com.datastax.oss-pulsar-broker-common-3.1.4.3.jar + - com.datastax.oss-pulsar-config-validation-3.1.4.3.jar + - com.datastax.oss-pulsar-docs-tools-3.1.4.3.jar + - com.datastax.oss-pulsar-functions-api-3.1.4.3.jar + - com.datastax.oss-pulsar-functions-instance-3.1.4.3.jar + - com.datastax.oss-pulsar-functions-local-runner-original-3.1.4.3.jar + - com.datastax.oss-pulsar-functions-proto-3.1.4.3.jar + - com.datastax.oss-pulsar-functions-runtime-3.1.4.3.jar + - com.datastax.oss-pulsar-functions-secrets-3.1.4.3.jar + - com.datastax.oss-pulsar-functions-utils-3.1.4.3.jar + - com.datastax.oss-pulsar-functions-worker-3.1.4.3.jar + - com.datastax.oss-pulsar-io-core-3.1.4.3.jar + - com.datastax.oss-pulsar-metadata-3.1.4.3.jar + - com.datastax.oss-pulsar-proxy-3.1.4.3.jar + - com.datastax.oss-pulsar-testclient-3.1.4.3.jar + - com.datastax.oss-pulsar-transaction-common-3.1.4.3.jar + - com.datastax.oss-pulsar-transaction-coordinator-3.1.4.3.jar + - com.datastax.oss-testmocks-3.1.4.3.jar + - com.datastax.oss-jetcd-core-shaded-3.1.4.3-shaded.jar ------------------------ From 84730188d10b22ccdbad36a888be4377d6a6d321 Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Tue, 2 Jul 2024 10:30:44 +0530 Subject: [PATCH 597/699] Release 3.1.4.3 --- 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-experimental/pom.xml | 2 +- docker/pulsar/pom.xml | 2 +- jclouds-shaded/pom.xml | 2 +- jetcd-core-shaded/pom.xml | 2 +- managed-ledger/pom.xml | 2 +- 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-client-1x-base/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-1x/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-2x-shaded/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/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/hdfs2/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-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-sql/pom.xml | 2 +- pulsar-sql/presto-distribution/pom.xml | 2 +- pulsar-sql/presto-pulsar-plugin/pom.xml | 2 +- pulsar-sql/presto-pulsar/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 +- 135 files changed, 137 insertions(+), 137 deletions(-) diff --git a/bouncy-castle/bc/pom.xml b/bouncy-castle/bc/pom.xml index 10cf3e8c1e940..6c5e7547deb59 100644 --- a/bouncy-castle/bc/pom.xml +++ b/bouncy-castle/bc/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.4.3-SNAPSHOT + 3.1.4.3 .. diff --git a/bouncy-castle/bcfips-include-test/pom.xml b/bouncy-castle/bcfips-include-test/pom.xml index 6921a394b8e2a..dbca662eafd50 100644 --- a/bouncy-castle/bcfips-include-test/pom.xml +++ b/bouncy-castle/bcfips-include-test/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.4.3-SNAPSHOT + 3.1.4.3 .. diff --git a/bouncy-castle/bcfips/pom.xml b/bouncy-castle/bcfips/pom.xml index bc946dbff7ed9..7c5497f5ee0bc 100644 --- a/bouncy-castle/bcfips/pom.xml +++ b/bouncy-castle/bcfips/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.4.3-SNAPSHOT + 3.1.4.3 .. diff --git a/bouncy-castle/pom.xml b/bouncy-castle/pom.xml index 9a75044e200f4..a920fe8063dce 100644 --- a/bouncy-castle/pom.xml +++ b/bouncy-castle/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.4.3-SNAPSHOT + 3.1.4.3 .. diff --git a/buildtools/pom.xml b/buildtools/pom.xml index ece9e78b2d977..98e395b00bbad 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -31,12 +31,12 @@ com.datastax.oss buildtools - 3.1.4.3-SNAPSHOT + 3.1.4.3 jar Pulsar Build Tools - 2024-06-10T05:43:07Z + 2024-07-02T05:00:41Z 1.8 1.8 3.1.0 diff --git a/distribution/io/pom.xml b/distribution/io/pom.xml index de782446bcb4d..3469bdb4ddd89 100644 --- a/distribution/io/pom.xml +++ b/distribution/io/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.4.3-SNAPSHOT + 3.1.4.3 .. diff --git a/distribution/offloaders/pom.xml b/distribution/offloaders/pom.xml index 279cf8abdc867..b18808f4f5226 100644 --- a/distribution/offloaders/pom.xml +++ b/distribution/offloaders/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.4.3-SNAPSHOT + 3.1.4.3 .. diff --git a/distribution/pom.xml b/distribution/pom.xml index 0329a21b39fc4..c35c757c7fced 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.4.3-SNAPSHOT + 3.1.4.3 .. diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index ede175f6f4ae8..b42b88d964c6b 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.4.3-SNAPSHOT + 3.1.4.3 .. diff --git a/distribution/shell/pom.xml b/distribution/shell/pom.xml index fade7a4c6cecc..93f28f67f936e 100644 --- a/distribution/shell/pom.xml +++ b/distribution/shell/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.4.3-SNAPSHOT + 3.1.4.3 .. diff --git a/docker/pom.xml b/docker/pom.xml index 9ae37091763a5..e0a9928c73a81 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.4.3-SNAPSHOT + 3.1.4.3 docker-images Apache Pulsar :: Docker Images diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index 5ab9ffb037ad8..ef7111af986ca 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.4.3-SNAPSHOT + 3.1.4.3 4.0.0 pulsar-all-docker-image diff --git a/docker/pulsar-experimental/pom.xml b/docker/pulsar-experimental/pom.xml index 61ff179b05082..a25c5e7d13653 100644 --- a/docker/pulsar-experimental/pom.xml +++ b/docker/pulsar-experimental/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.4.3-SNAPSHOT + 3.1.4.3 4.0.0 pulsar-experimental-docker-image diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 4f0ffb2fd99c2..4772433145d37 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.4.3-SNAPSHOT + 3.1.4.3 4.0.0 pulsar-docker-image diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml index 0fb57fbfe7cac..3d3680794eb7e 100644 --- a/jclouds-shaded/pom.xml +++ b/jclouds-shaded/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.4.3-SNAPSHOT + 3.1.4.3 .. diff --git a/jetcd-core-shaded/pom.xml b/jetcd-core-shaded/pom.xml index d92f55787928d..8d1b6de7f3247 100644 --- a/jetcd-core-shaded/pom.xml +++ b/jetcd-core-shaded/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.4.3-SNAPSHOT + 3.1.4.3 jetcd-core-shaded diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index f8a807b8c0562..9ba73c626e171 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.4.3-SNAPSHOT + 3.1.4.3 .. diff --git a/pom.xml b/pom.xml index da44f75458356..aa9d246051158 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ com.datastax.oss pulsar - 3.1.4.3-SNAPSHOT + 3.1.4.3 Pulsar Pulsar is a distributed pub-sub messaging platform with a very @@ -95,7 +95,7 @@ flexible messaging model and an intuitive client API. UTF-8 UTF-8 - 2024-06-10T05:43:07Z + 2024-07-02T05:00:41Z true + + + @@ -2564,7 +2574,6 @@ flexible messaging model and an intuitive client API. org.owasp dependency-check-maven - ${dependency-check-maven.version} ${pulsar.basedir}/src/owasp-dependency-check-false-positives.xml @@ -2599,7 +2608,6 @@ flexible messaging model and an intuitive client API. org.owasp dependency-check-maven - ${dependency-check-maven.version} diff --git a/pulsar-io/docs/pom.xml b/pulsar-io/docs/pom.xml index 262af86847184..26e75bd6eed3a 100644 --- a/pulsar-io/docs/pom.xml +++ b/pulsar-io/docs/pom.xml @@ -253,7 +253,6 @@ org.owasp dependency-check-maven - ${dependency-check-maven.version} diff --git a/pulsar-io/flume/pom.xml b/pulsar-io/flume/pom.xml index 256e96f917b5a..e5665ad7fa558 100644 --- a/pulsar-io/flume/pom.xml +++ b/pulsar-io/flume/pom.xml @@ -145,7 +145,6 @@ org.owasp dependency-check-maven - ${dependency-check-maven.version} diff --git a/pulsar-io/hbase/pom.xml b/pulsar-io/hbase/pom.xml index 50844b1c8c37d..5423cb378b8ea 100644 --- a/pulsar-io/hbase/pom.xml +++ b/pulsar-io/hbase/pom.xml @@ -108,7 +108,6 @@ org.owasp dependency-check-maven - ${dependency-check-maven.version} diff --git a/pulsar-io/hdfs2/pom.xml b/pulsar-io/hdfs2/pom.xml index 5c107c896d7dd..3d0b2671919e4 100644 --- a/pulsar-io/hdfs2/pom.xml +++ b/pulsar-io/hdfs2/pom.xml @@ -27,14 +27,14 @@ pulsar-io-hdfs2 Pulsar IO :: Hdfs2 - + ${project.groupId} pulsar-io-core ${project.version} - + com.fasterxml.jackson.core jackson-databind @@ -74,7 +74,7 @@ commons-lang3 - + @@ -113,7 +113,6 @@ org.owasp dependency-check-maven - ${dependency-check-maven.version} diff --git a/pulsar-io/hdfs3/pom.xml b/pulsar-io/hdfs3/pom.xml index ada1b9160e3c0..c5d0c49513fce 100644 --- a/pulsar-io/hdfs3/pom.xml +++ b/pulsar-io/hdfs3/pom.xml @@ -27,14 +27,14 @@ pulsar-io-hdfs3 Pulsar IO :: Hdfs3 - + ${project.groupId} pulsar-io-core ${project.version} - + com.fasterxml.jackson.core jackson-databind @@ -49,7 +49,7 @@ org.apache.commons commons-collections4 - + org.apache.hadoop hadoop-client @@ -80,7 +80,7 @@ - + @@ -119,7 +119,6 @@ org.owasp dependency-check-maven - ${dependency-check-maven.version} diff --git a/tiered-storage/file-system/pom.xml b/tiered-storage/file-system/pom.xml index db76005ea3555..e7f11f3ba8211 100644 --- a/tiered-storage/file-system/pom.xml +++ b/tiered-storage/file-system/pom.xml @@ -205,7 +205,6 @@ org.owasp dependency-check-maven - ${dependency-check-maven.version} From c34d44451c6f72b65ea2037b1fb7faf00d53a0b3 Mon Sep 17 00:00:00 2001 From: zhouyifan279 <88070094+zhouyifan279@users.noreply.github.com> Date: Mon, 8 Jul 2024 14:47:24 +0800 Subject: [PATCH 607/699] [fix][broker] PulsarStandalone started with error if --stream-storage-port is not 4181 (#22993) (cherry picked from commit 17e3f860b050443de74413c1d0b4a3d47173f68a) (cherry picked from commit 0753a2c9825ec594bec231b3fa7d967a26659d94) --- .../zookeeper/LocalBookkeeperEnsemble.java | 2 +- .../zookeeper/LocalBookkeeperEnsembleTest.java | 16 ++++++++++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java b/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java index 4c8d2dbbfa7d3..a1b0b75acce29 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsemble.java @@ -360,7 +360,7 @@ public void runStreamStorage(CompositeConfiguration conf) throws Exception { // create a default namespace try (StorageAdminClient admin = StorageClientBuilder.newBuilder() .withSettings(StorageClientSettings.newBuilder() - .serviceUri("bk://localhost:4181") + .serviceUri("bk://localhost:" + streamStoragePort) .backoffPolicy(Backoff.Jitter.of( Type.EXPONENTIAL, 1000, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsembleTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsembleTest.java index a4bc69a7266cc..bfbdf675bd81d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsembleTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/zookeeper/LocalBookkeeperEnsembleTest.java @@ -21,6 +21,8 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; + +import org.apache.bookkeeper.conf.ServerConfiguration; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -54,4 +56,18 @@ public void testStartStop() throws Exception { assertFalse(ensemble.getZkClient().getState().isConnected()); assertFalse(ensemble.getBookies()[0].isRunning()); } + + @Test(timeOut = 10_000) + public void testStartWithSpecifiedStreamStoragePort() throws Exception { + LocalBookkeeperEnsemble ensemble = null; + try { + ensemble = + new LocalBookkeeperEnsemble(1, 0, 0, 4182, null, null, true, null); + ensemble.startStandalone(new ServerConfiguration(), true); + } finally { + if (ensemble != null) { + ensemble.stop(); + } + } + } } From b36266cd598861582442378b9a56416c1fb4d003 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 8 Jul 2024 16:33:33 +0300 Subject: [PATCH 608/699] [improve][build] Upgrade dependency-check-maven-plugin to 10.0.2 (#23012) (cherry picked from commit 7924f9c3e3ddb5c65338816c90d5d14d5db00198) (cherry picked from commit 3db44b4095dfa7eda831792ad7990e47bd695ba2) --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a5122ff88a2d3..605fc4a9db7de 100644 --- a/pom.xml +++ b/pom.xml @@ -306,7 +306,7 @@ flexible messaging model and an intuitive client API. 0.1.4 1.3 0.4 - 10.0.1 + 10.0.2 0.9.44 1.6.1 6.4.0 From 538f68e2b782d3f61f3f221888fab8ff7f163a00 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Mon, 1 Jul 2024 21:41:43 +0800 Subject: [PATCH 609/699] [improve][broker] Improve exception for topic does not have schema to check (#22974) (cherry picked from commit 4c84788340b4a3df975bf4a919c7223b31835976) (cherry picked from commit bbb37b3faaff55aaeaa76e2258bf7e98b4416d39) --- .../nonpersistent/NonPersistentTopic.java | 13 +++++- .../service/persistent/PersistentTopic.java | 13 +++++- .../schema/SchemaRegistryServiceImpl.java | 3 +- .../exceptions/NotExistSchemaException.java | 43 +++++++++++++++++++ .../org/apache/pulsar/schema/SchemaTest.java | 16 ++++--- 5 files changed, 80 insertions(+), 8 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/exceptions/NotExistSchemaException.java 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 01aec2cbb8653..e163c0c0e5969 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 @@ -62,6 +62,8 @@ import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TopicPolicyListener; import org.apache.pulsar.broker.service.TransportCnx; +import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; +import org.apache.pulsar.broker.service.schema.exceptions.NotExistSchemaException; import org.apache.pulsar.broker.stats.ClusterReplicationMetrics; import org.apache.pulsar.broker.stats.NamespaceStats; import org.apache.pulsar.client.api.MessageId; @@ -1148,7 +1150,16 @@ public CompletableFuture addSchemaIfIdleOrCheckCompatible(SchemaData schem || (!producers.isEmpty()) || (numActiveConsumersWithoutAutoSchema != 0) || ENTRIES_ADDED_COUNTER_UPDATER.get(this) != 0) { - return checkSchemaCompatibleForConsumer(schema); + return checkSchemaCompatibleForConsumer(schema) + .exceptionally(ex -> { + Throwable realCause = FutureUtil.unwrapCompletionException(ex); + if (realCause instanceof NotExistSchemaException) { + throw FutureUtil.wrapToCompletionException( + new IncompatibleSchemaException("Failed to add schema to an active topic" + + " with empty(BYTES) schema: new schema type " + schema.getType())); + } + throw FutureUtil.wrapToCompletionException(realCause); + }); } else { return addSchema(schema).thenCompose(schemaVersion -> CompletableFuture.completedFuture(null)); } 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 e78211d5a1d2e..d4167afbda18e 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 @@ -124,6 +124,8 @@ import org.apache.pulsar.broker.service.TransportCnx; import org.apache.pulsar.broker.service.persistent.DispatchRateLimiter.Type; import org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage; +import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; +import org.apache.pulsar.broker.service.schema.exceptions.NotExistSchemaException; import org.apache.pulsar.broker.stats.ClusterReplicationMetrics; import org.apache.pulsar.broker.stats.NamespaceStats; import org.apache.pulsar.broker.stats.ReplicationMetrics; @@ -3721,7 +3723,16 @@ public CompletableFuture addSchemaIfIdleOrCheckCompatible(SchemaData schem || (userCreatedProducerCount > 0) || (numActiveConsumersWithoutAutoSchema != 0) || (ledger.getTotalSize() != 0)) { - return checkSchemaCompatibleForConsumer(schema); + return checkSchemaCompatibleForConsumer(schema) + .exceptionally(ex -> { + Throwable realCause = FutureUtil.unwrapCompletionException(ex); + if (realCause instanceof NotExistSchemaException) { + throw FutureUtil.wrapToCompletionException( + new IncompatibleSchemaException("Failed to add schema to an active topic" + + " with empty(BYTES) schema: new schema type " + schema.getType())); + } + throw FutureUtil.wrapToCompletionException(realCause); + }); } else { return addSchema(schema).thenCompose(schemaVersion -> CompletableFuture.completedFuture(null)); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java index 903f57cb7803a..b3b87b8923545 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/SchemaRegistryServiceImpl.java @@ -48,6 +48,7 @@ import org.apache.commons.lang3.mutable.MutableLong; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.service.schema.exceptions.IncompatibleSchemaException; +import org.apache.pulsar.broker.service.schema.exceptions.NotExistSchemaException; import org.apache.pulsar.broker.service.schema.exceptions.SchemaException; import org.apache.pulsar.broker.service.schema.proto.SchemaRegistryFormat; import org.apache.pulsar.common.policies.data.SchemaCompatibilityStrategy; @@ -393,7 +394,7 @@ public CompletableFuture checkConsumerCompatibility(String schemaId, Schem return checkCompatibilityWithAll(schemaId, schemaData, strategy); } } else { - return FutureUtil.failedFuture(new IncompatibleSchemaException("Topic does not have schema to check")); + return FutureUtil.failedFuture(new NotExistSchemaException("Topic does not have schema to check")); } }); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/exceptions/NotExistSchemaException.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/exceptions/NotExistSchemaException.java new file mode 100644 index 0000000000000..2fe0a09237545 --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/schema/exceptions/NotExistSchemaException.java @@ -0,0 +1,43 @@ +/* + * 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.schema.exceptions; + +/** + * Exception is thrown when an schema not exist. + */ +public class NotExistSchemaException extends SchemaException { + + private static final long serialVersionUID = -8342983749283749283L; + + public NotExistSchemaException() { + super("The schema does not exist"); + } + + public NotExistSchemaException(String message) { + super(message); + } + + public NotExistSchemaException(String message, Throwable e) { + super(message, e); + } + + public NotExistSchemaException(Throwable e) { + super(e); + } +} 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 e10b45868bf47..fb70e8f4ff552 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 @@ -90,6 +90,7 @@ import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Slf4j @@ -119,6 +120,11 @@ public void cleanup() throws Exception { super.internalCleanup(); } + @DataProvider(name = "topicDomain") + public static Object[] topicDomain() { + return new Object[] { "persistent://", "non-persistent://" }; + } + @Test public void testGetSchemaWhenCreateAutoProduceBytesProducer() throws Exception{ final String tenant = PUBLIC_TENANT; @@ -1330,19 +1336,19 @@ private void testIncompatibleSchema() throws Exception { * the new consumer to register new schema. But before we can solve this problem, we need to modify * "CmdProducer" to let the Broker know that the Producer uses a schema of type "AUTO_PRODUCE_BYTES". */ - @Test - public void testAutoProduceAndSpecifiedConsumer() throws Exception { + @Test(dataProvider = "topicDomain") + public void testAutoProduceAndSpecifiedConsumer(String domain) throws Exception { final String namespace = PUBLIC_TENANT + "/ns_" + randomName(16); admin.namespaces().createNamespace(namespace, Sets.newHashSet(CLUSTER_NAME)); - final String topicName = "persistent://" + namespace + "/tp_" + randomName(16); + final String topicName = domain + namespace + "/tp_" + randomName(16); admin.topics().createNonPartitionedTopic(topicName); Producer producer = pulsarClient.newProducer(Schema.AUTO_PRODUCE_BYTES()).topic(topicName).create(); try { pulsarClient.newConsumer(Schema.STRING).topic(topicName).subscriptionName("sub1").subscribe(); - fail("Should throw ex: Topic does not have schema to check"); + fail("Should throw ex: Failed to add schema to an active topic with empty(BYTES) schema"); } catch (Exception ex){ - assertTrue(ex.getMessage().contains("Topic does not have schema to check")); + assertTrue(ex.getMessage().contains("Failed to add schema to an active topic with empty(BYTES) schema")); } // Cleanup. From 2a0af99359a61ce839cee76b7c6849bc86827d32 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Tue, 2 Jul 2024 08:46:56 +0800 Subject: [PATCH 610/699] [fix][broker] Fix broker OOM when upload a large package. (#22989) (cherry picked from commit da2a1910a32e622ea609ff7b9e91711ecaf36de6) (cherry picked from commit 311b2a771969b17d012ee6da7dfb565459794125) --- .../storage/bookkeeper/DLOutputStream.java | 53 +++++++++---------- .../bookkeeper/DLOutputStreamTest.java | 14 ++--- 2 files changed, 31 insertions(+), 36 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 222987aa49d43..67345ebd47e31 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 @@ -22,8 +22,6 @@ import io.netty.buffer.Unpooled; import java.io.IOException; import java.io.InputStream; -import java.util.ArrayList; -import java.util.List; import java.util.concurrent.CompletableFuture; import lombok.extern.slf4j.Slf4j; import org.apache.distributedlog.LogRecord; @@ -38,6 +36,7 @@ class DLOutputStream { private final DistributedLogManager distributedLogManager; private final AsyncLogWriter writer; + private final byte[] readBuffer = new byte[8192]; private long offset = 0L; private DLOutputStream(DistributedLogManager distributedLogManager, AsyncLogWriter writer) { @@ -50,42 +49,38 @@ static CompletableFuture openWriterAsync(DistributedLogManager d return distributedLogManager.openAsyncLogWriter().thenApply(w -> new DLOutputStream(distributedLogManager, w)); } - private CompletableFuture> getRecords(InputStream inputStream) { - CompletableFuture> future = new CompletableFuture<>(); - CompletableFuture.runAsync(() -> { - byte[] readBuffer = new byte[8192]; - List records = new ArrayList<>(); - try { - int read = 0; - while ((read = inputStream.read(readBuffer)) != -1) { - log.info("write something into the ledgers offset: {}, length: {}", offset, read); - ByteBuf writeBuf = Unpooled.copiedBuffer(readBuffer, 0, read); - offset += writeBuf.readableBytes(); - LogRecord record = new LogRecord(offset, writeBuf); - records.add(record); - } - future.complete(records); - } catch (IOException e) { - log.error("Failed to get all records from the input stream", e); - future.completeExceptionally(e); + 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); + final ByteBuf writeBuf = Unpooled.wrappedBuffer(readBuffer, 0, read); + offset += writeBuf.readableBytes(); + final LogRecord record = new LogRecord(offset, writeBuf); + writer.write(record).thenAccept(v -> writeAsyncHelper(is, result)) + .exceptionally(e -> { + result.completeExceptionally(e); + return null; + }); + } else { + result.complete(this); } - }); - return future; + } catch (IOException e) { + log.error("Failed to get all records from the input stream", e); + result.completeExceptionally(e); + } } /** * Write all input stream data to the distribute log. * * @param inputStream the data we need to write - * @return + * @return CompletableFuture */ CompletableFuture writeAsync(InputStream inputStream) { - return getRecords(inputStream) - .thenCompose(this::writeAsync); - } - - private CompletableFuture writeAsync(List records) { - return writer.writeBulk(records).thenApply(ignore -> this); + CompletableFuture result = new CompletableFuture<>(); + writeAsyncHelper(inputStream, result); + return result; } /** 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 63fcf5e46ebe1..b55e0e0d34a4f 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 @@ -21,17 +21,18 @@ import java.io.ByteArrayInputStream; import java.io.IOException; -import java.util.Collections; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import org.apache.distributedlog.DLSN; +import org.apache.distributedlog.LogRecord; import org.apache.distributedlog.api.AsyncLogWriter; import org.apache.distributedlog.api.DistributedLogManager; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import static org.mockito.Mockito.anyList; @@ -53,9 +54,8 @@ public void setup() { when(dlm.asyncClose()).thenReturn(CompletableFuture.completedFuture(null)); when(writer.markEndOfStream()).thenReturn(CompletableFuture.completedFuture(null)); when(writer.asyncClose()).thenReturn(CompletableFuture.completedFuture(null)); - when(writer.writeBulk(anyList())) - .thenReturn(CompletableFuture.completedFuture( - Collections.singletonList(CompletableFuture.completedFuture(DLSN.InitialDLSN)))); + when(writer.write(any(LogRecord.class))) + .thenReturn(CompletableFuture.completedFuture(DLSN.InitialDLSN)); } @AfterMethod(alwaysRun = true) @@ -75,7 +75,7 @@ public void writeInputStreamData() throws ExecutionException, InterruptedExcepti .thenCompose(w -> w.writeAsync(new ByteArrayInputStream(data)) .thenCompose(DLOutputStream::closeAsync)).get(); - verify(writer, times(1)).writeBulk(anyList()); + verify(writer, times(1)).write(any(LogRecord.class)); verify(writer, times(1)).markEndOfStream(); verify(writer, times(1)).asyncClose(); verify(dlm, times(1)).asyncClose(); @@ -91,7 +91,7 @@ public void writeBytesArrayData() throws ExecutionException, InterruptedExceptio .thenCompose(w -> w.writeAsync(new ByteArrayInputStream(data)) .thenCompose(DLOutputStream::closeAsync)).get(); - verify(writer, times(1)).writeBulk(anyList()); + verify(writer, times(1)).write(any(LogRecord.class)); verify(writer, times(1)).markEndOfStream(); verify(writer, times(1)).asyncClose(); verify(dlm, times(1)).asyncClose(); @@ -104,7 +104,7 @@ public void writeLongBytesArrayData() throws ExecutionException, InterruptedExce .thenCompose(w -> w.writeAsync(new ByteArrayInputStream(data)) .thenCompose(DLOutputStream::closeAsync)).get(); - verify(writer, times(1)).writeBulk(anyList()); + verify(writer, times(4)).write(any(LogRecord.class)); verify(writer, times(1)).markEndOfStream(); verify(writer, times(1)).asyncClose(); verify(dlm, times(1)).asyncClose(); From 65c8b236a3dd2c7518bb05c4bdba88c62a960bc8 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Thu, 4 Jul 2024 07:02:26 +0800 Subject: [PATCH 611/699] [fix][broker] Can't connecte to non-persist topic when enable broker client tls (#22991) (cherry picked from commit deb26f7662268def7f838f722de4a677b3d546ed) (cherry picked from commit 998bd90fba5bbc46286e486f601f34875ee8e528) --- .../broker/namespace/NamespaceService.java | 10 ++++- .../TokenExpirationProduceConsumerTest.java | 45 +++++++++++++++---- 2 files changed, 45 insertions(+), 10 deletions(-) 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 c60f75b1c827b..f977b6e561969 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 @@ -1414,7 +1414,15 @@ public CompletableFuture checkNonPersistentNonPartitionedTopicExists(St return FutureUtil.failedFuture(new ServiceUnitNotReadyException( "No broker was available to own " + topicName)); } - return pulsarClient.getLookup(lookupResult.get().getLookupData().getBrokerUrl()) + LookupData lookupData = lookupResult.get().getLookupData(); + String brokerUrl; + if (pulsar.getConfiguration().isBrokerClientTlsEnabled() + && StringUtils.isNotEmpty(lookupData.getBrokerUrlTls())) { + brokerUrl = lookupData.getBrokerUrlTls(); + } else { + brokerUrl = lookupData.getBrokerUrl(); + } + return pulsarClient.getLookup(brokerUrl) .getPartitionedTopicMetadata(topicName, false) .thenApply(metadata -> true) .exceptionallyCompose(ex -> { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenExpirationProduceConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenExpirationProduceConsumerTest.java index 4fc0d315d2253..eb3056307d325 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenExpirationProduceConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/TokenExpirationProduceConsumerTest.java @@ -18,11 +18,23 @@ */ package org.apache.pulsar.client.api; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; import com.google.common.collect.Sets; import io.jsonwebtoken.Jwts; import io.jsonwebtoken.SignatureAlgorithm; +import java.time.Duration; +import java.util.Base64; +import java.util.Calendar; +import java.util.Date; +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import javax.crypto.SecretKey; +import java.nio.charset.StandardCharsets; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.authentication.AuthenticationProviderToken; @@ -40,15 +52,6 @@ import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import javax.crypto.SecretKey; -import java.time.Duration; -import java.util.Base64; -import java.util.Calendar; -import java.util.Date; -import java.util.HashSet; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.TimeUnit; @Test(groups = "broker-api") @Slf4j @@ -114,6 +117,7 @@ protected void internalSetUpForBroker() { conf.setAuthenticationProviders(Sets.newHashSet(AuthenticationProviderToken.class.getName())); conf.setBrokerClientAuthenticationPlugin(AuthenticationToken.class.getName()); conf.setBrokerClientAuthenticationParameters("token:" + ADMIN_TOKEN); + conf.setBrokerClientTlsEnabled(true); conf.getProperties().setProperty("tokenSecretKey", "data:;base64," + Base64.getEncoder().encodeToString(SECRET_KEY.getEncoded())); } @@ -139,6 +143,29 @@ private PulsarAdmin getAdmin(String token) throws Exception { return clientBuilder.build(); } + @Test + public void testNonPersistentTopic() throws Exception { + + @Cleanup + PulsarClient pulsarClient = getClient(ADMIN_TOKEN); + + String topic = "non-persistent://" + namespaceName + "/test-token-non-persistent"; + + @Cleanup + Consumer consumer = pulsarClient.newConsumer().topic(topic) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscriptionName("test").subscribe(); + + @Cleanup + Producer producer = pulsarClient.newProducer().topic(topic).create(); + byte[] msg = "Hello".getBytes(StandardCharsets.UTF_8); + producer.send(msg); + + Message receive = consumer.receive(3, TimeUnit.SECONDS); + assertNotNull(receive); + assertEquals(receive.getData(), msg); + } + @Test public void testTokenExpirationProduceConsumer() throws Exception { Calendar calendar = Calendar.getInstance(); From 1db5939d64b4c00feb2bd39f040da44c4c66c95e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Wed, 3 Jul 2024 21:09:31 +0800 Subject: [PATCH 612/699] [fix] Make operations on `individualDeletedMessages` in lock scope (#22966) (cherry picked from commit dbbb6b66c99afd12762dec198482dbf766bff3bb) (cherry picked from commit e01e90f0bf74da41b3dde33c1148b765af6e8a1f) --- .../mledger/ManagedLedgerConfig.java | 4 +- .../mledger/impl/ManagedCursorImpl.java | 84 +++++++++++++------ .../mledger/impl/RangeSetWrapper.java | 12 +-- ...angeSet.java => OpenLongPairRangeSet.java} | 10 ++- .../util/collections/DefaultRangeSetTest.java | 4 +- ...est.java => OpenLongPairRangeSetTest.java} | 40 ++++----- 6 files changed, 94 insertions(+), 60 deletions(-) rename pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/{ConcurrentOpenLongPairRangeSet.java => OpenLongPairRangeSet.java} (97%) rename pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/{ConcurrentOpenLongPairRangeSetTest.java => OpenLongPairRangeSetTest.java} (92%) 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 f16866ac7923a..fe8cb238dbce4 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 @@ -33,7 +33,7 @@ import org.apache.bookkeeper.mledger.impl.NullLedgerOffloader; import org.apache.bookkeeper.mledger.intercept.ManagedLedgerInterceptor; import org.apache.commons.collections4.MapUtils; -import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; +import org.apache.pulsar.common.util.collections.OpenLongPairRangeSet; /** * Configuration class for a ManagedLedger. @@ -282,7 +282,7 @@ public ManagedLedgerConfig setPassword(String password) { } /** - * should use {@link ConcurrentOpenLongPairRangeSet} to store unacked ranges. + * should use {@link OpenLongPairRangeSet} to store unacked ranges. * @return */ public boolean isUnackedRangesOpenCacheSetEnabled() { 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 4391fa0cb5fd9..025d40e1e89f3 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 @@ -1099,7 +1099,12 @@ public long getNumberOfEntriesSinceFirstNotAckedMessage() { @Override public int getTotalNonContiguousDeletedMessagesRange() { - return individualDeletedMessages.size(); + lock.readLock().lock(); + try { + return individualDeletedMessages.size(); + } finally { + lock.readLock().unlock(); + } } @Override @@ -2377,8 +2382,9 @@ public void asyncDelete(Iterable positions, AsyncCallbacks.DeleteCallb callback.deleteFailed(getManagedLedgerException(e), ctx); return; } finally { + boolean empty = individualDeletedMessages.isEmpty(); lock.writeLock().unlock(); - if (individualDeletedMessages.isEmpty()) { + if (empty) { callback.deleteComplete(ctx); } } @@ -2656,10 +2662,15 @@ public void operationFailed(MetaStoreException e) { } private boolean shouldPersistUnackRangesToLedger() { - return cursorLedger != null - && !isCursorLedgerReadOnly - && getConfig().getMaxUnackedRangesToPersist() > 0 - && individualDeletedMessages.size() > getConfig().getMaxUnackedRangesToPersistInMetadataStore(); + lock.readLock().lock(); + try { + return cursorLedger != null + && !isCursorLedgerReadOnly + && getConfig().getMaxUnackedRangesToPersist() > 0 + && individualDeletedMessages.size() > getConfig().getMaxUnackedRangesToPersistInMetadataStore(); + } finally { + lock.readLock().unlock(); + } } private void persistPositionMetaStore(long cursorsLedgerId, PositionImpl position, Map properties, @@ -3019,7 +3030,7 @@ private static List buildStringPropertiesMap(Map } private List buildIndividualDeletedMessageRanges() { - lock.readLock().lock(); + lock.writeLock().lock(); try { if (individualDeletedMessages.isEmpty()) { this.individualDeletedMessagesSerializedSize = 0; @@ -3061,7 +3072,7 @@ private List buildIndividualDeletedMessageRanges() { individualDeletedMessages.resetDirtyKeys(); return rangeList; } finally { - lock.readLock().unlock(); + lock.writeLock().unlock(); } } @@ -3447,8 +3458,13 @@ public LongPairRangeSet getIndividuallyDeletedMessagesSet() { public boolean isMessageDeleted(Position position) { checkArgument(position instanceof PositionImpl); - return ((PositionImpl) position).compareTo(markDeletePosition) <= 0 - || individualDeletedMessages.contains(position.getLedgerId(), position.getEntryId()); + lock.readLock().lock(); + try { + return ((PositionImpl) position).compareTo(markDeletePosition) <= 0 + || individualDeletedMessages.contains(position.getLedgerId(), position.getEntryId()); + } finally { + lock.readLock().unlock(); + } } //this method will return a copy of the position's ack set @@ -3477,13 +3493,19 @@ public long[] getBatchPositionAckSet(Position position) { * @return next available position */ public PositionImpl getNextAvailablePosition(PositionImpl position) { - Range range = individualDeletedMessages.rangeContaining(position.getLedgerId(), - position.getEntryId()); - if (range != null) { - PositionImpl nextPosition = range.upperEndpoint().getNext(); - return (nextPosition != null && nextPosition.compareTo(position) > 0) ? nextPosition : position.getNext(); + lock.readLock().lock(); + try { + Range range = individualDeletedMessages.rangeContaining(position.getLedgerId(), + position.getEntryId()); + if (range != null) { + PositionImpl nextPosition = range.upperEndpoint().getNext(); + return (nextPosition != null && nextPosition.compareTo(position) > 0) + ? nextPosition : position.getNext(); + } + return position.getNext(); + } finally { + lock.readLock().unlock(); } - return position.getNext(); } public Position getNextLedgerPosition(long currentLedgerId) { @@ -3534,7 +3556,12 @@ public ManagedLedger getManagedLedger() { @Override public Range getLastIndividualDeletedRange() { - return individualDeletedMessages.lastRange(); + lock.readLock().lock(); + try { + return individualDeletedMessages.lastRange(); + } finally { + lock.readLock().unlock(); + } } @Override @@ -3664,15 +3691,20 @@ public ManagedLedgerConfig getConfig() { public ManagedCursor duplicateNonDurableCursor(String nonDurableCursorName) throws ManagedLedgerException { NonDurableCursorImpl newNonDurableCursor = (NonDurableCursorImpl) ledger.newNonDurableCursor(getMarkDeletedPosition(), nonDurableCursorName); - if (individualDeletedMessages != null) { - this.individualDeletedMessages.forEach(range -> { - newNonDurableCursor.individualDeletedMessages.addOpenClosed( - range.lowerEndpoint().getLedgerId(), - range.lowerEndpoint().getEntryId(), - range.upperEndpoint().getLedgerId(), - range.upperEndpoint().getEntryId()); - return true; - }); + lock.readLock().lock(); + try { + if (individualDeletedMessages != null) { + this.individualDeletedMessages.forEach(range -> { + newNonDurableCursor.individualDeletedMessages.addOpenClosed( + range.lowerEndpoint().getLedgerId(), + range.lowerEndpoint().getEntryId(), + range.upperEndpoint().getLedgerId(), + range.upperEndpoint().getEntryId()); + return true; + }); + } + } finally { + lock.readLock().unlock(); } if (batchDeletedIndexes != null) { for (Map.Entry entry : this.batchDeletedIndexes.entrySet()) { 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 f235ffc63ace5..299fd3dc74cb4 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 @@ -25,8 +25,8 @@ import java.util.Collection; import java.util.List; import org.apache.bookkeeper.mledger.ManagedLedgerConfig; -import org.apache.pulsar.common.util.collections.ConcurrentOpenLongPairRangeSet; import org.apache.pulsar.common.util.collections.LongPairRangeSet; +import org.apache.pulsar.common.util.collections.OpenLongPairRangeSet; /** * Wraps other Range classes, and adds LRU, marking dirty data and other features on this basis. @@ -55,7 +55,7 @@ public RangeSetWrapper(LongPairConsumer rangeConverter, this.config = managedCursor.getManagedLedger().getConfig(); this.rangeConverter = rangeConverter; this.rangeSet = config.isUnackedRangesOpenCacheSetEnabled() - ? new ConcurrentOpenLongPairRangeSet<>(4096, rangeConverter) + ? new OpenLongPairRangeSet<>(4096, rangeConverter) : new LongPairRangeSet.DefaultRangeSet<>(rangeConverter, rangeBoundConsumer); this.enableMultiEntry = config.isPersistentUnackedRangesWithMultipleEntriesEnabled(); } @@ -148,16 +148,16 @@ public int cardinality(long lowerKey, long lowerValue, long upperKey, long upper @VisibleForTesting void add(Range range) { - if (!(rangeSet instanceof ConcurrentOpenLongPairRangeSet)) { + if (!(rangeSet instanceof OpenLongPairRangeSet)) { throw new UnsupportedOperationException("Only ConcurrentOpenLongPairRangeSet support this method"); } - ((ConcurrentOpenLongPairRangeSet) rangeSet).add(range); + ((OpenLongPairRangeSet) rangeSet).add(range); } @VisibleForTesting void remove(Range range) { - if (rangeSet instanceof ConcurrentOpenLongPairRangeSet) { - ((ConcurrentOpenLongPairRangeSet) rangeSet).remove((Range) range); + if (rangeSet instanceof OpenLongPairRangeSet) { + ((OpenLongPairRangeSet) rangeSet).remove((Range) range); } else { ((DefaultRangeSet) rangeSet).remove(range); } 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/OpenLongPairRangeSet.java similarity index 97% rename from pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSet.java rename to pulsar-common/src/main/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSet.java index 72215d7296cc3..c053c106be206 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/OpenLongPairRangeSet.java @@ -28,6 +28,7 @@ import java.util.NavigableMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicBoolean; +import javax.annotation.concurrent.NotThreadSafe; import org.apache.commons.lang.mutable.MutableInt; /** @@ -41,7 +42,8 @@ * So, this rangeSet is not suitable for large number of unique keys. * */ -public class ConcurrentOpenLongPairRangeSet> implements LongPairRangeSet { +@NotThreadSafe +public class OpenLongPairRangeSet> implements LongPairRangeSet { protected final NavigableMap rangeBitSetMap = new ConcurrentSkipListMap<>(); private boolean threadSafe = true; @@ -54,15 +56,15 @@ public class ConcurrentOpenLongPairRangeSet> implements private volatile boolean updatedAfterCachedForSize = true; private volatile boolean updatedAfterCachedForToString = true; - public ConcurrentOpenLongPairRangeSet(LongPairConsumer consumer) { + public OpenLongPairRangeSet(LongPairConsumer consumer) { this(1024, true, consumer); } - public ConcurrentOpenLongPairRangeSet(int size, LongPairConsumer consumer) { + public OpenLongPairRangeSet(int size, LongPairConsumer consumer) { this(size, true, consumer); } - public ConcurrentOpenLongPairRangeSet(int size, boolean threadSafe, LongPairConsumer consumer) { + public OpenLongPairRangeSet(int size, boolean threadSafe, LongPairConsumer consumer) { this.threadSafe = threadSafe; this.bitSetSize = size; this.consumer = consumer; diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/DefaultRangeSetTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/DefaultRangeSetTest.java index f6103061a420c..730f4b4ceca22 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/DefaultRangeSetTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/DefaultRangeSetTest.java @@ -34,8 +34,8 @@ public class DefaultRangeSetTest { public void testBehavior() { LongPairRangeSet.DefaultRangeSet set = new LongPairRangeSet.DefaultRangeSet<>(consumer, reverseConsumer); - ConcurrentOpenLongPairRangeSet rangeSet = - new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet rangeSet = + new OpenLongPairRangeSet<>(consumer); assertNull(set.firstRange()); assertNull(set.lastRange()); diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSetTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSetTest.java similarity index 92% rename from pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSetTest.java rename to pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSetTest.java index 40bb337935742..4dd0f5551f1f9 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/ConcurrentOpenLongPairRangeSetTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/util/collections/OpenLongPairRangeSetTest.java @@ -37,14 +37,14 @@ import com.google.common.collect.Range; import com.google.common.collect.TreeRangeSet; -public class ConcurrentOpenLongPairRangeSetTest { +public class OpenLongPairRangeSetTest { static final LongPairConsumer consumer = LongPair::new; static final RangeBoundConsumer reverseConsumer = pair -> pair; @Test public void testIsEmpty() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); assertTrue(set.isEmpty()); // lowerValueOpen and upperValue are both -1 so that an empty set will be added set.addOpenClosed(0, -1, 0, -1); @@ -55,7 +55,7 @@ public void testIsEmpty() { @Test public void testAddForSameKey() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); // add 0 to 5 set.add(Range.closed(new LongPair(0, 0), new LongPair(0, 5))); // add 8,9,10 @@ -76,7 +76,7 @@ public void testAddForSameKey() { @Test public void testAddForDifferentKey() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); // [98,100],[(1,5),(1,5)],[(1,10,1,15)],[(1,20),(1,20)],[(2,0),(2,10)] set.addOpenClosed(0, 98, 0, 99); set.addOpenClosed(0, 100, 1, 5); @@ -93,7 +93,7 @@ public void testAddForDifferentKey() { @Test public void testAddCompareCompareWithGuava() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); com.google.common.collect.RangeSet gSet = TreeRangeSet.create(); // add 10K values for key 0 @@ -132,14 +132,14 @@ public void testAddCompareCompareWithGuava() { @Test public void testNPE() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); assertNull(set.span()); } @Test public void testDeleteCompareWithGuava() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); com.google.common.collect.RangeSet gSet = TreeRangeSet.create(); // add 10K values for key 0 @@ -193,7 +193,7 @@ public void testDeleteCompareWithGuava() { @Test public void testRemoveRangeInSameKey() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); set.addOpenClosed(0, 1, 0, 50); set.addOpenClosed(0, 97, 0, 99); set.addOpenClosed(0, 99, 1, 5); @@ -217,7 +217,7 @@ public void testRemoveRangeInSameKey() { @Test public void testSpanWithGuava() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); com.google.common.collect.RangeSet gSet = TreeRangeSet.create(); set.add(Range.openClosed(new LongPair(0, 97), new LongPair(0, 99))); gSet.add(Range.openClosed(new LongPair(0, 97), new LongPair(0, 99))); @@ -242,7 +242,7 @@ public void testSpanWithGuava() { @Test public void testFirstRange() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); assertNull(set.firstRange()); Range range = Range.openClosed(new LongPair(0, 97), new LongPair(0, 99)); set.add(range); @@ -260,7 +260,7 @@ public void testFirstRange() { @Test public void testLastRange() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); assertNull(set.lastRange()); Range range = Range.openClosed(new LongPair(0, 97), new LongPair(0, 99)); set.add(range); @@ -282,7 +282,7 @@ public void testLastRange() { @Test public void testToString() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); Range range = Range.openClosed(new LongPair(0, 97), new LongPair(0, 99)); set.add(range); assertEquals(set.toString(), "[(0:97..0:99]]"); @@ -296,7 +296,7 @@ public void testToString() { @Test public void testDeleteForDifferentKey() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); set.addOpenClosed(0, 97, 0, 99); set.addOpenClosed(0, 99, 1, 5); set.addOpenClosed(1, 9, 1, 15); @@ -327,7 +327,7 @@ public void testDeleteForDifferentKey() { @Test public void testDeleteWithAtMost() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); set.add(Range.closed(new LongPair(0, 98), new LongPair(0, 99))); set.add(Range.closed(new LongPair(0, 100), new LongPair(1, 5))); set.add(Range.closed(new LongPair(1, 10), new LongPair(1, 15))); @@ -353,7 +353,7 @@ public void testDeleteWithAtMost() { @Test public void testDeleteWithLeastMost() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); set.add(Range.closed(new LongPair(0, 98), new LongPair(0, 99))); set.add(Range.closed(new LongPair(0, 100), new LongPair(1, 5))); set.add(Range.closed(new LongPair(1, 10), new LongPair(1, 15))); @@ -382,7 +382,7 @@ public void testDeleteWithLeastMost() { @Test public void testRangeContaining() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); set.add(Range.closed(new LongPair(0, 98), new LongPair(0, 99))); set.add(Range.closed(new LongPair(0, 100), new LongPair(1, 5))); com.google.common.collect.RangeSet gSet = TreeRangeSet.create(); @@ -423,7 +423,7 @@ public void testRangeContaining() { */ @Test public void testCacheFlagConflict() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); set.add(Range.openClosed(new LongPair(0, 1), new LongPair(0, 2))); set.add(Range.openClosed(new LongPair(0, 3), new LongPair(0, 4))); assertEquals(set.toString(), "[(0:1..0:2],(0:3..0:4]]"); @@ -466,7 +466,7 @@ private List> getConnectedRange(Set> gRanges) { @Test public void testCardinality() { - ConcurrentOpenLongPairRangeSet set = new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = new OpenLongPairRangeSet<>(consumer); int v = set.cardinality(0, 0, Integer.MAX_VALUE, Integer.MAX_VALUE); assertEquals(v, 0 ); set.addOpenClosed(1, 0, 1, 20); @@ -486,8 +486,8 @@ public void testCardinality() { @Test public void testForEachResultTheSameAsForEachWithRangeBoundMapper() { - ConcurrentOpenLongPairRangeSet set = - new ConcurrentOpenLongPairRangeSet<>(consumer); + OpenLongPairRangeSet set = + new OpenLongPairRangeSet<>(consumer); LongPairRangeSet.DefaultRangeSet defaultRangeSet = new LongPairRangeSet.DefaultRangeSet<>(consumer, reverseConsumer); From 1560edb99a6617f00eb9ff56ecdff99a6f508688 Mon Sep 17 00:00:00 2001 From: ken <1647023764@qq.com> Date: Sat, 6 Jul 2024 06:26:28 +0800 Subject: [PATCH 613/699] [fix][broker] Fix MessageDeduplication replay timeout cause topic loading stuck (#23004) Co-authored-by: fanjianye (cherry picked from commit 41ef3f6fb1c0b209307d7b4e14300a377c52c5ab) (cherry picked from commit 0252671d32976bbae9a8f083648bfccbabbec402) --- .../persistent/MessageDeduplication.java | 14 ++- .../persistent/TopicDuplicationTest.java | 103 ++++++++++++++++++ 2 files changed, 113 insertions(+), 4 deletions(-) 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 ab3b799093be6..1715e09dc7ba7 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 @@ -159,11 +159,12 @@ private CompletableFuture recoverSequenceIdsMap() { log.info("[{}] Replaying {} entries for deduplication", topic.getName(), managedCursor.getNumberOfEntries()); CompletableFuture future = new CompletableFuture<>(); replayCursor(future); - return future.thenAccept(lastPosition -> { + return future.thenCompose(lastPosition -> { if (lastPosition != null && snapshotCounter >= snapshotInterval) { snapshotCounter = 0; - takeSnapshot(lastPosition); + return takeSnapshot(lastPosition); } + return CompletableFuture.completedFuture(null); }); } @@ -438,13 +439,15 @@ public void resetHighestSequenceIdPushed() { } } - private void takeSnapshot(Position position) { + private CompletableFuture takeSnapshot(Position position) { + CompletableFuture future = new CompletableFuture<>(); if (log.isDebugEnabled()) { log.debug("[{}] Taking snapshot of sequence ids map", topic.getName()); } if (!snapshotTaking.compareAndSet(false, true)) { - return; + future.complete(null); + return future; } Map snapshot = new TreeMap<>(); @@ -462,14 +465,17 @@ public void markDeleteComplete(Object ctx) { } lastSnapshotTimestamp = System.currentTimeMillis(); snapshotTaking.set(false); + future.complete(null); } @Override public void markDeleteFailed(ManagedLedgerException exception, Object ctx) { log.warn("[{}] Failed to store new deduplication snapshot at {}", topic.getName(), position); snapshotTaking.set(false); + future.completeExceptionally(exception); } }, null); + return future; } /** diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/TopicDuplicationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/TopicDuplicationTest.java index 16721ca1203fd..7069a843e9866 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/TopicDuplicationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/TopicDuplicationTest.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.service.persistent; +import static org.apache.pulsar.broker.service.persistent.PersistentTopic.DEDUPLICATION_CURSOR_NAME; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotEquals; @@ -25,6 +26,7 @@ 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.Optional; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -33,12 +35,18 @@ import lombok.Cleanup; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.impl.PositionImpl; +import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; +import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.Topic; 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.common.naming.TopicName; +import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -529,6 +537,101 @@ public void testDisableNamespacePolicyTakeSnapshotShouldNotThrowException() thro persistentTopic.checkDeduplicationSnapshot(); } + @Test + public void testFinishTakeSnapshotWhenTopicLoading() throws Exception { + cleanup(); + setup(); + + // Create a topic and wait deduplication is started. + int brokerDeduplicationEntriesInterval = 1000; + pulsar.getConfiguration().setBrokerDeduplicationEnabled(true); + pulsar.getConfiguration().setBrokerDeduplicationEntriesInterval(brokerDeduplicationEntriesInterval); + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(topic); + final PersistentTopic persistentTopic1 = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + final ManagedLedgerImpl ml1 = (ManagedLedgerImpl) persistentTopic1.getManagedLedger(); + Awaitility.await().untilAsserted(() -> { + ManagedCursorImpl cursor1 = + (ManagedCursorImpl) ml1.getCursors().get(PersistentTopic.DEDUPLICATION_CURSOR_NAME); + assertNotNull(cursor1); + }); + final MessageDeduplication deduplication1 = persistentTopic1.getMessageDeduplication(); + + + // Send 999 messages, it is less than "brokerDeduplicationEntriesInterval". + // So it would not trigger takeSnapshot + final Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topic).enableBatching(false).create(); + for (int i = 0; i < brokerDeduplicationEntriesInterval - 1; i++) { + producer.send(i + ""); + } + producer.close(); + int snapshotCounter1 = WhiteboxImpl.getInternalState(deduplication1, "snapshotCounter"); + assertEquals(snapshotCounter1, brokerDeduplicationEntriesInterval - 1); + + + // Unload and load topic, simulate topic load is timeout. + // SetBrokerDeduplicationEntriesInterval to 10, therefore recoverSequenceIdsMap#takeSnapshot + // would trigger and should update the snapshot position. + // However, if topic close and takeSnapshot are concurrent, + // it would result in takeSnapshot throw exception + admin.topics().unload(topic); + pulsar.getConfiguration().setBrokerDeduplicationEntriesInterval(10); + + // Mock message deduplication recovery speed topicLoadTimeoutSeconds + pulsar.getConfiguration().setTopicLoadTimeoutSeconds(1); + String mlPath = BrokerService.MANAGED_LEDGER_PATH_ZNODE + "/" + + TopicName.get(topic).getPersistenceNamingEncoding() + "/" + DEDUPLICATION_CURSOR_NAME; + mockZooKeeper.delay(2 * 1000, (op, path) -> { + if (mlPath.equals(path)) { + return true; + } + return false; + }); + + Field field2 = BrokerService.class.getDeclaredField("topics"); + field2.setAccessible(true); + ConcurrentOpenHashMap>> topics = + (ConcurrentOpenHashMap>>) + field2.get(pulsar.getBrokerService()); + + try { + pulsar.getBrokerService().getTopic(topic, false).join().get(); + Assert.fail(); + } catch (Exception e) { + // topic loading should timeout. + } + Awaitility.await().untilAsserted(() -> { + // topic loading timeout then close topic and remove from topicsMap + Assert.assertFalse(topics.containsKey(topic)); + }); + + + // Load topic again, setBrokerDeduplicationEntriesInterval to 10000, + // make recoverSequenceIdsMap#takeSnapshot not trigger takeSnapshot. + // But actually it should not replay again in recoverSequenceIdsMap, + // since previous topic loading should finish the replay process. + pulsar.getConfiguration().setBrokerDeduplicationEntriesInterval(10000); + pulsar.getConfiguration().setTopicLoadTimeoutSeconds(60); + PersistentTopic persistentTopic2 = + (PersistentTopic) pulsar.getBrokerService().getTopic(topic, false).join().get(); + ManagedLedgerImpl ml2 = (ManagedLedgerImpl) persistentTopic2.getManagedLedger(); + MessageDeduplication deduplication2 = persistentTopic2.getMessageDeduplication(); + + Awaitility.await().untilAsserted(() -> { + int snapshotCounter3 = WhiteboxImpl.getInternalState(deduplication2, "snapshotCounter"); + Assert.assertEquals(snapshotCounter3, 0); + Assert.assertEquals(ml2.getLedgersInfo().size(), 1); + }); + + + // cleanup. + admin.topics().delete(topic); + cleanup(); + setup(); + } + private void waitCacheInit(String topicName) throws Exception { pulsarClient.newConsumer().topic(topicName).subscriptionName("my-sub").subscribe().close(); TopicName topic = TopicName.get(topicName); From b488f1f9d3461b0beecfa933de49b8d4db6167d5 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Sat, 6 Jul 2024 13:29:00 +0300 Subject: [PATCH 614/699] [improve][broker] Use RoaringBitmap in tracking individual acks to reduce memory usage (#23006) (cherry picked from commit ed39c4db671c29057e51b9142a0d4cdb71e3eb88) (cherry picked from commit 278d6798061c9ad82828895a2bb63d1479e6e524) --- distribution/server/src/assemble/LICENSE.bin.txt | 3 +-- distribution/shell/src/assemble/LICENSE.bin.txt | 2 ++ .../bookkeeper/mledger/impl/RangeSetWrapper.java | 2 +- pom.xml | 2 +- pulsar-common/pom.xml | 5 +++++ .../util/collections/OpenLongPairRangeSet.java | 15 ++------------- pulsar-sql/presto-distribution/LICENSE | 3 ++- 7 files changed, 14 insertions(+), 18 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 987c201683cf1..11a3703e5a200 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -498,8 +498,7 @@ The Apache Software License, Version 2.0 * RxJava - io.reactivex.rxjava3-rxjava-3.0.1.jar * RoaringBitmap - - org.roaringbitmap-RoaringBitmap-0.9.44.jar - - org.roaringbitmap-shims-0.9.44.jar + - org.roaringbitmap-RoaringBitmap-1.2.0.jar BSD 3-clause "New" or "Revised" License * Google auth library diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 233fc3d22b857..f753d09302702 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -379,6 +379,8 @@ The Apache Software License, Version 2.0 - simpleclient_tracer_common-0.16.0.jar - simpleclient_tracer_otel-0.16.0.jar - simpleclient_tracer_otel_agent-0.16.0.jar + * RoaringBitmap + - RoaringBitmap-1.2.0.jar * Log4J - log4j-api-2.18.0.jar - log4j-core-2.18.0.jar 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 299fd3dc74cb4..c193d71c64f7d 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 @@ -55,7 +55,7 @@ public RangeSetWrapper(LongPairConsumer rangeConverter, this.config = managedCursor.getManagedLedger().getConfig(); this.rangeConverter = rangeConverter; this.rangeSet = config.isUnackedRangesOpenCacheSetEnabled() - ? new OpenLongPairRangeSet<>(4096, rangeConverter) + ? new OpenLongPairRangeSet<>(rangeConverter) : new LongPairRangeSet.DefaultRangeSet<>(rangeConverter, rangeBoundConsumer); this.enableMultiEntry = config.isPersistentUnackedRangesWithMultipleEntriesEnabled(); } diff --git a/pom.xml b/pom.xml index 605fc4a9db7de..1100d184c36c6 100644 --- a/pom.xml +++ b/pom.xml @@ -307,7 +307,7 @@ flexible messaging model and an intuitive client API. 1.3 0.4 10.0.2 - 0.9.44 + 1.2.0 1.6.1 6.4.0 diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 291432daaa57c..4a084a34625a5 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -243,6 +243,11 @@ awaitility test + + + org.roaringbitmap + RoaringBitmap + 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 c053c106be206..5114675324ad7 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 @@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import javax.annotation.concurrent.NotThreadSafe; import org.apache.commons.lang.mutable.MutableInt; +import org.roaringbitmap.RoaringBitSet; /** * A Concurrent set comprising zero or more ranges of type {@link LongPair}. This can be alternative of @@ -46,8 +47,6 @@ public class OpenLongPairRangeSet> implements LongPairRangeSet { protected final NavigableMap rangeBitSetMap = new ConcurrentSkipListMap<>(); - private boolean threadSafe = true; - private final int bitSetSize; private final LongPairConsumer consumer; // caching place-holder for cpu-optimization to avoid calculating ranges again @@ -57,16 +56,6 @@ public class OpenLongPairRangeSet> implements LongPairRa private volatile boolean updatedAfterCachedForToString = true; public OpenLongPairRangeSet(LongPairConsumer consumer) { - this(1024, true, consumer); - } - - public OpenLongPairRangeSet(int size, LongPairConsumer consumer) { - this(size, true, consumer); - } - - public OpenLongPairRangeSet(int size, boolean threadSafe, LongPairConsumer consumer) { - this.threadSafe = threadSafe; - this.bitSetSize = size; this.consumer = consumer; } @@ -416,7 +405,7 @@ private int getSafeEntry(long value) { } private BitSet createNewBitSet() { - return this.threadSafe ? new ConcurrentBitSet(bitSetSize) : new BitSet(bitSetSize); + return new RoaringBitSet(); } } diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 37e474cd6f585..a92059903e757 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -477,7 +477,8 @@ The Apache Software License, Version 2.0 - stream-2.9.5.jar * High Performance Primitive Collections for Java - hppc-0.9.1.jar - + * RoaringBitmap + - RoaringBitmap-1.2.0.jar Protocol Buffers License * Protocol Buffers From c917de340dc7a3083aca3bf364163abcd5ebd9bd Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 8 Jul 2024 13:55:56 +0300 Subject: [PATCH 615/699] [fix][misc] Remove RoaringBitmap dependency from pulsar-common (#23008) (cherry picked from commit 32e29a3d45c2de5560e22201b0b4bfd5409f12f2) (cherry picked from commit 57a91b08f3d039d70c2dfd037a1b2c941540edab) --- distribution/shell/src/assemble/LICENSE.bin.txt | 2 -- managed-ledger/pom.xml | 4 ++++ .../bookkeeper/mledger/impl/RangeSetWrapper.java | 3 ++- pulsar-common/pom.xml | 5 ----- .../common/util/collections/OpenLongPairRangeSet.java | 10 ++++++++-- 5 files changed, 14 insertions(+), 10 deletions(-) diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index f753d09302702..233fc3d22b857 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -379,8 +379,6 @@ The Apache Software License, Version 2.0 - simpleclient_tracer_common-0.16.0.jar - simpleclient_tracer_otel-0.16.0.jar - simpleclient_tracer_otel_agent-0.16.0.jar - * RoaringBitmap - - RoaringBitmap-1.2.0.jar * Log4J - log4j-api-2.18.0.jar - log4j-core-2.18.0.jar diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index dfc33c776ade9..e45283576af72 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -104,6 +104,10 @@ + + org.roaringbitmap + RoaringBitmap + io.dropwizard.metrics metrics-core 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 c193d71c64f7d..a55e6444b2fd9 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 @@ -27,6 +27,7 @@ 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; /** * Wraps other Range classes, and adds LRU, marking dirty data and other features on this basis. @@ -55,7 +56,7 @@ public RangeSetWrapper(LongPairConsumer rangeConverter, this.config = managedCursor.getManagedLedger().getConfig(); this.rangeConverter = rangeConverter; this.rangeSet = config.isUnackedRangesOpenCacheSetEnabled() - ? new OpenLongPairRangeSet<>(rangeConverter) + ? new OpenLongPairRangeSet<>(rangeConverter, RoaringBitSet::new) : new LongPairRangeSet.DefaultRangeSet<>(rangeConverter, rangeBoundConsumer); this.enableMultiEntry = config.isPersistentUnackedRangesWithMultipleEntriesEnabled(); } diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index 4a084a34625a5..291432daaa57c 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -243,11 +243,6 @@ awaitility test - - - org.roaringbitmap - RoaringBitmap - 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 5114675324ad7..6df6d414871ec 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 @@ -28,9 +28,9 @@ import java.util.NavigableMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; import javax.annotation.concurrent.NotThreadSafe; import org.apache.commons.lang.mutable.MutableInt; -import org.roaringbitmap.RoaringBitSet; /** * A Concurrent set comprising zero or more ranges of type {@link LongPair}. This can be alternative of @@ -48,6 +48,7 @@ public class OpenLongPairRangeSet> implements LongPairRa protected final NavigableMap rangeBitSetMap = new ConcurrentSkipListMap<>(); private final LongPairConsumer consumer; + private final Supplier bitSetSupplier; // caching place-holder for cpu-optimization to avoid calculating ranges again private volatile int cachedSize = 0; @@ -56,7 +57,12 @@ public class OpenLongPairRangeSet> implements LongPairRa private volatile boolean updatedAfterCachedForToString = true; public OpenLongPairRangeSet(LongPairConsumer consumer) { + this(consumer, BitSet::new); + } + + public OpenLongPairRangeSet(LongPairConsumer consumer, Supplier bitSetSupplier) { this.consumer = consumer; + this.bitSetSupplier = bitSetSupplier; } /** @@ -405,7 +411,7 @@ private int getSafeEntry(long value) { } private BitSet createNewBitSet() { - return new RoaringBitSet(); + return bitSetSupplier.get(); } } From 044840a967926fe66ea0b14e189e9ea7c3d8a11f Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 9 Jul 2024 08:53:54 +0800 Subject: [PATCH 616/699] [fix][client] Fix pattern consumer create crash if a part of partitions of a topic have been deleted (#22854) (cherry picked from commit 9626e7e090e9481e12441a47cf7e89f209aadd03) (cherry picked from commit 98d4a53339b7d8b97b61a96ab838614a33e07db2) --- .../pulsar/broker/service/TopicGCTest.java | 291 +++++++++++++++++- .../api/PatternMultiTopicsConsumerTest.java | 37 +++ .../impl/PatternTopicsConsumerImplTest.java | 13 +- .../client/impl/TopicsConsumerImplTest.java | 5 +- .../pulsar/client/impl/ConsumerBase.java | 6 + .../pulsar/client/impl/LookupService.java | 9 +- .../client/impl/MultiTopicsConsumerImpl.java | 188 ++++++----- .../impl/PatternConsumerUpdateQueue.java | 254 +++++++++++++++ .../impl/PatternMultiTopicsConsumerImpl.java | 249 ++++++++++----- .../pulsar/client/impl/PulsarClientImpl.java | 15 +- .../pulsar/client/impl/TopicListWatcher.java | 16 +- .../impl/PatternConsumerUpdateQueueTest.java | 247 +++++++++++++++ .../PatternMultiTopicsConsumerImplTest.java | 6 +- .../client/impl/TopicListWatcherTest.java | 12 +- .../pulsar/common/lookup/GetTopicsResult.java | 24 ++ .../pulsar/common/naming/TopicName.java | 9 + .../apache/pulsar/common/util/FutureUtil.java | 3 + 17 files changed, 1187 insertions(+), 197 deletions(-) create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java create mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueueTest.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java index 8fdf0723ea8d1..172bd3702e129 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java @@ -18,24 +18,34 @@ */ package org.apache.pulsar.broker.service; +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.ConcurrentHashMap; import java.util.concurrent.TimeUnit; +import java.util.regex.Pattern; import lombok.EqualsAndHashCode; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.ConsumerBuilder; 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.Schema; +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.TopicMessageId; +import org.apache.pulsar.client.impl.ConsumerImpl; +import org.apache.pulsar.client.impl.PatternMultiTopicsConsumerImpl; import org.apache.pulsar.common.policies.data.InactiveTopicDeleteMode; import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Slf4j @@ -58,14 +68,38 @@ protected void cleanup() throws Exception { @EqualsAndHashCode.Include protected void doInitConf() throws Exception { super.doInitConf(); - this.conf.setBrokerDeleteInactiveTopicsEnabled(true); - this.conf.setBrokerDeleteInactiveTopicsMode(InactiveTopicDeleteMode.delete_when_subscriptions_caught_up); - this.conf.setBrokerDeleteInactiveTopicsFrequencySeconds(10); + conf.setBrokerDeleteInactiveTopicsEnabled(true); + conf.setBrokerDeleteInactiveTopicsMode( + InactiveTopicDeleteMode.delete_when_subscriptions_caught_up); + conf.setBrokerDeleteInactiveTopicsFrequencySeconds(10); } - @Test - public void testCreateConsumerAfterOnePartDeleted() throws Exception { + private enum SubscribeTopicType { + MULTI_PARTITIONED_TOPIC, + REGEX_TOPIC; + } + + @DataProvider(name = "subscribeTopicTypes") + public Object[][] subTopicTypes() { + return new Object[][]{ + {SubscribeTopicType.MULTI_PARTITIONED_TOPIC}, + {SubscribeTopicType.REGEX_TOPIC} + }; + } + + private void setSubscribeTopic(ConsumerBuilder consumerBuilder, SubscribeTopicType subscribeTopicType, + String topicName, String topicPattern) { + if (subscribeTopicType.equals(SubscribeTopicType.MULTI_PARTITIONED_TOPIC)) { + consumerBuilder.topic(topicName); + } else { + consumerBuilder.topicsPattern(Pattern.compile(topicPattern)); + } + } + + @Test(dataProvider = "subscribeTopicTypes", timeOut = 300 * 1000) + public void testRecreateConsumerAfterOnePartGc(SubscribeTopicType subscribeTopicType) throws Exception { final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String topicPattern = "persistent://public/default/tp.*"; final String partition0 = topic + "-partition-0"; final String partition1 = topic + "-partition-1"; final String subscription = "s1"; @@ -77,8 +111,12 @@ public void testCreateConsumerAfterOnePartDeleted() throws Exception { .enableBatching(false).create(); Producer producer1 = pulsarClient.newProducer(Schema.STRING).topic(partition1) .enableBatching(false).create(); - org.apache.pulsar.client.api.Consumer consumer1 = pulsarClient.newConsumer(Schema.STRING).topic(topic) - .subscriptionName(subscription).isAckReceiptEnabled(true).subscribe(); + ConsumerBuilder consumerBuilder1 = pulsarClient.newConsumer(Schema.STRING) + .subscriptionName(subscription) + .isAckReceiptEnabled(true) + .subscriptionType(SubscriptionType.Shared); + setSubscribeTopic(consumerBuilder1, subscribeTopicType, topic, topicPattern); + Consumer consumer1 = consumerBuilder1.subscribe(); // Make consume all messages for one topic, do not consume any messages for another one. producer0.send("1"); @@ -97,18 +135,247 @@ public void testCreateConsumerAfterOnePartDeleted() throws Exception { }); // Verify that the consumer subscribed with partitioned topic can be created successful. - Consumer consumerAllPartition = pulsarClient.newConsumer(Schema.STRING).topic(topic) - .subscriptionName(subscription).isAckReceiptEnabled(true).subscribe(); - Message msg = consumerAllPartition.receive(2, TimeUnit.SECONDS); + ConsumerBuilder consumerBuilder2 = pulsarClient.newConsumer(Schema.STRING) + .subscriptionName(subscription) + .isAckReceiptEnabled(true) + .subscriptionType(SubscriptionType.Shared); + setSubscribeTopic(consumerBuilder2, subscribeTopicType, topic, topicPattern); + Consumer consumer2 = consumerBuilder2.subscribe(); + Message msg = consumer2.receive(2, TimeUnit.SECONDS); + String receivedMsgValue = msg.getValue(); + log.info("received msg: {}", receivedMsgValue); + consumer2.acknowledge(msg); + + // cleanup. + consumer2.close(); + producer0.close(); + producer1.close(); + admin.topics().deletePartitionedTopic(topic); + } + + @Test(dataProvider = "subscribeTopicTypes", timeOut = 300 * 1000) + public void testAppendCreateConsumerAfterOnePartGc(SubscribeTopicType subscribeTopicType) throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String topicPattern = "persistent://public/default/tp.*"; + final String partition0 = topic + "-partition-0"; + final String partition1 = topic + "-partition-1"; + final String subscription = "s1"; + admin.topics().createPartitionedTopic(topic, 2); + admin.topics().createSubscription(topic, subscription, MessageId.earliest); + + // create consumers and producers. + Producer producer0 = pulsarClient.newProducer(Schema.STRING).topic(partition0) + .enableBatching(false).create(); + Producer producer1 = pulsarClient.newProducer(Schema.STRING).topic(partition1) + .enableBatching(false).create(); + ConsumerBuilder consumerBuilder1 = pulsarClient.newConsumer(Schema.STRING) + .subscriptionName(subscription) + .isAckReceiptEnabled(true) + .subscriptionType(SubscriptionType.Shared); + setSubscribeTopic(consumerBuilder1, subscribeTopicType, topic, topicPattern); + Consumer consumer1 = consumerBuilder1.subscribe(); + + // Make consume all messages for one topic, do not consume any messages for another one. + producer0.send("partition-0-1"); + producer1.send("partition-1-1"); + producer1.send("partition-1-2"); + producer1.send("partition-1-4"); + admin.topics().skipAllMessages(partition0, subscription); + + // Wait for topic GC. + // Partition 0 will be deleted about 20s later, left 2min to avoid flaky. + producer0.close(); + Awaitility.await().atMost(2, TimeUnit.MINUTES).untilAsserted(() -> { + CompletableFuture> tp1 = pulsar.getBrokerService().getTopic(partition0, false); + CompletableFuture> tp2 = pulsar.getBrokerService().getTopic(partition1, false); + assertTrue(tp1 == null || !tp1.get().isPresent()); + assertTrue(tp2 != null && tp2.get().isPresent()); + }); + + // Verify that the messages under "partition-1" still can be ack. + for (int i = 0; i < 2; i++) { + Message msg = consumer1.receive(2, TimeUnit.SECONDS); + assertNotNull(msg, "Expected at least received 2 messages."); + log.info("received msg[{}]: {}", i, msg.getValue()); + TopicMessageId messageId = (TopicMessageId) msg.getMessageId(); + if (messageId.getOwnerTopic().equals(partition1)) { + consumer1.acknowledgeAsync(msg); + } + } + consumer1.close(); + + // Verify that the consumer subscribed with partitioned topic can be created successful. + ConsumerBuilder consumerBuilder2 = pulsarClient.newConsumer(Schema.STRING) + .subscriptionName(subscription) + .isAckReceiptEnabled(true) + .subscriptionType(SubscriptionType.Shared); + setSubscribeTopic(consumerBuilder2, subscribeTopicType, topic, topicPattern); + Consumer consumer2 = consumerBuilder2.subscribe(); + producer1.send("partition-1-5"); + Message msg = consumer2.receive(2, TimeUnit.SECONDS); assertNotNull(msg); String receivedMsgValue = msg.getValue(); log.info("received msg: {}", receivedMsgValue); - consumerAllPartition.acknowledge(msg); + consumer2.acknowledge(msg); // cleanup. - consumerAllPartition.close(); + consumer2.close(); producer0.close(); producer1.close(); admin.topics().deletePartitionedTopic(topic); } + + @Test(timeOut = 180 * 1000) + public void testPhasePartDeletion() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String topicPattern = "persistent://public/default/tp.*"; + final String partition0 = topic + "-partition-0"; + final String partition1 = topic + "-partition-1"; + final String partition2 = topic + "-partition-2"; + final String subscription = "s1"; + admin.topics().createPartitionedTopic(topic, 3); + // Create consumer. + PatternMultiTopicsConsumerImpl c1 = (PatternMultiTopicsConsumerImpl) pulsarClient + .newConsumer(Schema.STRING) + .subscriptionName(subscription) + .isAckReceiptEnabled(true) + .subscriptionType(SubscriptionType.Shared) + .topicsPattern(Pattern.compile(topicPattern)).subscribe(); + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 1); + assertEquals(partitionedTopics.get(topic), 3); + assertEquals(consumers.size(), 3); + assertTrue(consumers.containsKey(partition0)); + assertTrue(consumers.containsKey(partition1)); + assertTrue(consumers.containsKey(partition2)); + }); + // Delete partitions the first time. + admin.topics().delete(partition0, true); + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 1); + assertEquals(partitionedTopics.get(topic), 3); + assertEquals(consumers.size(), 2); + assertTrue(consumers.containsKey(partition1)); + assertTrue(consumers.containsKey(partition2)); + }); + // Delete partitions the second time. + admin.topics().delete(partition1, true); + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 1); + assertEquals(partitionedTopics.get(topic), 3); + assertEquals(consumers.size(), 1); + assertTrue(consumers.containsKey(partition2)); + }); + // Delete partitions the third time. + admin.topics().delete(partition2, true); + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 0); + assertEquals(consumers.size(), 0); + }); + + // cleanup. + c1.close(); + admin.topics().deletePartitionedTopic(topic); + } + + @Test(timeOut = 180 * 1000) + public void testExpandPartitions() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String topicPattern = "persistent://public/default/tp.*"; + final String partition0 = topic + "-partition-0"; + final String partition1 = topic + "-partition-1"; + final String subscription = "s1"; + admin.topics().createPartitionedTopic(topic, 2); + // Delete partitions. + admin.topics().delete(partition0, true); + admin.topics().delete(partition1, true); + // Create consumer. + PatternMultiTopicsConsumerImpl c1 = (PatternMultiTopicsConsumerImpl) pulsarClient + .newConsumer(Schema.STRING) + .subscriptionName(subscription) + .isAckReceiptEnabled(true) + .subscriptionType(SubscriptionType.Shared) + .topicsPattern(Pattern.compile(topicPattern)).subscribe(); + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 0); + assertEquals(consumers.size(), 0); + }); + // Trigger partitions creation. + pulsarClient.newConsumer(Schema.STRING).subscriptionName(subscription) + .subscriptionType(SubscriptionType.Shared).topic(topic).subscribe().close(); + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 1); + assertEquals(partitionedTopics.get(topic), 2); + assertEquals(consumers.size(), 2); + assertTrue(consumers.containsKey(partition0)); + assertTrue(consumers.containsKey(partition1)); + }); + // Expand partitions the first time. + admin.topics().updatePartitionedTopic(topic, 3); + final String partition2 = topic + "-partition-2"; + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 1); + assertEquals(partitionedTopics.get(topic), 3); + assertEquals(consumers.size(), 3); + assertTrue(consumers.containsKey(partition0)); + assertTrue(consumers.containsKey(partition1)); + assertTrue(consumers.containsKey(partition2)); + }); + // Expand partitions the second time. + admin.topics().updatePartitionedTopic(topic, 4); + final String partition3 = topic + "-partition-3"; + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 1); + assertEquals(partitionedTopics.get(topic), 4); + assertEquals(consumers.size(), 4); + assertTrue(consumers.containsKey(partition0)); + assertTrue(consumers.containsKey(partition1)); + assertTrue(consumers.containsKey(partition2)); + assertTrue(consumers.containsKey(partition3)); + }); + + // cleanup. + c1.close(); + admin.topics().deletePartitionedTopic(topic); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternMultiTopicsConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternMultiTopicsConsumerTest.java index 00a47c3957150..475477ac52149 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternMultiTopicsConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternMultiTopicsConsumerTest.java @@ -18,11 +18,14 @@ */ package org.apache.pulsar.client.api; +import static org.testng.Assert.fail; import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.client.impl.PatternMultiTopicsConsumerImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; @@ -95,4 +98,38 @@ private void testWithConsumer(Consumer consumer) throws Exception { consumer.close(); } + @Test(timeOut = 30000) + public void testFailedSubscribe() throws Exception { + final String topicName1 = BrokerTestUtil.newUniqueName("persistent://public/default/tp_test"); + final String topicName2 = BrokerTestUtil.newUniqueName("persistent://public/default/tp_test"); + final String topicName3 = BrokerTestUtil.newUniqueName("persistent://public/default/tp_test"); + final String subName = "s1"; + admin.topics().createPartitionedTopic(topicName1, 2); + admin.topics().createPartitionedTopic(topicName2, 3); + admin.topics().createNonPartitionedTopic(topicName3); + + // Register a exclusive consumer to makes the pattern consumer failed to subscribe. + Consumer c1 = pulsarClient.newConsumer(Schema.STRING).topic(topicName3).subscriptionType(SubscriptionType.Exclusive) + .subscriptionName(subName).subscribe(); + + try { + PatternMultiTopicsConsumerImpl consumer = + (PatternMultiTopicsConsumerImpl) pulsarClient.newConsumer(Schema.STRING) + .topicsPattern("persistent://public/default/tp_test.*") + .subscriptionType(SubscriptionType.Failover) + .subscriptionName(subName) + .subscribe(); + fail("Expected a consumer busy error."); + } catch (Exception ex) { + log.info("consumer busy", ex); + } + + c1.close(); + // Verify all internal consumer will be closed. + // If delete topic without "-f" work, it means the internal consumers were closed. + admin.topics().delete(topicName3); + admin.topics().deletePartitionedTopic(topicName2); + admin.topics().deletePartitionedTopic(topicName1); + } + } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java index 7707abafde8de..1bd621738b3fb 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java @@ -19,7 +19,6 @@ package org.apache.pulsar.client.impl; import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertSame; import static org.testng.Assert.assertTrue; @@ -35,7 +34,6 @@ import java.util.regex.Pattern; import java.util.stream.IntStream; -import io.netty.util.Timeout; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.client.api.Consumer; @@ -52,6 +50,7 @@ import org.apache.pulsar.common.api.proto.BaseCommand; import org.apache.pulsar.common.api.proto.CommandWatchTopicListSuccess; import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; @@ -1023,17 +1022,17 @@ public void testAutoUnsubscribePatternConsumer() throws Exception { // 6. remove producer 1,3; verify only consumer 2 left // seems no direct way to verify auto-unsubscribe, because this patternConsumer also referenced the topic. - List topicNames = Lists.newArrayList(topicName2); + String tp2p0 = TopicName.get(topicName2).getPartition(0).toString(); + String tp2p1 = TopicName.get(topicName2).getPartition(1).toString(); + List topicNames = Lists.newArrayList(tp2p0, tp2p1); NamespaceService nss = pulsar.getNamespaceService(); doReturn(CompletableFuture.completedFuture(topicNames)).when(nss) .getListOfPersistentTopics(NamespaceName.get("my-property/my-ns")); // 7. call recheckTopics to unsubscribe topic 1,3, verify topics number: 2=6-1-3 log.debug("recheck topics change"); - PatternMultiTopicsConsumerImpl consumer1 = ((PatternMultiTopicsConsumerImpl) consumer); - Timeout recheckPatternTimeout = spy(consumer1.getRecheckPatternTimeout()); - doReturn(false).when(recheckPatternTimeout).isCancelled(); - consumer1.run(recheckPatternTimeout); + PatternConsumerUpdateQueue taskQueue = WhiteboxImpl.getInternalState(consumer, "updateTaskQueue"); + taskQueue.appendRecheckOp(); Thread.sleep(100); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getPartitions().size(), 2); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getConsumers().size(), 2); 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 c343ab0d6e294..83cb5f2a4400b 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 @@ -21,6 +21,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; import io.netty.util.Timeout; +import java.time.Duration; import lombok.Cleanup; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; @@ -1321,7 +1322,6 @@ public void testPartitionsUpdatesForMultipleTopics() throws Exception { Assert.assertEquals(consumer.allTopicPartitionsNumber.intValue(), 2); admin.topics().updatePartitionedTopic(topicName0, 5); - consumer.getPartitionsAutoUpdateTimeout().task().run(consumer.getPartitionsAutoUpdateTimeout()); Awaitility.await().untilAsserted(() -> { Assert.assertEquals(consumer.getPartitionsOfTheTopicMap(), 5); @@ -1341,9 +1341,8 @@ public void testPartitionsUpdatesForMultipleTopics() throws Exception { }); admin.topics().updatePartitionedTopic(topicName1, 5); - consumer.getPartitionsAutoUpdateTimeout().task().run(consumer.getPartitionsAutoUpdateTimeout()); - Awaitility.await().untilAsserted(() -> { + Awaitility.await().atMost(Duration.ofSeconds(30)).untilAsserted(() -> { Assert.assertEquals(consumer.getPartitionsOfTheTopicMap(), 10); Assert.assertEquals(consumer.allTopicPartitionsNumber.intValue(), 10); }); 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 31357e8225fdf..3747dc23d9d10 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 @@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Queues; import io.netty.util.Timeout; import java.nio.charset.StandardCharsets; @@ -1275,5 +1276,10 @@ public boolean hasBatchReceiveTimeout() { return batchReceiveTimeout != null; } + @VisibleForTesting + CompletableFuture> getSubscribeFuture() { + return subscribeFuture; + } + private static final Logger log = LoggerFactory.getLogger(ConsumerBase.class); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java index ba99cb77550f5..675781b122f0a 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java @@ -117,7 +117,14 @@ CompletableFuture getPartitionedTopicMetadata(TopicNam InetSocketAddress resolveHost(); /** - * Returns all the topics name for a given namespace. + * Returns all the topics that matches {@param topicPattern} for a given namespace. + * + * Note: {@param topicPattern} it relate to the topic name(without the partition suffix). For example: + * - There is a partitioned topic "tp-a" with two partitions. + * - tp-a-partition-0 + * - tp-a-partition-1 + * - If {@param topicPattern} is "tp-a", the consumer will subscribe to the two partitions. + * - if {@param topicPattern} is "tp-a-partition-0", the consumer will subscribe nothing. * * @param namespace : namespace-name * @return 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 62b6612fa3c26..526689934f8f1 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 @@ -47,6 +47,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; +import java.util.function.Predicate; +import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.IntStream; import javax.annotation.Nullable; @@ -68,6 +70,7 @@ import org.apache.pulsar.client.util.ConsumerName; import org.apache.pulsar.client.util.ExecutorProvider; import org.apache.pulsar.common.api.proto.CommandAck.AckType; +import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.util.CompletableFutureCancellationHandler; @@ -81,14 +84,14 @@ public class MultiTopicsConsumerImpl extends ConsumerBase { public static final String DUMMY_TOPIC_NAME_PREFIX = "MultiTopicsConsumer-"; // Map , when get do ACK, consumer will by find by topic name - private final ConcurrentHashMap> consumers; + protected final ConcurrentHashMap> consumers; // Map , store partition number for each topic protected final ConcurrentHashMap partitionedTopics; // Queue of partition consumers on which we have stopped calling receiveAsync() because the // shared incoming queue was full - private final ConcurrentLinkedQueue> pausedConsumers; + protected final ConcurrentLinkedQueue> pausedConsumers; // sum of topicPartitions, simple topic has 1, partitioned topic equals to partition number. AtomicInteger allTopicPartitionsNumber; @@ -1009,8 +1012,12 @@ CompletableFuture subscribeAsync(String topicName, int numberPartitions) { new PulsarClientException.AlreadyClosedException("Topic name not valid")); } String fullTopicName = topicNameInstance.toString(); - if (consumers.containsKey(fullTopicName) - || partitionedTopics.containsKey(topicNameInstance.getPartitionedTopicName())) { + if (consumers.containsKey(fullTopicName)) { + return FutureUtil.failedFuture( + new PulsarClientException.AlreadyClosedException("Already subscribed to " + topicName)); + } + if (!topicNameInstance.isPartitioned() + && partitionedTopics.containsKey(topicNameInstance.getPartitionedTopicName())) { return FutureUtil.failedFuture( new PulsarClientException.AlreadyClosedException("Already subscribed to " + topicName)); } @@ -1047,7 +1054,7 @@ private void doSubscribeTopicPartitions(Schema schema, log.debug("Subscribe to topic {} metadata.partitions: {}", topicName, numPartitions); } - List>> futureList; + CompletableFuture subscribeAllPartitionsFuture; if (numPartitions != PartitionedTopicMetadata.NON_PARTITIONED) { // Below condition is true if subscribeAsync() has been invoked second time with same // topicName before the first invocation had reached this point. @@ -1067,30 +1074,50 @@ private void doSubscribeTopicPartitions(Schema schema, ConsumerConfigurationData configurationData = getInternalConsumerConfig(); configurationData.setReceiverQueueSize(receiverQueueSize); - futureList = IntStream - .range(0, numPartitions) - .mapToObj( - partitionIndex -> { - String partitionName = TopicName.get(topicName).getPartition(partitionIndex).toString(); - CompletableFuture> subFuture = new CompletableFuture<>(); - configurationData.setStartPaused(paused); - ConsumerImpl newConsumer = createInternalConsumer(configurationData, partitionName, - partitionIndex, subFuture, createIfDoesNotExist, schema); - synchronized (pauseMutex) { - if (paused) { - newConsumer.pause(); - } else { - newConsumer.resume(); - } - consumers.putIfAbsent(newConsumer.getTopic(), newConsumer); + CompletableFuture> partitionsFuture; + if (createIfDoesNotExist || !TopicName.get(topicName).isPersistent()) { + partitionsFuture = CompletableFuture.completedFuture(IntStream.range(0, numPartitions) + .mapToObj(i -> Integer.valueOf(i)) + .collect(Collectors.toList())); + } else { + partitionsFuture = getExistsPartitions(topicName.toString()); + } + subscribeAllPartitionsFuture = partitionsFuture.thenCompose(partitions -> { + if (partitions.isEmpty()) { + partitionedTopics.remove(topicName, numPartitions); + return CompletableFuture.completedFuture(null); + } + List>> subscribeList = new ArrayList<>(); + for (int partitionIndex : partitions) { + String partitionName = TopicName.get(topicName).getPartition(partitionIndex).toString(); + CompletableFuture> subFuture = new CompletableFuture<>(); + configurationData.setStartPaused(paused); + ConsumerImpl newConsumer = createInternalConsumer(configurationData, partitionName, + partitionIndex, subFuture, createIfDoesNotExist, schema); + synchronized (pauseMutex) { + if (paused) { + newConsumer.pause(); + } else { + newConsumer.resume(); } - return subFuture; - }) - .collect(Collectors.toList()); + Consumer originalValue = consumers.putIfAbsent(newConsumer.getTopic(), newConsumer); + if (originalValue != null) { + newConsumer.closeAsync().exceptionally(ex -> { + log.error("[{}] [{}] Failed to close the orphan consumer", + partitionName, subscription, ex); + return null; + }); + } + } + subscribeList.add(subFuture); + } + return FutureUtil.waitForAll(subscribeList); + }); } else { allTopicPartitionsNumber.incrementAndGet(); - CompletableFuture> subFuture = new CompletableFuture<>(); + CompletableFuture> subscribeFuture = new CompletableFuture<>(); + subscribeAllPartitionsFuture = subscribeFuture.thenAccept(__ -> {}); synchronized (pauseMutex) { consumers.compute(topicName, (key, existingValue) -> { @@ -1104,7 +1131,7 @@ private void doSubscribeTopicPartitions(Schema schema, } else { internalConfig.setStartPaused(paused); ConsumerImpl newConsumer = createInternalConsumer(internalConfig, topicName, - -1, subFuture, createIfDoesNotExist, schema); + -1, subscribeFuture, createIfDoesNotExist, schema); if (paused) { newConsumer.pause(); } else { @@ -1114,11 +1141,10 @@ private void doSubscribeTopicPartitions(Schema schema, } }); } - futureList = Collections.singletonList(subFuture); + } - FutureUtil.waitForAll(futureList) - .thenAccept(finalFuture -> { + subscribeAllPartitionsFuture.thenAccept(finalFuture -> { if (allTopicPartitionsNumber.get() > getCurrentReceiverQueueSize()) { setCurrentReceiverQueueSize(allTopicPartitionsNumber.get()); } @@ -1139,6 +1165,8 @@ private void doSubscribeTopicPartitions(Schema schema, return; }) .exceptionally(ex -> { + log.warn("[{}] Failed to subscribe for topic [{}] in topics consumer {}", topic, topicName, + ex.getMessage()); handleSubscribeOneTopicError(topicName, ex, subscribeResult); return null; }); @@ -1162,7 +1190,7 @@ private ConsumerImpl createInternalConsumer(ConsumerConfigurationData conf } // handling failure during subscribe new topic, unsubscribe success created partitions - private void handleSubscribeOneTopicError(String topicName, + protected void handleSubscribeOneTopicError(String topicName, Throwable error, CompletableFuture subscribeFuture) { log.warn("[{}] Failed to subscribe for topic [{}] in topics consumer {}", topic, topicName, error.getMessage()); @@ -1255,59 +1283,6 @@ public CompletableFuture unsubscribeAsync(String topicName) { return unsubscribeFuture; } - /*** - * Remove a consumer for a topic. - * @param topicName topic name contains the partition suffix. - */ - public CompletableFuture removeConsumerAsync(String topicName) { - checkArgument(TopicName.isValid(topicName), "Invalid topic name:" + topicName); - - if (getState() == State.Closing || getState() == State.Closed) { - return FutureUtil.failedFuture( - new PulsarClientException.AlreadyClosedException("Topics Consumer was already closed")); - } - - CompletableFuture unsubscribeFuture = new CompletableFuture<>(); - String topicPartName = TopicName.get(topicName).getPartitionedTopicName(); - - - List> consumersToClose = consumers.values().stream() - .filter(consumer -> { - String consumerTopicName = consumer.getTopic(); - return TopicName.get(consumerTopicName).getPartitionedTopicName().equals(topicPartName); - }).collect(Collectors.toList()); - - List> futureList = consumersToClose.stream() - .map(ConsumerImpl::closeAsync).collect(Collectors.toList()); - - FutureUtil.waitForAll(futureList) - .whenComplete((r, ex) -> { - if (ex == null) { - consumersToClose.forEach(consumer1 -> { - consumers.remove(consumer1.getTopic()); - pausedConsumers.remove(consumer1); - allTopicPartitionsNumber.decrementAndGet(); - }); - - removeTopic(topicName); - if (unAckedMessageTracker instanceof UnAckedTopicMessageTracker) { - ((UnAckedTopicMessageTracker) unAckedMessageTracker).removeTopicMessages(topicName); - } - - unsubscribeFuture.complete(null); - log.info("[{}] [{}] [{}] Removed Topics Consumer, allTopicPartitionsNumber: {}", - topicName, subscription, consumerName, allTopicPartitionsNumber); - } else { - unsubscribeFuture.completeExceptionally(ex); - setState(State.Failed); - log.error("[{}] [{}] [{}] Could not remove Topics Consumer", - topicName, subscription, consumerName, ex.getCause()); - } - }); - - return unsubscribeFuture; - } - // get topics name public List getPartitionedTopics() { @@ -1573,4 +1548,51 @@ protected void setCurrentReceiverQueueSize(int newSize) { CURRENT_RECEIVER_QUEUE_SIZE_UPDATER.set(this, newSize); resumeReceivingFromPausedConsumersIfNeeded(); } + + /** + * Get the exists partitions of a partitioned topic, the result does not contain the partitions which has not been + * created yet(in other words, the partitions that do not exist in the response of "pulsar-admin topics list"). + * @return sorted partitions list if it is a partitioned topic; @return an empty list if it is a non-partitioned + * topic. + */ + private CompletableFuture> getExistsPartitions(String topic) { + TopicName topicName = TopicName.get(topic); + if (!topicName.isPersistent()) { + return FutureUtil.failedFuture(new IllegalArgumentException("The method getExistsPartitions" + + " does not support non-persistent topic yet.")); + } + return client.getLookup().getTopicsUnderNamespace(topicName.getNamespaceObject(), + CommandGetTopicsOfNamespace.Mode.PERSISTENT, + TopicName.getPattern(topicName.getPartitionedTopicName()), + null).thenApply(getTopicsResult -> { + if (getTopicsResult.getNonPartitionedOrPartitionTopics() == null + || getTopicsResult.getNonPartitionedOrPartitionTopics().isEmpty()) { + return Collections.emptyList(); + } + // If broker version is less than "2.11.x", it does not support broker-side pattern check, so append + // a client-side pattern check. + // If lookup service is typed HttpLookupService, the HTTP API does not support broker-side pattern + // check yet, so append a client-side pattern check. + Predicate clientSideFilter; + if (getTopicsResult.isFiltered()) { + clientSideFilter = __ -> true; + } else { + clientSideFilter = + tp -> Pattern.compile(TopicName.getPartitionPattern(topic)).matcher(tp).matches(); + } + ArrayList list = new ArrayList<>(getTopicsResult.getNonPartitionedOrPartitionTopics().size()); + for (String partition : getTopicsResult.getNonPartitionedOrPartitionTopics()) { + int partitionIndex = TopicName.get(partition).getPartitionIndex(); + if (partitionIndex < 0) { + // It is not a partition. + continue; + } + if (clientSideFilter.test(partition)) { + list.add(partitionIndex); + } + } + Collections.sort(list); + return list; + }); + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java new file mode 100644 index 0000000000000..d6eba6463a07d --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java @@ -0,0 +1,254 @@ +/* + * 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 com.google.common.annotations.VisibleForTesting; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.util.Collection; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.tuple.Pair; + +/** + * Used to make all tasks that will modify subscriptions will be executed one by one, and skip the unnecessary updating. + * + * So far, four three scenarios that will modify subscriptions: + * 1. When start pattern consumer. + * 2. After topic list watcher reconnected, it will call {@link PatternMultiTopicsConsumerImpl#recheckTopicsChange()}. + * this scenario only exists in the version >= 2.11 (both client-version and broker version are >= 2.11). + * 3. A scheduled task will call {@link PatternMultiTopicsConsumerImpl#recheckTopicsChange()}, this scenario only + * exists in the version < 2.11. + * 4. The topics change events will trigger a + * {@link PatternMultiTopicsConsumerImpl#topicsChangeListener#onTopicsRemoved(Collection)} or + * {@link PatternMultiTopicsConsumerImpl#topicsChangeListener#onTopicsAdded(Collection)}. + * + * When you are using this client connect to the broker whose version >= 2.11, there are three scenarios: [1, 2, 4]. + * When you are using this client connect to the broker whose version < 2.11, there is only one scenario: [3] and all + * the event will run in the same thread. + */ +@Slf4j +@SuppressFBWarnings("EI_EXPOSE_REP2") +public class PatternConsumerUpdateQueue { + + private static final Pair> RECHECK_OP = + Pair.of(UpdateSubscriptionType.RECHECK, null); + + private final LinkedBlockingQueue>> pendingTasks; + + private final PatternMultiTopicsConsumerImpl patternConsumer; + + private final PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener; + + /** + * Whether there is a task is in progress, this variable is used to confirm whether a next-task triggering is + * needed. + */ + private Pair> taskInProgress = null; + + /** + * Whether there is a recheck task in queue. + * - Since recheck task will do all changes, it can be used to compress multiple tasks to one. + * - To avoid skipping the newest changes, once the recheck task is starting to work, this variable will be set + * to "false". + */ + private boolean recheckTaskInQueue = false; + + private volatile long lastRecheckTaskStartingTimestamp = 0; + + private boolean closed; + + public PatternConsumerUpdateQueue(PatternMultiTopicsConsumerImpl patternConsumer) { + this(patternConsumer, patternConsumer.topicsChangeListener); + } + + /** This constructor is only for test. **/ + @VisibleForTesting + public PatternConsumerUpdateQueue(PatternMultiTopicsConsumerImpl patternConsumer, + PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener) { + this.patternConsumer = patternConsumer; + this.topicsChangeListener = topicsChangeListener; + this.pendingTasks = new LinkedBlockingQueue<>(); + // To avoid subscribing and topics changed events execute concurrently, let the change events starts after the + // subscribing task. + doAppend(Pair.of(UpdateSubscriptionType.CONSUMER_INIT, null)); + } + + synchronized void appendTopicsAddedOp(Collection topics) { + if (topics == null || topics.isEmpty()) { + return; + } + doAppend(Pair.of(UpdateSubscriptionType.TOPICS_ADDED, topics)); + } + + synchronized void appendTopicsRemovedOp(Collection topics) { + if (topics == null || topics.isEmpty()) { + return; + } + doAppend(Pair.of(UpdateSubscriptionType.TOPICS_REMOVED, topics)); + } + + synchronized void appendRecheckOp() { + doAppend(RECHECK_OP); + } + + synchronized void doAppend(Pair> task) { + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] try to append task. {} {}", patternConsumer.getSubscription(), + task.getLeft(), task.getRight() == null ? "" : task.getRight()); + } + // Once there is a recheck task in queue, it means other tasks can be skipped. + if (recheckTaskInQueue) { + return; + } + + // Once there are too many tasks in queue, compress them as a recheck task. + if (pendingTasks.size() >= 30 && !task.getLeft().equals(UpdateSubscriptionType.RECHECK)) { + appendRecheckOp(); + return; + } + + pendingTasks.add(task); + if (task.getLeft().equals(UpdateSubscriptionType.RECHECK)) { + recheckTaskInQueue = true; + } + + // If no task is in-progress, trigger a task execution. + if (taskInProgress == null) { + triggerNextTask(); + } + } + + synchronized void triggerNextTask() { + if (closed) { + return; + } + + final Pair> task = pendingTasks.poll(); + + // No pending task. + if (task == null) { + taskInProgress = null; + return; + } + + // If there is a recheck task in queue, skip others and only call the recheck task. + if (recheckTaskInQueue && !task.getLeft().equals(UpdateSubscriptionType.RECHECK)) { + triggerNextTask(); + return; + } + + // Execute pending task. + CompletableFuture newTaskFuture = null; + switch (task.getLeft()) { + case CONSUMER_INIT: { + newTaskFuture = patternConsumer.getSubscribeFuture().thenAccept(__ -> {}).exceptionally(ex -> { + // If the subscribe future was failed, the consumer will be closed. + synchronized (PatternConsumerUpdateQueue.this) { + this.closed = true; + patternConsumer.closeAsync().exceptionally(ex2 -> { + log.error("Pattern consumer failed to close, this error may left orphan consumers." + + " Subscription: {}", patternConsumer.getSubscription()); + return null; + }); + } + return null; + }); + break; + } + case TOPICS_ADDED: { + newTaskFuture = topicsChangeListener.onTopicsAdded(task.getRight()); + break; + } + case TOPICS_REMOVED: { + newTaskFuture = topicsChangeListener.onTopicsRemoved(task.getRight()); + break; + } + case RECHECK: { + recheckTaskInQueue = false; + lastRecheckTaskStartingTimestamp = System.currentTimeMillis(); + newTaskFuture = patternConsumer.recheckTopicsChange(); + break; + } + default: { + throw new RuntimeException("Un-support UpdateSubscriptionType"); + } + } + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] starting task. {} {} ", patternConsumer.getSubscription(), + task.getLeft(), task.getRight() == null ? "" : task.getRight()); + } + // Trigger next pending task. + taskInProgress = Pair.of(task.getLeft(), newTaskFuture); + newTaskFuture.thenAccept(ignore -> { + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] task finished. {} {} ", patternConsumer.getSubscription(), + task.getLeft(), task.getRight() == null ? "" : task.getRight()); + } + triggerNextTask(); + }).exceptionally(ex -> { + /** + * Once a updating fails, trigger a delayed new recheck task to guarantee all things is correct. + * - Skip if there is already a recheck task in queue. + * - Skip if the last recheck task has been executed after the current time. + */ + log.error("Pattern consumer [{}] task finished. {} {}. But it failed", patternConsumer.getSubscription(), + task.getLeft(), task.getRight() == null ? "" : task.getRight(), ex); + // Skip if there is already a recheck task in queue. + synchronized (PatternConsumerUpdateQueue.this) { + if (recheckTaskInQueue || PatternConsumerUpdateQueue.this.closed) { + return null; + } + } + // Skip if the last recheck task has been executed after the current time. + long failedTime = System.currentTimeMillis(); + patternConsumer.getClient().timer().newTimeout(timeout -> { + if (lastRecheckTaskStartingTimestamp <= failedTime) { + appendRecheckOp(); + } + }, 10, TimeUnit.SECONDS); + triggerNextTask(); + return null; + }); + } + + public synchronized CompletableFuture cancelAllAndWaitForTheRunningTask() { + this.closed = true; + if (taskInProgress == null) { + return CompletableFuture.completedFuture(null); + } + // If the in-progress task is consumer init task, it means nothing is in-progress. + if (taskInProgress.getLeft().equals(UpdateSubscriptionType.CONSUMER_INIT)) { + return CompletableFuture.completedFuture(null); + } + return taskInProgress.getRight().thenAccept(__ -> {}).exceptionally(ex -> null); + } + + private enum UpdateSubscriptionType { + /** A marker that indicates the consumer's subscribe task.**/ + CONSUMER_INIT, + /** Triggered by {@link PatternMultiTopicsConsumerImpl#topicsChangeListener}.**/ + TOPICS_ADDED, + /** Triggered by {@link PatternMultiTopicsConsumerImpl#topicsChangeListener}.**/ + TOPICS_REMOVED, + /** A fully check for pattern consumer. **/ + RECHECK; + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java index ec7ff7930c0ac..91c6da26d59b0 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java @@ -27,13 +27,13 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Pattern; -import java.util.stream.Collectors; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; @@ -42,6 +42,7 @@ import org.apache.pulsar.common.lookup.GetTopicsResult; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.topics.TopicList; import org.apache.pulsar.common.util.Backoff; import org.apache.pulsar.common.util.BackoffBuilder; @@ -51,7 +52,7 @@ public class PatternMultiTopicsConsumerImpl extends MultiTopicsConsumerImpl implements TimerTask { private final Pattern topicsPattern; - private final TopicsChangedListener topicsChangeListener; + final TopicsChangedListener topicsChangeListener; private final Mode subscriptionMode; private final CompletableFuture watcherFuture = new CompletableFuture<>(); protected NamespaceName namespaceName; @@ -69,6 +70,8 @@ public class PatternMultiTopicsConsumerImpl extends MultiTopicsConsumerImpl recheckTopicsChangeAfterReconnect()); watcherFuture .thenAccept(__ -> recheckPatternTimeout.cancel()) .exceptionally(ex -> { - log.warn("Unable to create topic list watcher. Falling back to only polling for new topics", ex); + log.warn("Pattern consumer [{}] unable to create topic list watcher. Falling back to only polling" + + " for new topics", conf.getSubscriptionName(), ex); return null; }); } else { - log.debug("Not creating topic list watcher for subscription mode {}", subscriptionMode); + log.debug("Pattern consumer [{}] not creating topic list watcher for subscription mode {}", + conf.getSubscriptionName(), subscriptionMode); watcherFuture.complete(null); } } @@ -129,17 +135,7 @@ private void recheckTopicsChangeAfterReconnect() { return; } // Do check. - recheckTopicsChange().whenComplete((ignore, ex) -> { - if (ex != null) { - log.warn("[{}] Failed to recheck topics change: {}", topic, ex.getMessage()); - long delayMs = recheckPatternTaskBackoff.next(); - client.timer().newTimeout(timeout -> { - recheckTopicsChangeAfterReconnect(); - }, delayMs, TimeUnit.MILLISECONDS); - } else { - recheckPatternTaskBackoff.reset(); - } - }); + updateTaskQueue.appendRecheckOp(); } // TimerTask to recheck topics change, and trigger subscribe/unsubscribe based on the change. @@ -148,18 +144,10 @@ public void run(Timeout timeout) throws Exception { if (timeout.isCancelled()) { return; } - recheckTopicsChange().exceptionally(ex -> { - log.warn("[{}] Failed to recheck topics change: {}", topic, ex.getMessage()); - return null; - }).thenAccept(__ -> { - // schedule the next re-check task - this.recheckPatternTimeout = client.timer() - .newTimeout(PatternMultiTopicsConsumerImpl.this, - Math.max(1, conf.getPatternAutoDiscoveryPeriod()), TimeUnit.SECONDS); - }); + updateTaskQueue.appendRecheckOp(); } - private CompletableFuture recheckTopicsChange() { + CompletableFuture recheckTopicsChange() { String pattern = topicsPattern.pattern(); final int epoch = recheckPatternEpoch.incrementAndGet(); return client.getLookup().getTopicsUnderNamespace(namespaceName, subscriptionMode, pattern, topicsHash) @@ -172,22 +160,18 @@ private CompletableFuture recheckTopicsChange() { return CompletableFuture.completedFuture(null); } if (log.isDebugEnabled()) { - log.debug("Get topics under namespace {}, topics.size: {}, topicsHash: {}, filtered: {}", + log.debug("Pattern consumer [{}] get topics under namespace {}, topics.size: {}," + + " topicsHash: {}, filtered: {}", + PatternMultiTopicsConsumerImpl.this.getSubscription(), namespaceName, getTopicsResult.getTopics().size(), getTopicsResult.getTopicsHash(), getTopicsResult.isFiltered()); getTopicsResult.getTopics().forEach(topicName -> log.debug("Get topics under namespace {}, topic: {}", namespaceName, topicName)); } - final List oldTopics = new ArrayList<>(getPartitionedTopics()); - for (String partition : getPartitions()) { - TopicName topicName = TopicName.get(partition); - if (!topicName.isPartitioned() || !oldTopics.contains(topicName.getPartitionedTopicName())) { - oldTopics.add(partition); - } - } + final List oldTopics = new ArrayList<>(getPartitions()); return updateSubscriptions(topicsPattern, this::setTopicsHash, getTopicsResult, - topicsChangeListener, oldTopics); + topicsChangeListener, oldTopics, subscription); } }); } @@ -196,7 +180,8 @@ static CompletableFuture updateSubscriptions(Pattern topicsPattern, java.util.function.Consumer topicsHashSetter, GetTopicsResult getTopicsResult, TopicsChangedListener topicsChangedListener, - List oldTopics) { + List oldTopics, + String subscriptionForLog) { topicsHashSetter.accept(getTopicsResult.getTopicsHash()); if (!getTopicsResult.isChanged()) { return CompletableFuture.completedFuture(null); @@ -204,14 +189,20 @@ static CompletableFuture updateSubscriptions(Pattern topicsPattern, List newTopics; if (getTopicsResult.isFiltered()) { - newTopics = getTopicsResult.getTopics(); + newTopics = getTopicsResult.getNonPartitionedOrPartitionTopics(); } else { - newTopics = TopicList.filterTopics(getTopicsResult.getTopics(), topicsPattern); + newTopics = getTopicsResult.filterTopics(topicsPattern).getNonPartitionedOrPartitionTopics(); } final List> listenersCallback = new ArrayList<>(2); - listenersCallback.add(topicsChangedListener.onTopicsAdded(TopicList.minus(newTopics, oldTopics))); - listenersCallback.add(topicsChangedListener.onTopicsRemoved(TopicList.minus(oldTopics, newTopics))); + Set topicsAdded = TopicList.minus(newTopics, oldTopics); + Set topicsRemoved = TopicList.minus(oldTopics, newTopics); + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] Recheck pattern consumer's topics. topicsAdded: {}, topicsRemoved: {}", + subscriptionForLog, topicsAdded, topicsRemoved); + } + listenersCallback.add(topicsChangedListener.onTopicsAdded(topicsAdded)); + listenersCallback.add(topicsChangedListener.onTopicsRemoved(topicsRemoved)); return FutureUtil.waitForAll(Collections.unmodifiableList(listenersCallback)); } @@ -247,23 +238,68 @@ private class PatternTopicsChangedListener implements TopicsChangedListener { */ @Override public CompletableFuture onTopicsRemoved(Collection removedTopics) { - CompletableFuture removeFuture = new CompletableFuture<>(); - if (removedTopics.isEmpty()) { - removeFuture.complete(null); - return removeFuture; + return CompletableFuture.completedFuture(null); } - List> futures = Lists.newArrayListWithExpectedSize(partitionedTopics.size()); - removedTopics.stream().forEach(topic -> futures.add(removeConsumerAsync(topic))); - FutureUtil.waitForAll(futures) - .thenAccept(finalFuture -> removeFuture.complete(null)) - .exceptionally(ex -> { - log.warn("[{}] Failed to unsubscribe from topics: {}", topic, ex.getMessage()); - removeFuture.completeExceptionally(ex); + // Unsubscribe and remove consumers in memory. + List> unsubscribeList = new ArrayList<>(removedTopics.size()); + Set partialRemoved = new HashSet<>(removedTopics.size()); + Set partialRemovedForLog = new HashSet<>(removedTopics.size()); + for (String tp : removedTopics) { + TopicName topicName = TopicName.get(tp); + ConsumerImpl consumer = consumers.get(topicName.toString()); + if (consumer != null) { + CompletableFuture unsubscribeFuture = new CompletableFuture<>(); + consumer.closeAsync().whenComplete((__, ex) -> { + if (ex != null) { + log.error("Pattern consumer [{}] failed to unsubscribe from topics: {}", + PatternMultiTopicsConsumerImpl.this.getSubscription(), topicName.toString(), ex); + unsubscribeFuture.completeExceptionally(ex); + } else { + consumers.remove(topicName.toString(), consumer); + unsubscribeFuture.complete(null); + } + }); + unsubscribeList.add(unsubscribeFuture); + partialRemoved.add(topicName.getPartitionedTopicName()); + partialRemovedForLog.add(topicName.toString()); + } + } + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] remove topics. {}", + PatternMultiTopicsConsumerImpl.this.getSubscription(), + partialRemovedForLog); + } + + // Remove partitioned topics in memory. + return FutureUtil.waitForAll(unsubscribeList).handle((__, ex) -> { + List removedPartitionedTopicsForLog = new ArrayList<>(); + for (String groupedTopicRemoved : partialRemoved) { + Integer partitions = partitionedTopics.get(groupedTopicRemoved); + if (partitions != null) { + boolean allPartitionsHasBeenRemoved = true; + for (int i = 0; i < partitions; i++) { + if (consumers.containsKey( + TopicName.get(groupedTopicRemoved).getPartition(i).toString())) { + allPartitionsHasBeenRemoved = false; + break; + } + } + if (allPartitionsHasBeenRemoved) { + removedPartitionedTopicsForLog.add(String.format("%s with %s partitions", + groupedTopicRemoved, partitions)); + partitionedTopics.remove(groupedTopicRemoved, partitions); + } + } + } + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] remove partitioned topics because all partitions have been" + + " removed. {}", PatternMultiTopicsConsumerImpl.this.getSubscription(), + removedPartitionedTopicsForLog); + } return null; }); - return removeFuture; } /** @@ -271,29 +307,90 @@ public CompletableFuture onTopicsRemoved(Collection removedTopics) */ @Override public CompletableFuture onTopicsAdded(Collection addedTopics) { - CompletableFuture addFuture = new CompletableFuture<>(); - if (addedTopics.isEmpty()) { - addFuture.complete(null); - return addFuture; + return CompletableFuture.completedFuture(null); } - - Set addTopicPartitionedName = addedTopics.stream() - .map(addTopicName -> TopicName.get(addTopicName).getPartitionedTopicName()) - .collect(Collectors.toSet()); - - List> futures = Lists.newArrayListWithExpectedSize(partitionedTopics.size()); - addTopicPartitionedName.forEach(partitionedTopic -> futures.add( - subscribeAsync(partitionedTopic, - false /* createTopicIfDoesNotExist */))); - FutureUtil.waitForAll(futures) - .thenAccept(finalFuture -> addFuture.complete(null)) - .exceptionally(ex -> { - log.warn("[{}] Failed to subscribe to topics: {}", topic, ex.getMessage()); - addFuture.completeExceptionally(ex); - return null; + List> futures = Lists.newArrayListWithExpectedSize(addedTopics.size()); + /** + * Three normal cases: + * 1. Expand partitions. + * 2. Non-partitioned topic, but has been subscribing. + * 3. Non-partitioned topic or Partitioned topic, but has not been subscribing. + * Two unexpected cases: + * Error-1: Received adding non-partitioned topic event, but has subscribed a partitioned topic with the + * same name. + * Error-2: Received adding partitioned topic event, but has subscribed a non-partitioned topic with the + * same name. + * + * Note: The events that triggered by {@link TopicsPartitionChangedListener} after expanding partitions has + * been disabled through "conf.setAutoUpdatePartitions(false)" when creating + * {@link PatternMultiTopicsConsumerImpl}. + */ + Set groupedTopics = new HashSet<>(); + List expendPartitionsForLog = new ArrayList<>(); + for (String tp : addedTopics) { + TopicName topicName = TopicName.get(tp); + groupedTopics.add(topicName.getPartitionedTopicName()); + } + for (String tp : addedTopics) { + TopicName topicName = TopicName.get(tp); + // Case 1: Expand partitions. + if (partitionedTopics.containsKey(topicName.getPartitionedTopicName())) { + if (consumers.containsKey(topicName.toString())) { + // Already subscribed. + } else if (topicName.getPartitionIndex() < 0) { + // Error-1: Received adding non-partitioned topic event, but has subscribed a partitioned topic + // with the same name. + log.error("Pattern consumer [{}] skip to subscribe to the non-partitioned topic {}, because has" + + "subscribed a partitioned topic with the same name", + PatternMultiTopicsConsumerImpl.this.getSubscription(), topicName.toString()); + } else { + if (topicName.getPartitionIndex() + 1 + > partitionedTopics.get(topicName.getPartitionedTopicName())) { + partitionedTopics.put(topicName.getPartitionedTopicName(), + topicName.getPartitionIndex() + 1); + } + expendPartitionsForLog.add(topicName.toString()); + CompletableFuture consumerFuture = subscribeAsync(topicName.toString(), + PartitionedTopicMetadata.NON_PARTITIONED); + consumerFuture.whenComplete((__, ex) -> { + if (ex != null) { + log.warn("Pattern consumer [{}] Failed to subscribe to topics: {}", + PatternMultiTopicsConsumerImpl.this.getSubscription(), topicName, ex); + } + }); + futures.add(consumerFuture); + } + groupedTopics.remove(topicName.getPartitionedTopicName()); + } else if (consumers.containsKey(topicName.toString())) { + // Case-2: Non-partitioned topic, but has been subscribing. + groupedTopics.remove(topicName.getPartitionedTopicName()); + } else if (consumers.containsKey(topicName.getPartitionedTopicName()) + && topicName.getPartitionIndex() >= 0) { + // Error-2: Received adding partitioned topic event, but has subscribed a non-partitioned topic + // with the same name. + log.error("Pattern consumer [{}] skip to subscribe to the partitioned topic {}, because has" + + "subscribed a non-partitioned topic with the same name", + PatternMultiTopicsConsumerImpl.this.getSubscription(), topicName); + groupedTopics.remove(topicName.getPartitionedTopicName()); + } + } + // Case 3: Non-partitioned topic or Partitioned topic, which has not been subscribed. + for (String partitionedTopic : groupedTopics) { + CompletableFuture consumerFuture = subscribeAsync(partitionedTopic, false); + consumerFuture.whenComplete((__, ex) -> { + if (ex != null) { + log.warn("Pattern consumer [{}] Failed to subscribe to topics: {}", + PatternMultiTopicsConsumerImpl.this.getSubscription(), partitionedTopic, ex); + } }); - return addFuture; + futures.add(consumerFuture); + } + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] add topics. expend partitions {}, new subscribing {}", + PatternMultiTopicsConsumerImpl.this.getSubscription(), expendPartitionsForLog, groupedTopics); + } + return FutureUtil.waitForAll(futures); } } @@ -313,7 +410,7 @@ public CompletableFuture closeAsync() { closeFutures.add(watcher.closeAsync()); } } - closeFutures.add(super.closeAsync()); + closeFutures.add(updateTaskQueue.cancelAllAndWaitForTheRunningTask().thenCompose(__ -> super.closeAsync())); return FutureUtil.waitForAll(closeFutures); } @@ -322,5 +419,11 @@ Timeout getRecheckPatternTimeout() { return recheckPatternTimeout; } + protected void handleSubscribeOneTopicError(String topicName, + Throwable error, + CompletableFuture subscribeFuture) { + subscribeFuture.completeExceptionally(error); + } + private static final Logger log = LoggerFactory.getLogger(PatternMultiTopicsConsumerImpl.class); } 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 899e4e8fae7c7..e93a84e298adb 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 @@ -576,12 +576,13 @@ private CompletableFuture> patternTopicSubscribeAsync(ConsumerCo lookup.getTopicsUnderNamespace(namespaceName, subscriptionMode, regex, null) .thenAccept(getTopicsResult -> { if (log.isDebugEnabled()) { - log.debug("Get topics under namespace {}, topics.size: {}," - + " topicsHash: {}, changed: {}, filtered: {}", + log.debug("Pattern consumer [{}] get topics under namespace {}, topics.size: {}," + + " topicsHash: {}, changed: {}, filtered: {}", conf.getSubscriptionName(), namespaceName, getTopicsResult.getTopics().size(), getTopicsResult.getTopicsHash(), getTopicsResult.isChanged(), getTopicsResult.isFiltered()); getTopicsResult.getTopics().forEach(topicName -> - log.debug("Get topics under namespace {}, topic: {}", namespaceName, topicName)); + log.debug("Pattern consumer [{}] get topics under namespace {}, topic: {}", + conf.getSubscriptionName(), namespaceName, topicName)); } List topicsList = getTopicsResult.getTopics(); @@ -589,6 +590,14 @@ private CompletableFuture> patternTopicSubscribeAsync(ConsumerCo topicsList = TopicList.filterTopics(getTopicsResult.getTopics(), conf.getTopicsPattern()); } conf.getTopicNames().addAll(topicsList); + + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] initialize topics. {}", conf.getSubscriptionName(), + getTopicsResult.getNonPartitionedOrPartitionTopics()); + } + + // Pattern consumer has his unique check mechanism, so do not need the feature "autoUpdatePartitions". + conf.setAutoUpdatePartitions(false); ConsumerBase consumer = new PatternMultiTopicsConsumerImpl<>(conf.getTopicsPattern(), getTopicsResult.getTopicsHash(), PulsarClientImpl.this, diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java index 4e635e0d2e8d2..9cb737bd73d8d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java @@ -43,7 +43,7 @@ public class TopicListWatcher extends HandlerState implements ConnectionHandler. AtomicLongFieldUpdater .newUpdater(TopicListWatcher.class, "createWatcherDeadline"); - private final PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener; + private final PatternConsumerUpdateQueue patternConsumerUpdateQueue; private final String name; private final ConnectionHandler connectionHandler; private final Pattern topicsPattern; @@ -63,13 +63,13 @@ public class TopicListWatcher extends HandlerState implements ConnectionHandler. /*** * @param topicsPattern The regexp for the topic name(not contains partition suffix). */ - public TopicListWatcher(PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener, + public TopicListWatcher(PatternConsumerUpdateQueue patternConsumerUpdateQueue, PulsarClientImpl client, Pattern topicsPattern, long watcherId, NamespaceName namespace, String topicsHash, CompletableFuture watcherFuture, Runnable recheckTopicsChangeAfterReconnect) { super(client, topicsPattern.pattern()); - this.topicsChangeListener = topicsChangeListener; + this.patternConsumerUpdateQueue = patternConsumerUpdateQueue; this.name = "Watcher(" + topicsPattern + ")"; this.connectionHandler = new ConnectionHandler(this, new BackoffBuilder() @@ -277,13 +277,7 @@ private void cleanupAtClose(CompletableFuture closeFuture, Throwable excep } public void handleCommandWatchTopicUpdate(CommandWatchTopicUpdate update) { - List deleted = update.getDeletedTopicsList(); - if (!deleted.isEmpty()) { - topicsChangeListener.onTopicsRemoved(deleted); - } - List added = update.getNewTopicsList(); - if (!added.isEmpty()) { - topicsChangeListener.onTopicsAdded(added); - } + patternConsumerUpdateQueue.appendTopicsRemovedOp(update.getDeletedTopicsList()); + patternConsumerUpdateQueue.appendTopicsAddedOp(update.getNewTopicsList()); } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueueTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueueTest.java new file mode 100644 index 0000000000000..01f0be6a85ef6 --- /dev/null +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueueTest.java @@ -0,0 +1,247 @@ +/* + * 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.anyCollection; +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 io.netty.util.HashedWheelTimer; +import java.io.Closeable; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collection; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import lombok.AllArgsConstructor; +import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.common.util.FutureUtil; +import org.awaitility.Awaitility; +import org.testng.annotations.Test; + +@Test(groups = "utils") +public class PatternConsumerUpdateQueueTest { + + private QueueInstance createInstance(CompletableFuture customizedRecheckFuture, + CompletableFuture customizedPartialUpdateFuture, + CompletableFuture customizedConsumerInitFuture) { + return createInstance(customizedRecheckFuture, customizedPartialUpdateFuture, customizedConsumerInitFuture, + null, null); + } + + private QueueInstance createInstance(CompletableFuture customizedRecheckFuture, + CompletableFuture customizedPartialUpdateFuture, + CompletableFuture customizedConsumerInitFuture, + Collection successTopics, + Collection errorTopics) { + HashedWheelTimer timer = new HashedWheelTimer(new ExecutorProvider.ExtendedThreadFactory("timer-x", + Thread.currentThread().isDaemon()), 1, TimeUnit.MILLISECONDS); + PulsarClientImpl client = mock(PulsarClientImpl.class); + when(client.timer()).thenReturn(timer); + + PatternMultiTopicsConsumerImpl patternConsumer = mock(PatternMultiTopicsConsumerImpl.class); + when(patternConsumer.recheckTopicsChange()).thenReturn(customizedRecheckFuture); + when(patternConsumer.getClient()).thenReturn(client); + if (customizedConsumerInitFuture != null) { + when(patternConsumer.getSubscribeFuture()).thenReturn(customizedConsumerInitFuture); + } else { + when(patternConsumer.getSubscribeFuture()).thenReturn(CompletableFuture.completedFuture(null)); + } + + PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener = + mock(PatternMultiTopicsConsumerImpl.TopicsChangedListener.class); + if (successTopics == null && errorTopics == null) { + when(topicsChangeListener.onTopicsAdded(anyCollection())).thenReturn(customizedPartialUpdateFuture); + when(topicsChangeListener.onTopicsRemoved(anyCollection())).thenReturn(customizedPartialUpdateFuture); + } else { + CompletableFuture ex = FutureUtil.failedFuture(new RuntimeException("mock error")); + when(topicsChangeListener.onTopicsAdded(successTopics)).thenReturn(customizedPartialUpdateFuture); + when(topicsChangeListener.onTopicsRemoved(successTopics)).thenReturn(customizedPartialUpdateFuture); + when(topicsChangeListener.onTopicsAdded(errorTopics)).thenReturn(ex); + when(topicsChangeListener.onTopicsRemoved(errorTopics)).thenReturn(ex); + } + + PatternConsumerUpdateQueue queue = new PatternConsumerUpdateQueue(patternConsumer, topicsChangeListener); + return new QueueInstance(queue, patternConsumer, topicsChangeListener); + } + + private QueueInstance createInstance() { + CompletableFuture completedFuture = CompletableFuture.completedFuture(null); + return createInstance(completedFuture, completedFuture, completedFuture); + } + + @AllArgsConstructor + private static class QueueInstance implements Closeable { + private PatternConsumerUpdateQueue queue; + private PatternMultiTopicsConsumerImpl mockedConsumer; + private PatternMultiTopicsConsumerImpl.TopicsChangedListener mockedListener; + + @Override + public void close() { + mockedConsumer.getClient().timer().stop(); + } + } + + @Test + public void testTopicsChangedEvents() { + QueueInstance instance = createInstance(); + + Collection topics = Arrays.asList("a"); + for (int i = 0; i < 10; i++) { + instance.queue.appendTopicsAddedOp(topics); + instance.queue.appendTopicsRemovedOp(topics); + } + Awaitility.await().untilAsserted(() -> { + verify(instance.mockedListener, times(10)).onTopicsAdded(topics); + verify(instance.mockedListener, times(10)).onTopicsRemoved(topics); + }); + + // cleanup. + instance.close(); + } + + @Test + public void testRecheckTask() { + QueueInstance instance = createInstance(); + + for (int i = 0; i < 10; i++) { + instance.queue.appendRecheckOp(); + } + + Awaitility.await().untilAsserted(() -> { + verify(instance.mockedConsumer, times(10)).recheckTopicsChange(); + }); + + // cleanup. + instance.close(); + } + + @Test + public void testDelayedRecheckTask() { + CompletableFuture recheckFuture = new CompletableFuture<>(); + CompletableFuture partialUpdateFuture = CompletableFuture.completedFuture(null); + CompletableFuture consumerInitFuture = CompletableFuture.completedFuture(null); + QueueInstance instance = createInstance(recheckFuture, partialUpdateFuture, consumerInitFuture); + + for (int i = 0; i < 10; i++) { + instance.queue.appendRecheckOp(); + } + + recheckFuture.complete(null); + Awaitility.await().untilAsserted(() -> { + // The first task will be running, and never completed until all tasks have been added. + // Since the first was started, the second one will not be skipped. + // The others after the second task will be skipped. + // So the times that called "recheckTopicsChange" will be 2. + verify(instance.mockedConsumer, times(2)).recheckTopicsChange(); + }); + + // cleanup. + instance.close(); + } + + @Test + public void testCompositeTasks() { + CompletableFuture recheckFuture = new CompletableFuture<>(); + CompletableFuture partialUpdateFuture = CompletableFuture.completedFuture(null); + CompletableFuture consumerInitFuture = CompletableFuture.completedFuture(null); + QueueInstance instance = createInstance(recheckFuture, partialUpdateFuture, consumerInitFuture); + + Collection topics = Arrays.asList("a"); + for (int i = 0; i < 10; i++) { + instance.queue.appendRecheckOp(); + instance.queue.appendTopicsAddedOp(topics); + instance.queue.appendTopicsRemovedOp(topics); + } + recheckFuture.complete(null); + Awaitility.await().untilAsserted(() -> { + // The first task will be running, and never completed until all tasks have been added. + // Since the first was started, the second one will not be skipped. + // The others after the second task will be skipped. + // So the times that called "recheckTopicsChange" will be 2. + verify(instance.mockedConsumer, times(2)).recheckTopicsChange(); + // The tasks after the second "recheckTopicsChange" will be skipped due to there is a previous + // "recheckTopicsChange" that has not been executed. + // The tasks between the fist "recheckTopicsChange" and the second "recheckTopicsChange" will be skipped + // due to there is a following "recheckTopicsChange". + verify(instance.mockedListener, times(0)).onTopicsAdded(topics); + verify(instance.mockedListener, times(0)).onTopicsRemoved(topics); + }); + + // cleanup. + instance.close(); + } + + @Test + public void testErrorTask() { + CompletableFuture immediatelyCompleteFuture = CompletableFuture.completedFuture(null); + Collection successTopics = Arrays.asList("a"); + Collection errorTopics = Arrays.asList(UUID.randomUUID().toString()); + QueueInstance instance = createInstance(immediatelyCompleteFuture, immediatelyCompleteFuture, + immediatelyCompleteFuture, successTopics, errorTopics); + + instance.queue.appendTopicsAddedOp(successTopics); + instance.queue.appendTopicsRemovedOp(successTopics); + instance.queue.appendTopicsAddedOp(errorTopics); + instance.queue.appendTopicsAddedOp(successTopics); + instance.queue.appendTopicsRemovedOp(successTopics); + + Awaitility.await().atMost(Duration.ofSeconds(60)).untilAsserted(() -> { + verify(instance.mockedListener, times(2)).onTopicsAdded(successTopics); + verify(instance.mockedListener, times(2)).onTopicsRemoved(successTopics); + verify(instance.mockedListener, times(1)).onTopicsAdded(errorTopics); + // After an error task will push a recheck task to offset. + verify(instance.mockedConsumer, times(1)).recheckTopicsChange(); + }); + + // cleanup. + instance.close(); + } + + @Test + public void testFailedSubscribe() { + CompletableFuture immediatelyCompleteFuture = CompletableFuture.completedFuture(null); + CompletableFuture consumerInitFuture = new CompletableFuture<>(); + Collection successTopics = Arrays.asList("a"); + Collection errorTopics = Arrays.asList(UUID.randomUUID().toString()); + QueueInstance instance = createInstance(immediatelyCompleteFuture, immediatelyCompleteFuture, + consumerInitFuture, successTopics, errorTopics); + + instance.queue.appendTopicsAddedOp(successTopics); + instance.queue.appendTopicsRemovedOp(successTopics); + instance.queue.appendTopicsAddedOp(errorTopics); + instance.queue.appendTopicsAddedOp(successTopics); + instance.queue.appendTopicsRemovedOp(successTopics); + + // Consumer init failed after multi topics changes. + // All the topics changes events should be skipped. + consumerInitFuture.completeExceptionally(new RuntimeException("mocked ex")); + Awaitility.await().untilAsserted(() -> { + verify(instance.mockedListener, times(0)).onTopicsAdded(successTopics); + verify(instance.mockedListener, times(0)).onTopicsRemoved(successTopics); + verify(instance.mockedListener, times(0)).onTopicsAdded(errorTopics); + verify(instance.mockedConsumer, times(0)).recheckTopicsChange(); + }); + + // cleanup. + instance.close(); + } +} diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java index 5baca24cf8aa1..9c239785da71c 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java @@ -61,7 +61,7 @@ public void testChangedUnfilteredResponse() { "persistent://tenant/my-ns/non-matching"), null, false, true), mockListener, - Collections.emptyList()); + Collections.emptyList(), ""); verify(mockListener).onTopicsAdded(Sets.newHashSet( "persistent://tenant/my-ns/name-1", "persistent://tenant/my-ns/name-2")); @@ -80,7 +80,7 @@ public void testChangedFilteredResponse() { "persistent://tenant/my-ns/name-2"), "TOPICS_HASH", true, true), mockListener, - Arrays.asList("persistent://tenant/my-ns/name-0")); + Arrays.asList("persistent://tenant/my-ns/name-0"), ""); verify(mockListener).onTopicsAdded(Sets.newHashSet( "persistent://tenant/my-ns/name-1", "persistent://tenant/my-ns/name-2")); @@ -99,7 +99,7 @@ public void testUnchangedResponse() { "persistent://tenant/my-ns/name-2"), "TOPICS_HASH", true, false), mockListener, - Arrays.asList("persistent://tenant/my-ns/name-0")); + Arrays.asList("persistent://tenant/my-ns/name-0"), ""); verify(mockListener, never()).onTopicsAdded(any()); verify(mockListener, never()).onTopicsRemoved(any()); verify(mockTopicsHashSetter).accept("TOPICS_HASH"); diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java index 4fa4284109d2e..b5960ca3e59c7 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java @@ -28,6 +28,7 @@ import org.apache.pulsar.common.api.proto.CommandWatchTopicUpdate; import org.apache.pulsar.common.naming.NamespaceName; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyCollection; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; @@ -67,8 +68,17 @@ public void setup() { when(client.getConnection(topic, 0)).thenReturn(clientCnxFuture); when(client.getConnection(any(), any(), anyInt())).thenReturn(clientCnxFuture); when(connectionPool.getConnection(any(), any(), anyInt())).thenReturn(clientCnxFuture); + + CompletableFuture completedFuture = CompletableFuture.completedFuture(null); + PatternMultiTopicsConsumerImpl patternConsumer = mock(PatternMultiTopicsConsumerImpl.class); + when(patternConsumer.getSubscribeFuture()).thenReturn(completedFuture); + when(patternConsumer.recheckTopicsChange()).thenReturn(completedFuture); + when(listener.onTopicsAdded(anyCollection())).thenReturn(completedFuture); + when(listener.onTopicsRemoved(anyCollection())).thenReturn(completedFuture); + PatternConsumerUpdateQueue queue = new PatternConsumerUpdateQueue(patternConsumer, listener); + watcherFuture = new CompletableFuture<>(); - watcher = new TopicListWatcher(listener, client, + watcher = new TopicListWatcher(queue, client, Pattern.compile(topic), 7, NamespaceName.get("tenant/ns"), null, watcherFuture, () -> {}); } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/lookup/GetTopicsResult.java b/pulsar-common/src/main/java/org/apache/pulsar/common/lookup/GetTopicsResult.java index 80f16e6c36717..d36595d91efb0 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/lookup/GetTopicsResult.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/lookup/GetTopicsResult.java @@ -20,7 +20,10 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.HashSet; import java.util.List; +import java.util.Set; +import java.util.regex.Pattern; import lombok.Getter; import lombok.ToString; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; @@ -119,4 +122,25 @@ public List getTopics() { return topics; } } + + public GetTopicsResult filterTopics(Pattern topicsPattern) { + List topicsFiltered = TopicList.filterTopics(getTopics(), topicsPattern); + // If nothing changed. + if (topicsFiltered.equals(getTopics())) { + GetTopicsResult newObj = new GetTopicsResult(nonPartitionedOrPartitionTopics, null, true, true); + newObj.topics = topics; + return newObj; + } + // Filtered some topics. + Set topicsFilteredSet = new HashSet<>(topicsFiltered); + List newTps = new ArrayList<>(); + for (String tp: nonPartitionedOrPartitionTopics) { + if (topicsFilteredSet.contains(TopicName.get(tp).getPartitionedTopicName())) { + newTps.add(tp); + } + } + GetTopicsResult newObj = new GetTopicsResult(newTps, null, true, true); + newObj.topics = topicsFiltered; + return newObj; + } } 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 e051e01495dbe..2298a8d06a880 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 @@ -28,6 +28,7 @@ import java.util.Objects; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.regex.Pattern; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.common.util.Codec; @@ -102,6 +103,14 @@ public static boolean isValid(String topic) { } } + public static String getPartitionPattern(String topic) { + return "^" + Pattern.quote(get(topic).getPartitionedTopicName().toString()) + "-partition-[0-9]+$"; + } + + public static String getPattern(String topic) { + return "^" + Pattern.quote(get(topic).getPartitionedTopicName().toString()) + "$"; + } + @SuppressFBWarnings("DCN_NULLPOINTER_EXCEPTION") private TopicName(String completeTopicName) { try { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/FutureUtil.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/FutureUtil.java index 0628d494af3af..454eee0f966c5 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/FutureUtil.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/FutureUtil.java @@ -54,6 +54,9 @@ public class FutureUtil { * @return a new CompletableFuture that is completed when all of the given CompletableFutures complete */ public static CompletableFuture waitForAll(Collection> futures) { + if (futures == null || futures.isEmpty()) { + return CompletableFuture.completedFuture(null); + } return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])); } From 3d9d7e689ec676aad642d2abdc25d015bccdc1ac Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 10 Jul 2024 18:08:48 +0800 Subject: [PATCH 617/699] [fix][broker]Fix lookupService.getTopicsUnderNamespace can not work with a quote pattern (#23014) (cherry picked from commit 7c0e82739215fbae9e21270d4c70c9a52dd3e403) (cherry picked from commit c0029d7381e9b5a91fad35102044bb81ebffc158) --- .../impl/PatternTopicsConsumerImplTest.java | 56 ++++++++++++++++++ .../pulsar/common/topics/TopicList.java | 20 +++++-- .../pulsar/common/topics/TopicListTest.java | 58 ++++++++++++++++++- 3 files changed, 128 insertions(+), 6 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java index 1bd621738b3fb..8ba3f49208715 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java @@ -27,6 +27,8 @@ import com.google.common.collect.Lists; import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -48,6 +50,7 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.api.proto.BaseCommand; +import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; import org.apache.pulsar.common.api.proto.CommandWatchTopicListSuccess; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; @@ -1113,4 +1116,57 @@ public void testTopicDeletion() throws Exception { assertEquals(pulsar.getBrokerService().getTopicIfExists(baseTopicName + "-1").join(), Optional.empty()); assertTrue(pulsar.getBrokerService().getTopicIfExists(baseTopicName + "-2").join().isPresent()); } + + @Test(dataProvider = "partitioned") + public void testPatternQuote(boolean partitioned) throws Exception { + final NamespaceName namespace = NamespaceName.get("public/default"); + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final PulsarClientImpl client = (PulsarClientImpl) pulsarClient; + final LookupService lookup = client.getLookup(); + List expectedRes = new ArrayList<>(); + if (partitioned) { + admin.topics().createPartitionedTopic(topicName, 2); + expectedRes.add(TopicName.get(topicName).getPartition(0).toString()); + expectedRes.add(TopicName.get(topicName).getPartition(1).toString()); + Collections.sort(expectedRes); + } else { + admin.topics().createNonPartitionedTopic(topicName); + expectedRes.add(topicName); + } + + // Verify 1: "java.util.regex.Pattern.quote". + String pattern1 = java.util.regex.Pattern.quote(topicName); + List res1 = lookup.getTopicsUnderNamespace(namespace, CommandGetTopicsOfNamespace.Mode.PERSISTENT, + pattern1, null).join().getNonPartitionedOrPartitionTopics(); + Collections.sort(res1); + assertEquals(res1, expectedRes); + + // Verify 2: "com.google.re2j.Pattern.quote" + String pattern2 = com.google.re2j.Pattern.quote(topicName); + List res2 = lookup.getTopicsUnderNamespace(namespace, CommandGetTopicsOfNamespace.Mode.PERSISTENT, + pattern2, null).join().getNonPartitionedOrPartitionTopics(); + Collections.sort(res2); + assertEquals(res2, expectedRes); + + // Verify 3: "java.util.regex.Pattern.quote" & "^$" + String pattern3 = "^" + java.util.regex.Pattern.quote(topicName) + "$"; + List res3 = lookup.getTopicsUnderNamespace(namespace, CommandGetTopicsOfNamespace.Mode.PERSISTENT, + pattern3, null).join().getNonPartitionedOrPartitionTopics(); + Collections.sort(res3); + assertEquals(res3, expectedRes); + + // Verify 4: "com.google.re2j.Pattern.quote" & "^$" + String pattern4 = "^" + com.google.re2j.Pattern.quote(topicName) + "$"; + List res4 = lookup.getTopicsUnderNamespace(namespace, CommandGetTopicsOfNamespace.Mode.PERSISTENT, + pattern4, null).join().getNonPartitionedOrPartitionTopics(); + Collections.sort(res4); + assertEquals(res4, expectedRes); + + // cleanup. + if (partitioned) { + admin.topics().deletePartitionedTopic(topicName, false); + } else { + admin.topics().delete(topicName, false); + } + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/topics/TopicList.java b/pulsar-common/src/main/java/org/apache/pulsar/common/topics/TopicList.java index 4c0a8d500b703..4dd48732225e1 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/topics/TopicList.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/topics/TopicList.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.common.topics; +import com.google.common.annotations.VisibleForTesting; import com.google.common.hash.Hashing; import java.nio.charset.StandardCharsets; import java.util.Collection; @@ -28,6 +29,7 @@ import java.util.stream.Collectors; import lombok.experimental.UtilityClass; import org.apache.pulsar.common.naming.SystemTopicNames; +import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; @UtilityClass @@ -82,15 +84,23 @@ public static Set minus(Collection list1, Collection lis return s1; } - private static String removeTopicDomainScheme(String originalRegexp) { + @VisibleForTesting + static String removeTopicDomainScheme(String originalRegexp) { if (!originalRegexp.toString().contains(SCHEME_SEPARATOR)) { return originalRegexp; } - String removedTopicDomain = SCHEME_SEPARATOR_PATTERN.split(originalRegexp.toString())[1]; - if (originalRegexp.contains("^")) { - return String.format("^%s", removedTopicDomain); + String[] parts = SCHEME_SEPARATOR_PATTERN.split(originalRegexp.toString()); + String prefix = parts[0]; + String removedTopicDomain = parts[1]; + if (prefix.equals(TopicDomain.persistent.value()) || prefix.equals(TopicDomain.non_persistent.value())) { + prefix = ""; + } else if (prefix.endsWith(TopicDomain.non_persistent.value())) { + prefix = prefix.substring(0, prefix.length() - TopicDomain.non_persistent.value().length()); + } else if (prefix.endsWith(TopicDomain.persistent.value())){ + prefix = prefix.substring(0, prefix.length() - TopicDomain.persistent.value().length()); } else { - return removedTopicDomain; + throw new IllegalArgumentException("Does not support topic domain: " + prefix); } + return String.format("%s%s", prefix, removedTopicDomain); } } diff --git a/pulsar-common/src/test/java/org/apache/pulsar/common/topics/TopicListTest.java b/pulsar-common/src/test/java/org/apache/pulsar/common/topics/TopicListTest.java index 9069dd6dcc7b9..b3a7536ebff9e 100644 --- a/pulsar-common/src/test/java/org/apache/pulsar/common/topics/TopicListTest.java +++ b/pulsar-common/src/test/java/org/apache/pulsar/common/topics/TopicListTest.java @@ -30,6 +30,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; public class TopicListTest { @@ -107,5 +108,60 @@ public void testCalculateHash() { } - + @Test + public void testRemoveTopicDomainScheme() { + // persistent. + final String tpName1 = "persistent://public/default/tp"; + String res1 = TopicList.removeTopicDomainScheme(tpName1); + assertEquals(res1, "public/default/tp"); + + // non-persistent + final String tpName2 = "non-persistent://public/default/tp"; + String res2 = TopicList.removeTopicDomainScheme(tpName2); + assertEquals(res2, "public/default/tp"); + + // without topic domain. + final String tpName3 = "public/default/tp"; + String res3 = TopicList.removeTopicDomainScheme(tpName3); + assertEquals(res3, "public/default/tp"); + + // persistent & "java.util.regex.Pattern.quote". + final String tpName4 = java.util.regex.Pattern.quote(tpName1); + String res4 = TopicList.removeTopicDomainScheme(tpName4); + assertEquals(res4, java.util.regex.Pattern.quote("public/default/tp")); + + // persistent & "java.util.regex.Pattern.quote" & "^$". + final String tpName5 = "^" + java.util.regex.Pattern.quote(tpName1) + "$"; + String res5 = TopicList.removeTopicDomainScheme(tpName5); + assertEquals(res5, "^" + java.util.regex.Pattern.quote("public/default/tp") + "$"); + + // persistent & "com.google.re2j.Pattern.quote". + final String tpName6 = Pattern.quote(tpName1); + String res6 = TopicList.removeTopicDomainScheme(tpName6); + assertEquals(res6, Pattern.quote("public/default/tp")); + + // non-persistent & "java.util.regex.Pattern.quote". + final String tpName7 = java.util.regex.Pattern.quote(tpName2); + String res7 = TopicList.removeTopicDomainScheme(tpName7); + assertEquals(res7, java.util.regex.Pattern.quote("public/default/tp")); + + // non-persistent & "com.google.re2j.Pattern.quote". + final String tpName8 = Pattern.quote(tpName2); + String res8 = TopicList.removeTopicDomainScheme(tpName8); + assertEquals(res8, Pattern.quote("public/default/tp")); + + // non-persistent & "com.google.re2j.Pattern.quote" & "^$". + final String tpName9 = "^" + Pattern.quote(tpName2) + "$"; + String res9 = TopicList.removeTopicDomainScheme(tpName9); + assertEquals(res9, "^" + Pattern.quote("public/default/tp") + "$"); + + // wrong topic domain. + final String tpName10 = "xx://public/default/tp"; + try { + TopicList.removeTopicDomainScheme(tpName10); + fail("Does not support the topic domain xx"); + } catch (Exception ex) { + // expected error. + } + } } From f74d3df2447c21585f05cd08edb78fa3a53d14a1 Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Thu, 9 May 2024 13:53:02 +0800 Subject: [PATCH 618/699] [fix][broker] Fix geo-replication admin client url (#22584) (cherry picked from commit bd4c57d27c4acd37206a4f5ffdad3705cdc96c8c) (cherry picked from commit 3ccc2136ff0630ea8075dabc5ed63d8d9afc9098) --- .../pulsar/broker/service/BrokerService.java | 8 +++----- .../pulsar/broker/service/ReplicatorTestBase.java | 14 ++++++++++++-- 2 files changed, 15 insertions(+), 7 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 7ce9898bb38a5..f62173f1acea6 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 @@ -1516,13 +1516,11 @@ public PulsarAdmin getClusterPulsarAdmin(String cluster, Optional c } boolean isTlsEnabled = data.isBrokerClientTlsEnabled() || conf.isBrokerClientTlsEnabled(); - if (isTlsEnabled && StringUtils.isEmpty(data.getServiceUrlTls())) { - throw new IllegalArgumentException("serviceUrlTls is empty, brokerClientTlsEnabled: " + final String adminApiUrl = isTlsEnabled ? data.getServiceUrlTls() : data.getServiceUrl(); + if (StringUtils.isEmpty(adminApiUrl)) { + throw new IllegalArgumentException("The adminApiUrl is empty, brokerClientTlsEnabled: " + isTlsEnabled); - } else if (StringUtils.isEmpty(data.getServiceUrl())) { - throw new IllegalArgumentException("serviceUrl is empty, brokerClientTlsEnabled: " + isTlsEnabled); } - String adminApiUrl = isTlsEnabled ? data.getServiceUrlTls() : data.getServiceUrl(); builder.serviceHttpUrl(adminApiUrl); if (data.isBrokerClientTlsEnabled()) { configAdminTlsSettings(builder, diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java index 97a2f633d81a8..b3a078552d6a6 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorTestBase.java @@ -20,6 +20,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; import com.google.common.io.Resources; import com.google.common.collect.Sets; @@ -254,9 +255,7 @@ protected void setup() throws Exception { .brokerClientTlsTrustStoreType(keyStoreType) .build()); admin4.clusters().createCluster(cluster4, ClusterData.builder() - .serviceUrl(url4.toString()) .serviceUrlTls(urlTls4.toString()) - .brokerServiceUrl(pulsar4.getBrokerServiceUrl()) .brokerServiceUrlTls(pulsar4.getBrokerServiceUrlTls()) .brokerClientTlsEnabled(true) .brokerClientCertificateFilePath(clientCertFilePath) @@ -279,9 +278,20 @@ protected void setup() throws Exception { assertEquals(admin2.clusters().getCluster(cluster1).getServiceUrl(), url1.toString()); assertEquals(admin2.clusters().getCluster(cluster2).getServiceUrl(), url2.toString()); assertEquals(admin2.clusters().getCluster(cluster3).getServiceUrl(), url3.toString()); + assertNull(admin2.clusters().getCluster(cluster4).getServiceUrl()); assertEquals(admin2.clusters().getCluster(cluster1).getBrokerServiceUrl(), pulsar1.getBrokerServiceUrl()); assertEquals(admin2.clusters().getCluster(cluster2).getBrokerServiceUrl(), pulsar2.getBrokerServiceUrl()); assertEquals(admin2.clusters().getCluster(cluster3).getBrokerServiceUrl(), pulsar3.getBrokerServiceUrl()); + assertNull(admin2.clusters().getCluster(cluster4).getBrokerServiceUrl()); + + assertEquals(admin2.clusters().getCluster(cluster1).getServiceUrlTls(), urlTls1.toString()); + assertEquals(admin2.clusters().getCluster(cluster2).getServiceUrlTls(), urlTls2.toString()); + assertEquals(admin2.clusters().getCluster(cluster3).getServiceUrlTls(), urlTls3.toString()); + assertEquals(admin2.clusters().getCluster(cluster4).getServiceUrlTls(), urlTls4.toString()); + assertEquals(admin2.clusters().getCluster(cluster1).getBrokerServiceUrlTls(), pulsar1.getBrokerServiceUrlTls()); + assertEquals(admin2.clusters().getCluster(cluster2).getBrokerServiceUrlTls(), pulsar2.getBrokerServiceUrlTls()); + assertEquals(admin2.clusters().getCluster(cluster3).getBrokerServiceUrlTls(), pulsar3.getBrokerServiceUrlTls()); + assertEquals(admin2.clusters().getCluster(cluster4).getBrokerServiceUrlTls(), pulsar4.getBrokerServiceUrlTls()); // Also create V1 namespace for compatibility check admin1.clusters().createCluster("global", ClusterData.builder() From 182b7047bac664649381a461bc9793e3f31c0259 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 10 Jul 2024 10:28:47 +0800 Subject: [PATCH 619/699] [fix][admin] Fix half deletion when attempt to topic with a incorrect API (#23002) (cherry picked from commit 1f3449736e614428ea4d625e48cafa09b35e608d) (cherry picked from commit 0fab9ed7a1713e42bc506f5592a3930b26b60f1b) --- .../admin/impl/PersistentTopicsBase.java | 12 +++- .../broker/admin/v2/PersistentTopics.java | 14 ++++- .../broker/admin/AdminTopicApiTest.java | 62 +++++++++++++++++++ 3 files changed, 86 insertions(+), 2 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 a899eef63d57b..3b67808403f60 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 @@ -752,7 +752,17 @@ protected void internalDeletePartitionedTopic(AsyncResponse asyncResponse, .thenCompose(partitionedMeta -> { final int numPartitions = partitionedMeta.partitions; if (numPartitions < 1) { - return CompletableFuture.completedFuture(null); + return pulsar().getNamespaceService().checkNonPartitionedTopicExists(topicName) + .thenApply(exists -> { + if (exists) { + throw new RestException(Response.Status.CONFLICT, + String.format("%s is a non-partitioned topic. Instead of calling" + + " delete-partitioned-topic please call delete.", topicName)); + } else { + throw new RestException(Status.NOT_FOUND, + String.format("Topic %s not found.", topicName)); + } + }); } return internalRemovePartitionsAuthenticationPoliciesAsync() .thenCompose(unused -> internalRemovePartitionsTopicAsync(numPartitions, force)); 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 e619769d81ea0..1f28283737f73 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 @@ -1157,7 +1157,17 @@ public void deleteTopic( @ApiParam(value = "Whether leader broker redirected this call to this broker. For internal use.") @QueryParam("authoritative") @DefaultValue("false") boolean authoritative) { validateTopicName(tenant, namespace, encodedTopic); - internalDeleteTopicAsync(authoritative, force) + + getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .partitionedTopicExistsAsync(topicName).thenAccept(exists -> { + if (exists) { + RestException restException = new RestException(Response.Status.CONFLICT, + String.format("%s is a partitioned topic, instead of calling delete topic, please call" + + " delete-partitioned-topic.", topicName)); + resumeAsyncResponseExceptionally(asyncResponse, restException); + return; + } + internalDeleteTopicAsync(authoritative, force) .thenAccept(__ -> asyncResponse.resume(Response.noContent().build())) .exceptionally(ex -> { Throwable t = FutureUtil.unwrapCompletionException(ex); @@ -1176,6 +1186,8 @@ public void deleteTopic( resumeAsyncResponseExceptionally(asyncResponse, ex); return null; }); + }); + } @GET diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTopicApiTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTopicApiTest.java index 93bf2349103c3..45bbb3a2912f1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTopicApiTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/AdminTopicApiTest.java @@ -19,21 +19,27 @@ package org.apache.pulsar.broker.admin; import static java.nio.charset.StandardCharsets.UTF_8; +import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import java.time.Duration; import java.util.List; import java.util.UUID; import java.util.concurrent.TimeUnit; import lombok.Cleanup; +import org.apache.pulsar.broker.BrokerTestUtil; 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.PulsarClient; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.policies.data.stats.NonPersistentTopicStatsImpl; import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; +import org.awaitility.Awaitility; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; @@ -59,6 +65,62 @@ protected void cleanup() throws Exception { super.internalCleanup(); } + @Test + public void testDeleteNonExistTopic() throws Exception { + // Case 1: call delete for a partitioned topic. + final String topic1 = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createPartitionedTopic(topic1, 2); + admin.schemas().createSchemaAsync(topic1, Schema.STRING.getSchemaInfo()); + Awaitility.await().untilAsserted(() -> { + assertEquals(admin.schemas().getAllSchemas(topic1).size(), 1); + }); + try { + admin.topics().delete(topic1); + fail("expected a 409 error"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("please call delete-partitioned-topic")); + } + Awaitility.await().pollDelay(Duration.ofSeconds(2)).untilAsserted(() -> { + assertEquals(admin.schemas().getAllSchemas(topic1).size(), 1); + }); + // cleanup. + admin.topics().deletePartitionedTopic(topic1, false); + + // Case 2: call delete-partitioned-topi for a non-partitioned topic. + final String topic2 = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(topic2); + admin.schemas().createSchemaAsync(topic2, Schema.STRING.getSchemaInfo()); + Awaitility.await().untilAsserted(() -> { + assertEquals(admin.schemas().getAllSchemas(topic2).size(), 1); + }); + try { + admin.topics().deletePartitionedTopic(topic2); + fail("expected a 409 error"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("Instead of calling delete-partitioned-topic please call delete")); + } + Awaitility.await().pollDelay(Duration.ofSeconds(2)).untilAsserted(() -> { + assertEquals(admin.schemas().getAllSchemas(topic2).size(), 1); + }); + // cleanup. + admin.topics().delete(topic2, false); + + // Case 3: delete topic does not exist. + final String topic3 = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + try { + admin.topics().delete(topic3); + fail("expected a 404 error"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("not found")); + } + try { + admin.topics().deletePartitionedTopic(topic3); + fail("expected a 404 error"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("not found")); + } + } + @Test public void testPeekMessages() throws Exception { @Cleanup From f6f2db2b21a63b3345a949b8bd66131487775ff9 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 15 Jul 2024 10:04:48 +0800 Subject: [PATCH 620/699] [fix][broker] Fix stuck when enable topic level replication and build remote admin fails (#23028) (cherry picked from commit 88ebe785dbdab239104981453a9bd0e4a7e896d3) (cherry picked from commit 16da5f78e113778a80bcb26ec80d7e6eb0fcce58) --- .../pulsar/broker/admin/AdminResource.java | 15 +++++-- .../broker/service/OneWayReplicatorTest.java | 39 ++++++++++++++++++- 2 files changed, 49 insertions(+), 5 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 bd084f84e2aa0..9183630bb8116 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 @@ -48,6 +48,7 @@ import org.apache.pulsar.broker.service.plugin.InvalidEntryFilterException; import org.apache.pulsar.broker.web.PulsarWebResource; import org.apache.pulsar.broker.web.RestException; +import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.internal.TopicsImpl; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; @@ -623,7 +624,7 @@ private void internalCreatePartitionedTopicToReplicatedClustersInBackground(int }); } - protected Map> internalCreatePartitionedTopicToReplicatedClustersInBackground( + protected Map> internalCreatePartitionedTopicToReplicatedClustersInBackground ( Set clusters, int numPartitions) { final String shortTopicName = topicName.getPartitionedTopicName(); Map> tasksForAllClusters = new HashMap<>(); @@ -642,9 +643,17 @@ protected Map> internalCreatePartitionedTopicToR createRemoteTopicFuture.completeExceptionally(new RestException(ex1)); return; } + PulsarAdmin remotePulsarAdmin; + try { + remotePulsarAdmin = pulsar().getBrokerService().getClusterPulsarAdmin(cluster, clusterData); + } catch (Exception ex) { + log.error("[{}] [{}] An un-expected error occurs when trying to create remote pulsar admin for" + + " cluster {}", clientAppId(), topicName, cluster, ex); + createRemoteTopicFuture.completeExceptionally(new RestException(ex)); + return; + } // Get cluster data success. - TopicsImpl topics = - (TopicsImpl) pulsar().getBrokerService().getClusterPulsarAdmin(cluster, clusterData).topics(); + TopicsImpl topics = (TopicsImpl) remotePulsarAdmin.topics(); topics.createPartitionedTopicAsync(shortTopicName, numPartitions, true, null) .whenComplete((ignore, ex2) -> { if (ex2 == null) { 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 b7653509596a0..e5df612f5a555 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 @@ -36,7 +36,9 @@ import java.time.Duration; import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; import java.util.Optional; +import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; @@ -55,6 +57,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.resources.ClusterResources; import org.apache.pulsar.broker.service.persistent.GeoPersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -70,11 +73,13 @@ import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; +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.RetentionPolicies; +import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; -import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.util.FutureUtil; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; @@ -915,6 +920,36 @@ protected void disableReplication(String topic) throws Exception { admin1.topics().setReplicationClusters(topic, Arrays.asList(cluster1, cluster2)); } + @Test(timeOut = 30 * 1000) + public void testCreateRemoteAdminFailed() throws Exception { + final TenantInfo tenantInfo = admin1.tenants().getTenantInfo(defaultTenant); + final String ns1 = defaultTenant + "/ns_" + UUID.randomUUID().toString().replace("-", ""); + final String randomClusterName = "c_" + UUID.randomUUID().toString().replace("-", ""); + final String topic = BrokerTestUtil.newUniqueName(ns1 + "/tp"); + admin1.namespaces().createNamespace(ns1); + admin1.topics().createPartitionedTopic(topic, 2); + + // Inject a wrong cluster data which with empty fields. + ClusterResources clusterResources = broker1.getPulsar().getPulsarResources().getClusterResources(); + clusterResources.createCluster(randomClusterName, ClusterData.builder().build()); + Set allowedClusters = new HashSet<>(tenantInfo.getAllowedClusters()); + allowedClusters.add(randomClusterName); + admin1.tenants().updateTenant(defaultTenant, TenantInfo.builder().adminRoles(tenantInfo.getAdminRoles()) + .allowedClusters(allowedClusters).build()); + + // Verify. + try { + admin1.topics().setReplicationClusters(topic, Arrays.asList(cluster1, randomClusterName)); + fail("Expected a error due to empty fields"); + } catch (Exception ex) { + // Expected an error. + } + + // cleanup. + admin1.topics().deletePartitionedTopic(topic); + admin1.tenants().updateTenant(defaultTenant, tenantInfo); + } + @Test public void testConfigReplicationStartAt() throws Exception { // Initialize. From 47883750dda67becace5d46e413528d4119396e0 Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Thu, 25 Jul 2024 10:31:18 +0530 Subject: [PATCH 621/699] Debezium upgrade to 2.6.1.Final --- pom.xml | 2 +- .../pulsar/io/debezium/DebeziumSource.java | 2 +- .../io/debezium/PulsarDatabaseHistory.java | 22 ++++++------- .../debezium/PulsarDatabaseHistoryTest.java | 18 +++++------ .../containers/DebeziumMongoDbContainer.java | 2 +- .../containers/DebeziumMySQLContainer.java | 2 +- .../DebeziumPostgreSqlContainer.java | 2 +- .../integration/io/sources/SourceTester.java | 6 ++++ .../debezium/DebeziumMongoDbSourceTester.java | 31 ++++++++++++------- .../debezium/DebeziumMsSqlSourceTester.java | 13 +++++--- .../debezium/DebeziumMySqlSourceTester.java | 7 +++-- .../DebeziumOracleDbSourceTester.java | 9 +++--- .../DebeziumPostgreSqlSourceTester.java | 6 ++-- .../PulsarDebeziumOracleSourceTest.java | 2 +- .../debezium/PulsarDebeziumSourcesTest.java | 6 ++-- 15 files changed, 77 insertions(+), 53 deletions(-) diff --git a/pom.xml b/pom.xml index 1100d184c36c6..4a8f3fda3adcb 100644 --- a/pom.xml +++ b/pom.xml @@ -197,7 +197,7 @@ flexible messaging model and an intuitive client API. 1.2.4 8.12.1 368 - 1.9.7.Final + 2.6.1.Final 42.5.5 8.0.30 diff --git a/pulsar-io/debezium/core/src/main/java/org/apache/pulsar/io/debezium/DebeziumSource.java b/pulsar-io/debezium/core/src/main/java/org/apache/pulsar/io/debezium/DebeziumSource.java index 6c422c4f036a9..749e5db6934c9 100644 --- a/pulsar-io/debezium/core/src/main/java/org/apache/pulsar/io/debezium/DebeziumSource.java +++ b/pulsar-io/debezium/core/src/main/java/org/apache/pulsar/io/debezium/DebeziumSource.java @@ -88,7 +88,7 @@ public void open(Map config, SourceContext sourceContext) throws setConfigIfNull(config, PulsarKafkaWorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, DEFAULT_CONVERTER); // database.history : implementation class for database history. - setConfigIfNull(config, HistorizedRelationalDatabaseConnectorConfig.DATABASE_HISTORY.name(), DEFAULT_HISTORY); + setConfigIfNull(config, HistorizedRelationalDatabaseConnectorConfig.SCHEMA_HISTORY.name(), DEFAULT_HISTORY); // database.history.pulsar.service.url String pulsarUrl = (String) config.get(PulsarDatabaseHistory.SERVICE_URL.name()); diff --git a/pulsar-io/debezium/core/src/main/java/org/apache/pulsar/io/debezium/PulsarDatabaseHistory.java b/pulsar-io/debezium/core/src/main/java/org/apache/pulsar/io/debezium/PulsarDatabaseHistory.java index 7ca0d309cf973..c33c85d399d20 100644 --- a/pulsar-io/debezium/core/src/main/java/org/apache/pulsar/io/debezium/PulsarDatabaseHistory.java +++ b/pulsar-io/debezium/core/src/main/java/org/apache/pulsar/io/debezium/PulsarDatabaseHistory.java @@ -26,12 +26,12 @@ import io.debezium.config.Configuration; import io.debezium.config.Field; import io.debezium.document.DocumentReader; -import io.debezium.relational.history.AbstractDatabaseHistory; -import io.debezium.relational.history.DatabaseHistory; -import io.debezium.relational.history.DatabaseHistoryException; -import io.debezium.relational.history.DatabaseHistoryListener; +import io.debezium.relational.history.AbstractSchemaHistory; import io.debezium.relational.history.HistoryRecord; import io.debezium.relational.history.HistoryRecordComparator; +import io.debezium.relational.history.SchemaHistory; +import io.debezium.relational.history.SchemaHistoryException; +import io.debezium.relational.history.SchemaHistoryListener; import java.io.IOException; import java.util.Collections; import java.util.HashMap; @@ -52,12 +52,12 @@ import org.apache.pulsar.client.api.Schema; /** - * A {@link DatabaseHistory} implementation that records schema changes as normal pulsar messages on the specified + * A {@link SchemaHistory} implementation that records schema changes as normal pulsar messages on the specified * topic, and that recovers the history by establishing a Kafka Consumer re-processing all messages on that topic. */ @Slf4j @ThreadSafe -public final class PulsarDatabaseHistory extends AbstractDatabaseHistory { +public final class PulsarDatabaseHistory extends AbstractSchemaHistory { public static final Field TOPIC = Field.create(CONFIGURATION_FIELD_PREFIX_STRING + "pulsar.topic") .withDisplayName("Database history topic name") @@ -97,7 +97,7 @@ public final class PulsarDatabaseHistory extends AbstractDatabaseHistory { TOPIC, SERVICE_URL, CLIENT_BUILDER, - DatabaseHistory.NAME, + SchemaHistory.NAME, READER_CONFIG); private final ObjectMapper mapper = new ObjectMapper(); @@ -113,7 +113,7 @@ public final class PulsarDatabaseHistory extends AbstractDatabaseHistory { public void configure( Configuration config, HistoryRecordComparator comparator, - DatabaseHistoryListener listener, + SchemaHistoryListener listener, boolean useCatalogBeforeSchema) { super.configure(config, comparator, listener, useCatalogBeforeSchema); if (!config.validateAndRecord(ALL_FIELDS, logger::error)) { @@ -148,7 +148,7 @@ public void configure( } // Copy the relevant portions of the configuration and add useful defaults ... - this.dbHistoryName = config.getString(DatabaseHistory.NAME, UUID.randomUUID().toString()); + this.dbHistoryName = config.getString(SchemaHistory.NAME, UUID.randomUUID().toString()); log.info("Configure to store the debezium database history {} to pulsar topic {}", dbHistoryName, topicName); @@ -201,7 +201,7 @@ public void start() { } @Override - protected void storeRecord(HistoryRecord record) throws DatabaseHistoryException { + protected void storeRecord(HistoryRecord record) throws SchemaHistoryException { if (this.producer == null) { throw new IllegalStateException("No producer is available. Ensure that 'start()'" + " is called before storing database history records."); @@ -212,7 +212,7 @@ protected void storeRecord(HistoryRecord record) throws DatabaseHistoryException try { producer.send(record.toString()); } catch (PulsarClientException e) { - throw new DatabaseHistoryException(e); + throw new SchemaHistoryException(e); } } diff --git a/pulsar-io/debezium/core/src/test/java/org/apache/pulsar/io/debezium/PulsarDatabaseHistoryTest.java b/pulsar-io/debezium/core/src/test/java/org/apache/pulsar/io/debezium/PulsarDatabaseHistoryTest.java index 1c5863e557e9a..a709676ddb1b7 100644 --- a/pulsar-io/debezium/core/src/test/java/org/apache/pulsar/io/debezium/PulsarDatabaseHistoryTest.java +++ b/pulsar-io/debezium/core/src/test/java/org/apache/pulsar/io/debezium/PulsarDatabaseHistoryTest.java @@ -27,8 +27,8 @@ import io.debezium.connector.mysql.antlr.MySqlAntlrDdlParser; import io.debezium.relational.Tables; import io.debezium.relational.ddl.DdlParser; -import io.debezium.relational.history.DatabaseHistory; -import io.debezium.relational.history.DatabaseHistoryListener; +import io.debezium.relational.history.SchemaHistory; +import io.debezium.relational.history.SchemaHistoryListener; import io.debezium.text.ParsingException; import io.debezium.util.Collect; @@ -80,8 +80,8 @@ protected void cleanup() throws Exception { private void testHistoryTopicContent(boolean skipUnparseableDDL, boolean testWithClientBuilder, boolean testWithReaderConfig) throws Exception { Configuration.Builder configBuidler = Configuration.create() .with(PulsarDatabaseHistory.TOPIC, topicName) - .with(DatabaseHistory.NAME, "my-db-history") - .with(DatabaseHistory.SKIP_UNPARSEABLE_DDL_STATEMENTS, skipUnparseableDDL); + .with(SchemaHistory.NAME, "my-db-history") + .with(SchemaHistory.SKIP_UNPARSEABLE_DDL_STATEMENTS, skipUnparseableDDL); if (testWithClientBuilder) { ClientBuilder builder = PulsarClient.builder().serviceUrl(brokerUrl.toString()); @@ -101,7 +101,7 @@ private void testHistoryTopicContent(boolean skipUnparseableDDL, boolean testWit } // Start up the history ... - history.configure(configBuidler.build(), null, DatabaseHistoryListener.NOOP, true); + history.configure(configBuidler.build(), null, SchemaHistoryListener.NOOP, true); history.start(); // Should be able to call start more than once ... @@ -160,7 +160,7 @@ private void testHistoryTopicContent(boolean skipUnparseableDDL, boolean testWit // Stop the history (which should stop the producer) ... history.stop(); history = new PulsarDatabaseHistory(); - history.configure(configBuidler.build(), null, DatabaseHistoryListener.NOOP, true); + history.configure(configBuidler.build(), null, SchemaHistoryListener.NOOP, true); // no need to start // Recover from the very beginning to just past the first change ... @@ -240,11 +240,11 @@ public void testExists() throws Exception { Configuration config = Configuration.create() .with(PulsarDatabaseHistory.SERVICE_URL, brokerUrl.toString()) .with(PulsarDatabaseHistory.TOPIC, "persistent://my-property/my-ns/dummytopic") - .with(DatabaseHistory.NAME, "my-db-history") - .with(DatabaseHistory.SKIP_UNPARSEABLE_DDL_STATEMENTS, true) + .with(SchemaHistory.NAME, "my-db-history") + .with(SchemaHistory.SKIP_UNPARSEABLE_DDL_STATEMENTS, true) .build(); - history.configure(config, null, DatabaseHistoryListener.NOOP, true); + history.configure(config, null, SchemaHistoryListener.NOOP, true); history.start(); // dummytopic should not exist yet diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/DebeziumMongoDbContainer.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/DebeziumMongoDbContainer.java index 481725d145b2a..bf6b8cd6462e2 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/DebeziumMongoDbContainer.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/containers/DebeziumMongoDbContainer.java @@ -25,7 +25,7 @@ public class DebeziumMongoDbContainer extends ChaosContainer i protected int numEntriesToInsert = 1; protected int numEntriesExpectAfterStart = 9; + /* + *In Debezium 2.5, they introduced several new timestamp fields, + * ts_us, and ts_ns, which represent the millisecond-based time values in microseconds and nanoseconds respectively. + */ public static final Set DEBEZIUM_FIELD_SET = new HashSet() {{ add("before"); add("after"); add("source"); add("op"); add("ts_ms"); + add("ts_us"); + add("ts_ns"); add("transaction"); }}; diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumMongoDbSourceTester.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumMongoDbSourceTester.java index 8c4a7a91a15fa..74014a3675157 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumMongoDbSourceTester.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumMongoDbSourceTester.java @@ -42,15 +42,21 @@ public DebeziumMongoDbSourceTester(PulsarCluster cluster) { this.pulsarCluster = cluster; pulsarServiceUrl = "pulsar://pulsar-proxy:" + PulsarContainer.BROKER_PORT; - sourceConfig.put("mongodb.hosts", "rs0/" + DebeziumMongoDbContainer.NAME + ":27017"); + /* + *The `mongodb.connection.string` property replaces the deprecated `mongodb.hosts` property in release 2.2 + * that was used to provide earlier versions of the connector with the host address of the configuration server replica. + * In the current release, use mongodb.connection.string to provide the connector with the addresses of MongoDB routers, + * also known as mongos. + */ + sourceConfig.put("connector.class", "io.debezium.connector.mongodb.MongoDbConnector"); + sourceConfig.put("mongodb.connection.string", "mongodb://" + DebeziumMongoDbContainer.NAME + ":27017/?replicaSet=rs0"); sourceConfig.put("mongodb.name", "dbserver1"); sourceConfig.put("mongodb.user", "debezium"); sourceConfig.put("mongodb.password", "dbz"); - sourceConfig.put("mongodb.task.id","1"); - sourceConfig.put("database.include.list", "inventory"); - sourceConfig.put("database.history.pulsar.service.url", pulsarServiceUrl); + sourceConfig.put("mongodb.task.id", "1"); + sourceConfig.put("schema.history.internal.pulsar.service.url", pulsarServiceUrl); sourceConfig.put("topic.namespace", "debezium/mongodb"); - sourceConfig.put("capture.mode", "oplog"); + sourceConfig.put("topic.prefix", "dbserver1"); } @Override @@ -66,13 +72,16 @@ public void prepareSource() throws Exception { log.info("debezium mongodb server already contains preconfigured data."); } + /* + * mongo is deprecated in 2.6.1.Final release and now we have use mongosh instead + */ @Override public void prepareInsertEvent() throws Exception { this.debeziumMongoDbContainer.execCmd("/bin/bash", "-c", - "mongo -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory " + + "mongosh -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory " + "--eval 'db.products.find()'"); this.debeziumMongoDbContainer.execCmd("/bin/bash", "-c", - "mongo -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory " + + "mongosh -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory " + "--eval 'db.products.insert({ " + "_id : NumberLong(\"110\")," + "name : \"test-debezium\"," + @@ -84,20 +93,20 @@ public void prepareInsertEvent() throws Exception { @Override public void prepareDeleteEvent() throws Exception { this.debeziumMongoDbContainer.execCmd("/bin/bash", "-c", - "mongo -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory " + + "mongosh -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory " + "--eval 'db.products.find()'"); this.debeziumMongoDbContainer.execCmd("/bin/bash", "-c", - "mongo -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory " + + "mongosh -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory " + "--eval 'db.products.deleteOne({name : \"test-debezium-update\"})'"); } @Override public void prepareUpdateEvent() throws Exception { this.debeziumMongoDbContainer.execCmd("/bin/bash", "-c", - "mongo -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory " + + "mongosh -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory " + "--eval 'db.products.find()'"); this.debeziumMongoDbContainer.execCmd("/bin/bash", "-c", - "mongo -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory " + + "mongosh -u debezium -p dbz --authenticationDatabase admin localhost:27017/inventory " + "--eval 'db.products.update({" + "_id : 110}," + "{$set:{name:\"test-debezium-update\", description: \"this is update description\"}})'"); diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumMsSqlSourceTester.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumMsSqlSourceTester.java index a745cae60409d..45ec00ef66817 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumMsSqlSourceTester.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumMsSqlSourceTester.java @@ -54,15 +54,18 @@ public DebeziumMsSqlSourceTester(PulsarCluster cluster) { pulsarServiceUrl = "pulsar://pulsar-proxy:" + PulsarContainer.BROKER_PORT; + sourceConfig.put("connector.class", "io.debezium.connector.sqlserver.SqlServerConnector"); sourceConfig.put("database.hostname", DebeziumMsSqlContainer.NAME); sourceConfig.put("database.port", "1433"); sourceConfig.put("database.user", "sa"); sourceConfig.put("database.password", DebeziumMsSqlContainer.SA_PASSWORD); sourceConfig.put("database.server.name", "mssql"); - sourceConfig.put("database.dbname", "TestDB"); - sourceConfig.put("snapshot.mode", "schema_only"); - sourceConfig.put("database.history.pulsar.service.url", pulsarServiceUrl); + sourceConfig.put("database.names", "TestDB"); + sourceConfig.put("schema.history.internal.pulsar.service.url", pulsarServiceUrl); sourceConfig.put("topic.namespace", "debezium/mssql"); + sourceConfig.put("topic.prefix", "mssql"); + sourceConfig.put("database.encrypt", "false"); + sourceConfig.put("task.id", "1"); } @Override @@ -145,12 +148,12 @@ public int initialDelayForMsgReceive() { @Override public String keyContains() { - return "mssql.dbo.customers.Key"; + return "mssql.TestDB.dbo.customers.Key"; } @Override public String valueContains() { - return "mssql.dbo.customers.Value"; + return "mssql.TestDB.dbo.customers.Value"; } @Override diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumMySqlSourceTester.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumMySqlSourceTester.java index 2f2bb8a2c0c25..7bbbf7f7ef39e 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumMySqlSourceTester.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumMySqlSourceTester.java @@ -32,7 +32,7 @@ * It reads binlog from MySQL, and store the debezium output into Pulsar. * This test verify that the target topic contains wanted number messages. * - * Debezium MySQL Container is "debezium/example-mysql:0.8", + * Debezium MySQL Container is "debezium/example-mysql:2.5.0.Final", * which is a MySQL database server preconfigured with an inventory database. */ @Slf4j @@ -53,6 +53,7 @@ public DebeziumMySqlSourceTester(PulsarCluster cluster, String converterClassNam this.pulsarCluster = cluster; pulsarServiceUrl = "pulsar://pulsar-proxy:" + PulsarContainer.BROKER_PORT; + sourceConfig.put("connector.class", "io.debezium.connector.mysql.MySqlConnector"); sourceConfig.put("database.hostname", DebeziumMySQLContainer.NAME); sourceConfig.put("database.port", "3306"); sourceConfig.put("database.user", "debezium"); @@ -60,8 +61,10 @@ public DebeziumMySqlSourceTester(PulsarCluster cluster, String converterClassNam sourceConfig.put("database.server.id", "184054"); sourceConfig.put("database.server.name", "dbserver1"); sourceConfig.put("database.whitelist", "inventory"); + sourceConfig.put("database.include.list", "inventory"); + sourceConfig.put("topic.prefix", "dbserver1"); if (!testWithClientBuilder) { - sourceConfig.put("database.history.pulsar.service.url", pulsarServiceUrl); + sourceConfig.put("schema.history.internal.pulsar.service.url", pulsarServiceUrl); } sourceConfig.put("key.converter", converterClassName); sourceConfig.put("value.converter", converterClassName); diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumOracleDbSourceTester.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumOracleDbSourceTester.java index 17eaf319b3be2..d2fb5b0654f11 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumOracleDbSourceTester.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumOracleDbSourceTester.java @@ -50,21 +50,22 @@ public DebeziumOracleDbSourceTester(PulsarCluster cluster) { super(NAME); this.pulsarCluster = cluster; this.numEntriesToInsert = 1; - this.numEntriesExpectAfterStart = 0; + this.numEntriesExpectAfterStart = 1; pulsarServiceUrl = "pulsar://pulsar-proxy:" + PulsarContainer.BROKER_PORT; + sourceConfig.put("connector.class", "io.debezium.connector.oracle.OracleConnector"); sourceConfig.put("database.hostname", DebeziumOracleDbContainer.NAME); sourceConfig.put("database.port", "1521"); sourceConfig.put("database.user", "dbzuser"); sourceConfig.put("database.password", "dbz"); sourceConfig.put("database.server.name", "XE"); sourceConfig.put("database.dbname", "XE"); - sourceConfig.put("snapshot.mode", "schema_only"); - + sourceConfig.put("database.whitelist", "inv"); sourceConfig.put("schema.include.list", "inv"); - sourceConfig.put("database.history.pulsar.service.url", pulsarServiceUrl); + sourceConfig.put("schema.history.internal.pulsar.service.url", pulsarServiceUrl); sourceConfig.put("topic.namespace", "debezium/oracle"); + sourceConfig.put("topic.prefix", "XE"); } @Override diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumPostgreSqlSourceTester.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumPostgreSqlSourceTester.java index 5f82cc52e7c40..881570b100dae 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumPostgreSqlSourceTester.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumPostgreSqlSourceTester.java @@ -35,7 +35,7 @@ * It reads binlog from Postgres, and store the debezium output into Pulsar. * This test verify that the target topic contains wanted number messages. * - * Debezium Postgresql Container is "debezium/example-postgres:0.10", + * Debezium Postgresql Container is "debezium/example-postgres:2.5.0.Final", * which is a Postgresql database server preconfigured with an inventory database. */ @Slf4j @@ -65,6 +65,7 @@ public DebeziumPostgreSqlSourceTester(PulsarCluster cluster) { pulsarServiceUrl = "pulsar://pulsar-proxy:" + PulsarContainer.BROKER_PORT; + sourceConfig.put("connector.class", "io.debezium.connector.postgresql.PostgresConnector"); sourceConfig.put("database.hostname", DebeziumPostgreSqlContainer.NAME); sourceConfig.put("database.port", "5432"); sourceConfig.put("database.user", "postgres"); @@ -74,8 +75,9 @@ public DebeziumPostgreSqlSourceTester(PulsarCluster cluster) { sourceConfig.put("database.dbname", "postgres"); sourceConfig.put("schema.whitelist", "inventory"); sourceConfig.put("table.blacklist", "inventory.spatial_ref_sys,inventory.geom"); - sourceConfig.put("database.history.pulsar.service.url", pulsarServiceUrl); + sourceConfig.put("schema.history.internal.pulsar.service.url", pulsarServiceUrl); sourceConfig.put("topic.namespace", "debezium/postgresql"); + sourceConfig.put("topic.prefix", "dbserver1"); } @Override diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/PulsarDebeziumOracleSourceTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/PulsarDebeziumOracleSourceTest.java index 7fe5b4fb48e26..35f4c92cdbcb8 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/PulsarDebeziumOracleSourceTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/PulsarDebeziumOracleSourceTest.java @@ -56,7 +56,7 @@ private void testDebeziumOracleDbConnect(String converterClassName, boolean json final String sourceName = "test-source-debezium-oracle-" + functionRuntimeType + "-" + randomName(8); // This is the event count to be created by prepareSource. - final int numMessages = 1; + final int numMessages = 39; @Cleanup PulsarClient client = PulsarClient.builder() diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/PulsarDebeziumSourcesTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/PulsarDebeziumSourcesTest.java index 5c57c904fc77f..fdb9fe7b7baea 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/PulsarDebeziumSourcesTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/PulsarDebeziumSourcesTest.java @@ -93,7 +93,7 @@ private void testDebeziumMySqlConnect(String converterClassName, boolean jsonWit + "-" + functionRuntimeType + "-" + randomName(8); // This is the binlog count that contained in mysql container. - final int numMessages = 47; + final int numMessages = 52; @Cleanup PulsarClient client = PulsarClient.builder() @@ -138,7 +138,7 @@ private void testDebeziumPostgreSqlConnect(String converterClassName, boolean js final String sourceName = "test-source-debezium-postgersql-" + functionRuntimeType + "-" + randomName(8); // This is the binlog count that contained in postgresql container. - final int numMessages = 26; + final int numMessages = 29; @Cleanup PulsarClient client = PulsarClient.builder() @@ -211,7 +211,7 @@ private void testDebeziumMsSqlConnect(String converterClassName, boolean jsonWit final String tenant = TopicName.PUBLIC_TENANT; final String namespace = TopicName.DEFAULT_NAMESPACE; final String outputTopicName = "debe-output-topic-name-" + testId.getAndIncrement(); - final String consumeTopicName = "debezium/mssql/mssql.dbo.customers"; + final String consumeTopicName = "debezium/mssql/mssql.TestDB.dbo.customers"; final String sourceName = "test-source-debezium-mssql-" + functionRuntimeType + "-" + randomName(8); final int numMessages = 1; From 4c5870112d48aa76bf99b5eca7ede69edc45558f Mon Sep 17 00:00:00 2001 From: mukesh-ctds Date: Thu, 25 Jul 2024 15:02:57 +0530 Subject: [PATCH 622/699] Reverted changes for oracle --- .../io/sources/debezium/DebeziumOracleDbSourceTester.java | 4 ++-- .../io/sources/debezium/PulsarDebeziumOracleSourceTest.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumOracleDbSourceTester.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumOracleDbSourceTester.java index d2fb5b0654f11..d9325b848d050 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumOracleDbSourceTester.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/DebeziumOracleDbSourceTester.java @@ -50,7 +50,7 @@ public DebeziumOracleDbSourceTester(PulsarCluster cluster) { super(NAME); this.pulsarCluster = cluster; this.numEntriesToInsert = 1; - this.numEntriesExpectAfterStart = 1; + this.numEntriesExpectAfterStart = 0; pulsarServiceUrl = "pulsar://pulsar-proxy:" + PulsarContainer.BROKER_PORT; @@ -61,7 +61,7 @@ public DebeziumOracleDbSourceTester(PulsarCluster cluster) { sourceConfig.put("database.password", "dbz"); sourceConfig.put("database.server.name", "XE"); sourceConfig.put("database.dbname", "XE"); - sourceConfig.put("database.whitelist", "inv"); + sourceConfig.put("snapshot.mode", "schema_only"); sourceConfig.put("schema.include.list", "inv"); sourceConfig.put("schema.history.internal.pulsar.service.url", pulsarServiceUrl); sourceConfig.put("topic.namespace", "debezium/oracle"); diff --git a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/PulsarDebeziumOracleSourceTest.java b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/PulsarDebeziumOracleSourceTest.java index 35f4c92cdbcb8..7fe5b4fb48e26 100644 --- a/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/PulsarDebeziumOracleSourceTest.java +++ b/tests/integration/src/test/java/org/apache/pulsar/tests/integration/io/sources/debezium/PulsarDebeziumOracleSourceTest.java @@ -56,7 +56,7 @@ private void testDebeziumOracleDbConnect(String converterClassName, boolean json final String sourceName = "test-source-debezium-oracle-" + functionRuntimeType + "-" + randomName(8); // This is the event count to be created by prepareSource. - final int numMessages = 39; + final int numMessages = 1; @Cleanup PulsarClient client = PulsarClient.builder() From 326271a8a1e24e4bc868cc45df0d675d1e2736fe Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 15 Jul 2024 23:01:47 +0800 Subject: [PATCH 623/699] [fix][broker] Replication stuck when partitions count between two clusters is not the same (#22983) (cherry picked from commit a8ce990a72c3024fafe689f3bc3c5127583021e6) (cherry picked from commit 25542d8822d72bffd85eb35b58b586bb55efd147) --- .../broker/service/AbstractReplicator.java | 5 + .../persistent/PersistentReplicator.java | 6 + .../service/AbstractReplicatorTest.java | 7 +- .../broker/service/OneWayReplicatorTest.java | 93 +++++++++++++- .../OneWayReplicatorUsingGlobalZKTest.java | 6 + .../api/NonPartitionedTopicExpectedTest.java | 118 ++++++++++++++++++ .../pulsar/client/impl/PulsarClientImpl.java | 47 +++++-- .../impl/conf/ProducerConfigurationData.java | 2 + 8 files changed, 273 insertions(+), 11 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPartitionedTopicExpectedTest.java diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java index 869a4bc81d310..d7769d233bf60 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java @@ -36,6 +36,7 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.ProducerBuilderImpl; import org.apache.pulsar.client.impl.ProducerImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.naming.TopicName; @@ -166,6 +167,10 @@ public void startProducer() { } log.info("[{}] Starting replicator", replicatorId); + // Force only replicate messages to a non-partitioned topic, to avoid auto-create a partitioned topic on + // the remote cluster. + ProducerBuilderImpl builderImpl = (ProducerBuilderImpl) producerBuilder; + builderImpl.getConf().setNonPartitionedTopicExpected(true); producerBuilder.createAsync().thenAccept(producer -> { setProducerAndTriggerReadEntries(producer); }).exceptionally(ex -> { 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 2aa8e9b748c32..4d74aa83c40d0 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 @@ -150,6 +150,12 @@ protected void setProducerAndTriggerReadEntries(Producer producer) { Pair changeStateRes; changeStateRes = compareSetAndGetState(Starting, Started); if (changeStateRes.getLeft()) { + 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); + doCloseProducerAsync(producer, () -> {}); + throw new ClassCastException(producer.getClass().getName() + " can not be cast to ProducerImpl"); + } this.producer = (ProducerImpl) producer; HAVE_PENDING_READ_UPDATER.set(this, FALSE); // Trigger a new read. diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java index fd2bbdff161da..3fac36e4d008f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/AbstractReplicatorTest.java @@ -38,10 +38,11 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.client.api.Producer; -import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.ConnectionPool; +import org.apache.pulsar.client.impl.ProducerBuilderImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; +import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; import org.apache.pulsar.common.policies.data.stats.ReplicatorStatsImpl; import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; @@ -69,7 +70,8 @@ public void testRetryStartProducerStoppedByTopicRemove() throws Exception { when(localClient.getCnxPool()).thenReturn(connectionPool); final PulsarClientImpl remoteClient = mock(PulsarClientImpl.class); when(remoteClient.getCnxPool()).thenReturn(connectionPool); - final ProducerBuilder producerBuilder = mock(ProducerBuilder.class); + final ProducerConfigurationData producerConf = new ProducerConfigurationData(); + final ProducerBuilderImpl producerBuilder = mock(ProducerBuilderImpl.class); final ConcurrentOpenHashMap>> topics = new ConcurrentOpenHashMap<>(); when(broker.executor()).thenReturn(eventLoopGroup); when(broker.getTopics()).thenReturn(topics); @@ -85,6 +87,7 @@ public void testRetryStartProducerStoppedByTopicRemove() throws Exception { when(producerBuilder.sendTimeout(anyInt(), any())).thenReturn(producerBuilder); when(producerBuilder.maxPendingMessages(anyInt())).thenReturn(producerBuilder); when(producerBuilder.producerName(anyString())).thenReturn(producerBuilder); + when(producerBuilder.getConf()).thenReturn(producerConf); // Mock create producer fail. when(producerBuilder.create()).thenThrow(new RuntimeException("mocked ex")); when(producerBuilder.createAsync()) 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 e5df612f5a555..90a67ead4d0f5 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 @@ -37,6 +37,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashSet; +import java.util.List; import java.util.Optional; import java.util.Set; import java.util.UUID; @@ -47,7 +48,9 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiFunction; +import java.util.function.Predicate; import java.util.function.Supplier; +import java.util.stream.Collectors; import lombok.AllArgsConstructor; import lombok.Data; import lombok.SneakyThrows; @@ -76,11 +79,13 @@ 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.AutoTopicCreationOverride; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TopicStats; -import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; +import org.apache.pulsar.common.policies.data.impl.AutoTopicCreationOverrideImpl; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; import org.mockito.Mockito; @@ -1037,4 +1042,90 @@ public void testConfigReplicationStartAt() throws Exception { admin1.topics().delete(topic3, false); admin2.topics().delete(topic3, false); } + + @DataProvider(name = "replicationModes") + public Object[][] replicationModes() { + return new Object[][]{ + {ReplicationMode.OneWay}, + {ReplicationMode.DoubleWay} + }; + } + + protected enum ReplicationMode { + OneWay, + DoubleWay; + } + + @Test(dataProvider = "replicationModes") + public void testDifferentTopicCreationRule(ReplicationMode replicationMode) throws Exception { + String ns = defaultTenant + "/" + UUID.randomUUID().toString().replace("-", ""); + admin1.namespaces().createNamespace(ns); + admin2.namespaces().createNamespace(ns); + + // Set topic auto-creation rule. + // c1: no-partitioned topic + // c2: partitioned topic with 2 partitions. + AutoTopicCreationOverride autoTopicCreation = + AutoTopicCreationOverrideImpl.builder().allowAutoTopicCreation(true) + .topicType("partitioned").defaultNumPartitions(2).build(); + admin2.namespaces().setAutoTopicCreation(ns, autoTopicCreation); + Awaitility.await().untilAsserted(() -> { + assertEquals(admin2.namespaces().getAutoTopicCreationAsync(ns).join().getDefaultNumPartitions(), 2); + // Trigger system topic __change_event's initialize. + pulsar2.getTopicPoliciesService().getTopicPoliciesAsync(TopicName.get("persistent://" + ns + "/1")); + }); + + // Create non-partitioned topic. + // Enable replication. + final String tp = BrokerTestUtil.newUniqueName("persistent://" + ns + "/tp_"); + admin1.topics().createNonPartitionedTopic(tp); + admin1.namespaces().setNamespaceReplicationClusters(ns, new HashSet<>(Arrays.asList(cluster1, cluster2))); + if (replicationMode.equals(ReplicationMode.DoubleWay)) { + admin2.namespaces().setNamespaceReplicationClusters(ns, new HashSet<>(Arrays.asList(cluster1, cluster2))); + } + + // Trigger and wait for replicator starts. + Producer p1 = client1.newProducer(Schema.STRING).topic(tp).create(); + p1.send("msg-1"); + p1.close(); + Awaitility.await().untilAsserted(() -> { + PersistentTopic persistentTopic = (PersistentTopic) broker1.getTopic(tp, false).join().get(); + assertFalse(persistentTopic.getReplicators().isEmpty()); + }); + + // Verify: the topics are the same between two clusters. + Predicate topicNameFilter = t -> { + TopicName topicName = TopicName.get(t); + if (!topicName.getNamespace().equals(ns)) { + return false; + } + return t.startsWith(tp); + }; + Awaitility.await().untilAsserted(() -> { + List topics1 = pulsar1.getBrokerService().getTopics().keys() + .stream().filter(topicNameFilter).collect(Collectors.toList()); + List topics2 = pulsar2.getBrokerService().getTopics().keys() + .stream().filter(topicNameFilter).collect(Collectors.toList()); + Collections.sort(topics1); + Collections.sort(topics2); + assertEquals(topics1, topics2); + }); + + // cleanup. + admin1.namespaces().setNamespaceReplicationClusters(ns, new HashSet<>(Arrays.asList(cluster1))); + if (replicationMode.equals(ReplicationMode.DoubleWay)) { + admin2.namespaces().setNamespaceReplicationClusters(ns, new HashSet<>(Arrays.asList(cluster2))); + } + Awaitility.await().untilAsserted(() -> { + PersistentTopic persistentTopic = (PersistentTopic) broker1.getTopic(tp, false).join().get(); + assertTrue(persistentTopic.getReplicators().isEmpty()); + if (replicationMode.equals(ReplicationMode.DoubleWay)) { + assertTrue(persistentTopic.getReplicators().isEmpty()); + } + }); + admin1.topics().delete(tp, false); + admin2.topics().delete(tp, false); + admin1.namespaces().deleteNamespace(ns); + admin2.namespaces().deleteNamespace(ns); + } } 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 31e94f435f0f6..34810bbe9057b 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 @@ -161,4 +161,10 @@ public void testConfigReplicationStartAt() throws Exception { pulsar1.getConfiguration().getReplicationStartAt().equalsIgnoreCase("latest"); }); } + + @Test(enabled = false) + @Override + public void testDifferentTopicCreationRule(ReplicationMode replicationMode) throws Exception { + super.testDifferentTopicCreationRule(replicationMode); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPartitionedTopicExpectedTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPartitionedTopicExpectedTest.java new file mode 100644 index 0000000000000..7b0edd314d055 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonPartitionedTopicExpectedTest.java @@ -0,0 +1,118 @@ +/* + * 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 lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.client.impl.ProducerBuilderImpl; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.AutoTopicCreationOverride; +import org.apache.pulsar.common.policies.data.TopicType; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Slf4j +public class NonPartitionedTopicExpectedTest extends ProducerConsumerBase { + + @BeforeClass + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Test + public void testWhenNonPartitionedTopicExists() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(topic); + ProducerBuilderImpl producerBuilder = + (ProducerBuilderImpl) pulsarClient.newProducer(Schema.STRING).topic(topic); + producerBuilder.getConf().setNonPartitionedTopicExpected(true); + // Verify: create successfully. + Producer producer = producerBuilder.create(); + // cleanup. + producer.close(); + admin.topics().delete(topic, false); + } + + @Test + public void testWhenPartitionedTopicExists() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createPartitionedTopic(topic, 2); + ProducerBuilderImpl producerBuilder = + (ProducerBuilderImpl) pulsarClient.newProducer(Schema.STRING).topic(topic); + producerBuilder.getConf().setNonPartitionedTopicExpected(true); + // Verify: failed to create. + try { + producerBuilder.create(); + Assert.fail("expected an error since producer expected a non-partitioned topic"); + } catch (Exception ex) { + // expected an error. + log.error("expected error", ex); + } + // cleanup. + admin.topics().deletePartitionedTopic(topic, false); + } + + @DataProvider(name = "topicTypes") + public Object[][] topicTypes() { + return new Object[][]{ + {TopicType.PARTITIONED}, + {TopicType.NON_PARTITIONED} + }; + } + + @Test(dataProvider = "topicTypes") + public void testWhenTopicNotExists(TopicType topicType) throws Exception { + final String namespace = "public/default"; + final String topic = BrokerTestUtil.newUniqueName("persistent://" + namespace + "/tp"); + final TopicName topicName = TopicName.get(topic); + AutoTopicCreationOverride.Builder policyBuilder = AutoTopicCreationOverride.builder() + .topicType(topicType.toString()).allowAutoTopicCreation(true); + if (topicType.equals(TopicType.PARTITIONED)) { + policyBuilder.defaultNumPartitions(2); + } + AutoTopicCreationOverride policy = policyBuilder.build(); + admin.namespaces().setAutoTopicCreation(namespace, policy); + + ProducerBuilderImpl producerBuilder = + (ProducerBuilderImpl) pulsarClient.newProducer(Schema.STRING).topic(topic); + producerBuilder.getConf().setNonPartitionedTopicExpected(true); + // Verify: create successfully. + Producer producer = producerBuilder.create(); + // Verify: only create non-partitioned topic. + Assert.assertFalse(pulsar.getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .partitionedTopicExists(topicName)); + Assert.assertTrue(pulsar.getNamespaceService().checkNonPartitionedTopicExists(topicName).join()); + + // cleanup. + producer.close(); + admin.topics().delete(topic, false); + admin.namespaces().removeAutoTopicCreation(namespace); + } +} 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 e93a84e298adb..327f59081c4e6 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 @@ -48,8 +48,11 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import javax.annotation.Nullable; import lombok.Builder; import lombok.Getter; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; @@ -374,26 +377,55 @@ public CompletableFuture> createProducerAsync(ProducerConfigurat } + private CompletableFuture checkPartitions(String topic, boolean forceNoPartitioned, + @Nullable String producerNameForLog) { + CompletableFuture checkPartitions = new CompletableFuture<>(); + getPartitionedTopicMetadata(topic, !forceNoPartitioned).thenAccept(metadata -> { + if (forceNoPartitioned && metadata.partitions > 0) { + String errorMsg = String.format("Can not create the producer[%s] for the topic[%s] that contains %s" + + " partitions, but the producer does not support for a partitioned topic.", + producerNameForLog, topic, metadata.partitions); + log.error(errorMsg); + checkPartitions.completeExceptionally( + new PulsarClientException.LookupException(errorMsg)); + } else { + checkPartitions.complete(metadata.partitions); + } + }).exceptionally(ex -> { + Throwable actEx = FutureUtil.unwrapCompletionException(ex); + if (forceNoPartitioned && actEx instanceof PulsarClientException.NotFoundException + || actEx instanceof PulsarClientException.TopicDoesNotExistException + || actEx instanceof PulsarAdminException.NotFoundException) { + checkPartitions.complete(0); + } else { + checkPartitions.completeExceptionally(ex); + } + return null; + }); + return checkPartitions; + } + private CompletableFuture> createProducerAsync(String topic, ProducerConfigurationData conf, Schema schema, ProducerInterceptors interceptors) { CompletableFuture> producerCreatedFuture = new CompletableFuture<>(); - getPartitionedTopicMetadata(topic, true).thenAccept(metadata -> { + + + checkPartitions(topic, conf.isNonPartitionedTopicExpected(), conf.getProducerName()).thenAccept(partitions -> { if (log.isDebugEnabled()) { - log.debug("[{}] Received topic metadata. partitions: {}", topic, metadata.partitions); + log.debug("[{}] Received topic metadata. partitions: {}", topic, partitions); } ProducerBase producer; - if (metadata.partitions > 0) { + if (partitions > 0) { producer = newPartitionedProducerImpl(topic, conf, schema, interceptors, producerCreatedFuture, - metadata); + partitions); } else { producer = newProducerImpl(topic, -1, conf, schema, interceptors, producerCreatedFuture, Optional.empty()); } - producers.add(producer); }).exceptionally(ex -> { log.warn("[{}] Failed to get partitioned topic metadata: {}", topic, ex.getMessage()); @@ -414,7 +446,6 @@ private CompletableFuture> createProducerAsync(String topic, * @param schema topic schema * @param interceptors producer interceptors * @param producerCreatedFuture future for signaling completion of async producer creation - * @param metadata partitioned topic metadata * @param message type class * @return new PartitionedProducerImpl instance */ @@ -424,8 +455,8 @@ protected PartitionedProducerImpl newPartitionedProducerImpl(String topic ProducerInterceptors interceptors, CompletableFuture> producerCreatedFuture, - PartitionedTopicMetadata metadata) { - return new PartitionedProducerImpl<>(PulsarClientImpl.this, topic, conf, metadata.partitions, + int partitions) { + return new PartitionedProducerImpl<>(PulsarClientImpl.this, topic, conf, partitions, producerCreatedFuture, schema, interceptors); } 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 581b3d8a1635e..6ec738bbf4c8d 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 @@ -204,6 +204,8 @@ public class ProducerConfigurationData implements Serializable, Cloneable { private SortedMap properties = new TreeMap<>(); + private boolean isNonPartitionedTopicExpected; + @ApiModelProperty( name = "initialSubscriptionName", value = "Use this configuration to automatically create an initial subscription when creating a topic." From b4acd58dbce2d5a125e7b5b9a0750848c05af9d7 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Tue, 16 Jul 2024 10:08:36 +0800 Subject: [PATCH 624/699] fix code style introduce by #22983 (cherry picked from commit fdd9747968c60ac5c6c89bab5786cd7e65db0716) --- .../java/org/apache/pulsar/client/impl/PulsarClientImpl.java | 1 - 1 file changed, 1 deletion(-) 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 327f59081c4e6..3ac9c5727caad 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 @@ -51,7 +51,6 @@ import javax.annotation.Nullable; import lombok.Builder; import lombok.Getter; -import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.Consumer; From 235f7c59533c17a602a4345baede37d48e409a2b Mon Sep 17 00:00:00 2001 From: Wenzhi Feng <52550727+thetumbled@users.noreply.github.com> Date: Fri, 19 Jul 2024 15:52:38 +0800 Subject: [PATCH 625/699] [improve][broker][branch-3.0] PIP-364: Introduce a new load balance algorithm AvgShedder (#23053) (cherry picked from commit 3bdd598787732f9c89c3e82af0bc19aeb46415db) --- conf/broker.conf | 19 ++ .../pulsar/broker/ServiceConfiguration.java | 36 +- .../broker/loadbalance/impl/AvgShedder.java | 318 ++++++++++++++++++ .../impl/ModularLoadManagerImpl.java | 16 +- .../ModularLoadManagerStrategyTest.java | 43 +++ .../loadbalance/impl/AvgShedderTest.java | 283 ++++++++++++++++ 6 files changed, 711 insertions(+), 4 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/AvgShedder.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/AvgShedderTest.java diff --git a/conf/broker.conf b/conf/broker.conf index 906a84ce89e15..670e05d75b497 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1416,6 +1416,25 @@ loadBalancerBundleUnloadMinThroughputThreshold=10 # Time to wait for the unloading of a namespace bundle namespaceBundleUnloadingTimeoutMs=60000 +# configuration for AvgShedder, a new shedding and placement strategy +# The low threshold for the difference between the highest and lowest loaded brokers. +loadBalancerAvgShedderLowThreshold = 15 + +# The high threshold for the difference between the highest and lowest loaded brokers. +loadBalancerAvgShedderHighThreshold = 40 + +# The number of times the low threshold is triggered before the bundle is unloaded. +loadBalancerAvgShedderHitCountLowThreshold = 8 + +# The number of times the high threshold is triggered before the bundle is unloaded. +loadBalancerAvgShedderHitCountHighThreshold = 2 + +# In the UniformLoadShedder and AvgShedder strategy, the maximum unload ratio. +# For AvgShedder, recommend to set to 0.5, so that it will distribute the load evenly +# between the highest and lowest brokers. +maxUnloadPercentage = 0.2 + + ### --- Load balancer extension --- ### # Option to enable the debug mode for the load balancer logics. 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 e6d84cd768682..f54de8e8a1cd6 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 @@ -2387,21 +2387,51 @@ The delayed message index time step(in seconds) in per bucket snapshot segment, @FieldContext( dynamic = true, category = CATEGORY_LOAD_BALANCER, - doc = "In the UniformLoadShedder strategy, the minimum message that triggers unload." + doc = "The low threshold for the difference between the highest and lowest loaded brokers." + ) + private int loadBalancerAvgShedderLowThreshold = 15; + + @FieldContext( + dynamic = true, + category = CATEGORY_LOAD_BALANCER, + doc = "The high threshold for the difference between the highest and lowest loaded brokers." + ) + private int loadBalancerAvgShedderHighThreshold = 40; + + @FieldContext( + dynamic = true, + category = CATEGORY_LOAD_BALANCER, + doc = "The number of times the low threshold is triggered before the bundle is unloaded." + ) + private int loadBalancerAvgShedderHitCountLowThreshold = 8; + + @FieldContext( + dynamic = true, + category = CATEGORY_LOAD_BALANCER, + doc = "The number of times the high threshold is triggered before the bundle is unloaded." + ) + private int loadBalancerAvgShedderHitCountHighThreshold = 2; + + @FieldContext( + dynamic = true, + category = CATEGORY_LOAD_BALANCER, + doc = "In the UniformLoadShedder and AvgShedder strategy, the minimum message that triggers unload." ) private int minUnloadMessage = 1000; @FieldContext( dynamic = true, category = CATEGORY_LOAD_BALANCER, - doc = "In the UniformLoadShedder strategy, the minimum throughput that triggers unload." + doc = "In the UniformLoadShedder and AvgShedder strategy, the minimum throughput that triggers unload." ) private int minUnloadMessageThroughput = 1 * 1024 * 1024; @FieldContext( dynamic = true, category = CATEGORY_LOAD_BALANCER, - doc = "In the UniformLoadShedder strategy, the maximum unload ratio." + doc = "In the UniformLoadShedder and AvgShedder strategy, the maximum unload ratio." + + "For AvgShedder, recommend to set to 0.5, so that it will distribute the load " + + "evenly between the highest and lowest brokers." ) private double maxUnloadPercentage = 0.2; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/AvgShedder.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/AvgShedder.java new file mode 100644 index 0000000000000..e33ff097164ae --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/impl/AvgShedder.java @@ -0,0 +1,318 @@ +/* + * 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.impl; + +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.Multimap; +import com.google.common.hash.Hashing; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Random; +import java.util.Set; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.mutable.MutableDouble; +import org.apache.commons.lang3.mutable.MutableInt; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.loadbalance.LoadData; +import org.apache.pulsar.broker.loadbalance.LoadSheddingStrategy; +import org.apache.pulsar.broker.loadbalance.ModularLoadManagerStrategy; +import org.apache.pulsar.policies.data.loadbalancer.BrokerData; +import org.apache.pulsar.policies.data.loadbalancer.BundleData; +import org.apache.pulsar.policies.data.loadbalancer.LocalBrokerData; +import org.apache.pulsar.policies.data.loadbalancer.TimeAverageMessageData; + +@Slf4j +public class AvgShedder implements LoadSheddingStrategy, ModularLoadManagerStrategy { + // map bundle to broker. + private final Map bundleBrokerMap = new HashMap<>(); + // map broker to Scores. scores:0-100 + private final Map brokerScoreMap = new HashMap<>(); + // map broker hit count for high threshold/low threshold + private final Map brokerHitCountForHigh = new HashMap<>(); + private final Map brokerHitCountForLow = new HashMap<>(); + private static final double MB = 1024 * 1024; + + @Override + public Multimap findBundlesForUnloading(LoadData loadData, ServiceConfiguration conf) { + // result returned by shedding, map broker to bundles. + Multimap selectedBundlesCache = ArrayListMultimap.create(); + + // configuration for shedding. + final double minThroughputThreshold = conf.getMinUnloadMessageThroughput(); + final double minMsgThreshold = conf.getMinUnloadMessage(); + final double maxUnloadPercentage = conf.getMaxUnloadPercentage(); + final double lowThreshold = conf.getLoadBalancerAvgShedderLowThreshold(); + final double highThreshold = conf.getLoadBalancerAvgShedderHighThreshold(); + final int hitCountHighThreshold = conf.getLoadBalancerAvgShedderHitCountHighThreshold(); + final int hitCountLowThreshold = conf.getLoadBalancerAvgShedderHitCountLowThreshold(); + if (log.isDebugEnabled()) { + log.debug("highThreshold:{}, lowThreshold:{}, hitCountHighThreshold:{}, hitCountLowThreshold:{}, " + + "minMsgThreshold:{}, minThroughputThreshold:{}", + highThreshold, lowThreshold, hitCountHighThreshold, hitCountLowThreshold, + minMsgThreshold, minThroughputThreshold); + } + + List brokers = calculateScoresAndSort(loadData, conf); + log.info("sorted broker list:{}", brokers); + + // find broker pairs for shedding. + List> pairs = findBrokerPairs(brokers, lowThreshold, highThreshold); + log.info("brokerHitCountForHigh:{}, brokerHitCountForLow:{}", brokerHitCountForHigh, brokerHitCountForLow); + if (pairs.isEmpty()) { + if (log.isDebugEnabled()) { + log.debug("there is no any overload broker, no need to shedding bundles."); + } + brokerHitCountForHigh.clear(); + brokerHitCountForLow.clear(); + return selectedBundlesCache; + } + + // choosing bundles to unload. + for (Pair pair : pairs) { + String overloadedBroker = pair.getRight(); + String underloadedBroker = pair.getLeft(); + + // check hit count for high threshold and low threshold. + if (!(brokerHitCountForHigh.computeIfAbsent(underloadedBroker, __ -> new MutableInt(0)) + .intValue() >= hitCountHighThreshold) + && !(brokerHitCountForHigh.computeIfAbsent(overloadedBroker, __ -> new MutableInt(0)) + .intValue() >= hitCountHighThreshold) + && !(brokerHitCountForLow.computeIfAbsent(underloadedBroker, __ -> new MutableInt(0)) + .intValue() >= hitCountLowThreshold) + && !(brokerHitCountForLow.computeIfAbsent(overloadedBroker, __ -> new MutableInt(0)) + .intValue() >= hitCountLowThreshold)) { + continue; + } + + // if hit, remove entry. + brokerHitCountForHigh.remove(underloadedBroker); + brokerHitCountForHigh.remove(overloadedBroker); + brokerHitCountForLow.remove(underloadedBroker); + brokerHitCountForLow.remove(overloadedBroker); + + // select bundle for unloading. + selectBundleForUnloading(loadData, overloadedBroker, underloadedBroker, minThroughputThreshold, + minMsgThreshold, maxUnloadPercentage, selectedBundlesCache); + } + return selectedBundlesCache; + } + + private void selectBundleForUnloading(LoadData loadData, String overloadedBroker, String underloadedBroker, + double minThroughputThreshold, double minMsgThreshold, + double maxUnloadPercentage, Multimap selectedBundlesCache) { + // calculate how much throughput to unload. + LocalBrokerData minLocalBrokerData = loadData.getBrokerData().get(underloadedBroker).getLocalData(); + LocalBrokerData maxLocalBrokerData = loadData.getBrokerData().get(overloadedBroker).getLocalData(); + + double minMsgRate = minLocalBrokerData.getMsgRateIn() + minLocalBrokerData.getMsgRateOut(); + double maxMsgRate = maxLocalBrokerData.getMsgRateIn() + maxLocalBrokerData.getMsgRateOut(); + + double minThroughput = minLocalBrokerData.getMsgThroughputIn() + minLocalBrokerData.getMsgThroughputOut(); + double maxThroughput = maxLocalBrokerData.getMsgThroughputIn() + maxLocalBrokerData.getMsgThroughputOut(); + + double msgRequiredFromUnloadedBundles = (maxMsgRate - minMsgRate) * maxUnloadPercentage; + double throughputRequiredFromUnloadedBundles = (maxThroughput - minThroughput) * maxUnloadPercentage; + + boolean isMsgRateToOffload; + MutableDouble trafficMarkedToOffload = new MutableDouble(0); + + if (msgRequiredFromUnloadedBundles > minMsgThreshold) { + isMsgRateToOffload = true; + trafficMarkedToOffload.setValue(msgRequiredFromUnloadedBundles); + } else if (throughputRequiredFromUnloadedBundles > minThroughputThreshold) { + isMsgRateToOffload = false; + trafficMarkedToOffload.setValue(throughputRequiredFromUnloadedBundles); + } else { + log.info( + "broker:[{}] is planning to shed bundles to broker:[{}],but the throughput {} MByte/s is " + + "less than minimumThroughputThreshold {} MByte/s, and the msgRate {} rate/s" + + " is also less than minimumMsgRateThreshold {} rate/s, skipping bundle unload.", + overloadedBroker, underloadedBroker, throughputRequiredFromUnloadedBundles / MB, + minThroughputThreshold / MB, msgRequiredFromUnloadedBundles, minMsgThreshold); + return; + } + + if (maxLocalBrokerData.getBundles().size() == 1) { + log.warn("HIGH USAGE WARNING : Sole namespace bundle {} is overloading broker {}. " + + "No Load Shedding will be done on this broker", + maxLocalBrokerData.getBundles().iterator().next(), overloadedBroker); + } else if (maxLocalBrokerData.getBundles().isEmpty()) { + log.warn("Broker {} is overloaded despite having no bundles", overloadedBroker); + } + + // do shedding + log.info( + "broker:[{}] is planning to shed bundles to broker:[{}]. " + + "maxBroker stat:scores:{}, throughput:{}, msgRate:{}. " + + "minBroker stat:scores:{}, throughput:{}, msgRate:{}. " + + "isMsgRateToOffload:{}, trafficMarkedToOffload:{}", + overloadedBroker, underloadedBroker, brokerScoreMap.get(overloadedBroker), maxThroughput, + maxMsgRate, brokerScoreMap.get(underloadedBroker), minThroughput, minMsgRate, + isMsgRateToOffload, trafficMarkedToOffload); + + loadData.getBundleDataForLoadShedding().entrySet().stream().filter(e -> + maxLocalBrokerData.getBundles().contains(e.getKey()) + ).filter(e -> + !loadData.getRecentlyUnloadedBundles().containsKey(e.getKey()) + ).map((e) -> { + BundleData bundleData = e.getValue(); + TimeAverageMessageData shortTermData = bundleData.getShortTermData(); + double traffic = isMsgRateToOffload + ? shortTermData.getMsgRateIn() + shortTermData.getMsgRateOut() + : shortTermData.getMsgThroughputIn() + shortTermData.getMsgThroughputOut(); + return Pair.of(e, traffic); + }).sorted((e1, e2) -> + Double.compare(e2.getRight(), e1.getRight()) + ).forEach(e -> { + Map.Entry bundle = e.getLeft(); + double traffic = e.getRight(); + if (traffic > 0 && traffic <= trafficMarkedToOffload.getValue()) { + selectedBundlesCache.put(overloadedBroker, bundle.getKey()); + bundleBrokerMap.put(bundle.getValue(), underloadedBroker); + trafficMarkedToOffload.add(-traffic); + if (log.isDebugEnabled()) { + log.debug("Found bundle to unload:{}, isMsgRateToOffload:{}, traffic:{}", + bundle, isMsgRateToOffload, traffic); + } + } + }); + } + + @Override + public void onActiveBrokersChange(Set activeBrokers) { + LoadSheddingStrategy.super.onActiveBrokersChange(activeBrokers); + } + + private List calculateScoresAndSort(LoadData loadData, ServiceConfiguration conf) { + brokerScoreMap.clear(); + + // calculate scores of brokers. + for (Map.Entry entry : loadData.getBrokerData().entrySet()) { + LocalBrokerData localBrokerData = entry.getValue().getLocalData(); + String broker = entry.getKey(); + Double score = calculateScores(localBrokerData, conf); + brokerScoreMap.put(broker, score); + if (log.isDebugEnabled()) { + log.info("broker:{}, scores:{}, throughput:{}, messageRate:{}", broker, score, + localBrokerData.getMsgThroughputIn() + localBrokerData.getMsgThroughputOut(), + localBrokerData.getMsgRateIn() + localBrokerData.getMsgRateOut()); + } + } + + // sort brokers by scores. + return brokerScoreMap.entrySet().stream().sorted((o1, o2) -> (int) (o1.getValue() - o2.getValue())) + .map(Map.Entry::getKey).toList(); + } + + private Double calculateScores(LocalBrokerData localBrokerData, final ServiceConfiguration conf) { + return localBrokerData.getMaxResourceUsageWithWeight( + conf.getLoadBalancerCPUResourceWeight(), + conf.getLoadBalancerDirectMemoryResourceWeight(), + conf.getLoadBalancerBandwithInResourceWeight(), + conf.getLoadBalancerBandwithOutResourceWeight()) * 100; + } + + private List> findBrokerPairs(List brokers, + double lowThreshold, double highThreshold) { + List> pairs = new LinkedList<>(); + int i = 0, j = brokers.size() - 1; + while (i <= j) { + String maxBroker = brokers.get(j); + String minBroker = brokers.get(i); + if (brokerScoreMap.get(maxBroker) - brokerScoreMap.get(minBroker) < lowThreshold) { + brokerHitCountForHigh.remove(maxBroker); + brokerHitCountForHigh.remove(minBroker); + + brokerHitCountForLow.remove(maxBroker); + brokerHitCountForLow.remove(minBroker); + } else { + pairs.add(Pair.of(minBroker, maxBroker)); + if (brokerScoreMap.get(maxBroker) - brokerScoreMap.get(minBroker) < highThreshold) { + brokerHitCountForLow.computeIfAbsent(minBroker, k -> new MutableInt(0)).increment(); + brokerHitCountForLow.computeIfAbsent(maxBroker, k -> new MutableInt(0)).increment(); + + brokerHitCountForHigh.remove(maxBroker); + brokerHitCountForHigh.remove(minBroker); + } else { + brokerHitCountForLow.computeIfAbsent(minBroker, k -> new MutableInt(0)).increment(); + brokerHitCountForLow.computeIfAbsent(maxBroker, k -> new MutableInt(0)).increment(); + + brokerHitCountForHigh.computeIfAbsent(minBroker, k -> new MutableInt(0)).increment(); + brokerHitCountForHigh.computeIfAbsent(maxBroker, k -> new MutableInt(0)).increment(); + } + } + i++; + j--; + } + return pairs; + } + + @Override + public Optional selectBroker(Set candidates, BundleData bundleToAssign, LoadData loadData, + ServiceConfiguration conf) { + final var brokerToUnload = bundleBrokerMap.getOrDefault(bundleToAssign, null); + if (brokerToUnload == null || !candidates.contains(bundleBrokerMap.get(bundleToAssign))) { + // cluster initializing or broker is shutdown + if (log.isDebugEnabled()) { + if (!bundleBrokerMap.containsKey(bundleToAssign)) { + log.debug("cluster is initializing"); + } else { + log.debug("expected broker:{} is shutdown, candidates:{}", bundleBrokerMap.get(bundleToAssign), + candidates); + } + } + String broker = getExpectedBroker(candidates, bundleToAssign); + bundleBrokerMap.put(bundleToAssign, broker); + return Optional.of(broker); + } else { + return Optional.of(brokerToUnload); + } + } + + private static String getExpectedBroker(Collection brokers, BundleData bundle) { + List sortedBrokers = new ArrayList<>(brokers); + Collections.sort(sortedBrokers); + + try { + // use random number as input of hashing function to avoid special case that, + // if there is 4 brokers running in the cluster,and add broker5,and shutdown broker3, + // then all bundles belonging to broker3 will be loaded on the same broker. + final long hashcode = Hashing.crc32().hashString(String.valueOf(new Random().nextInt()), + StandardCharsets.UTF_8).padToLong(); + final int index = (int) (Math.abs(hashcode) % sortedBrokers.size()); + if (log.isDebugEnabled()) { + log.debug("Assignment details: brokers={}, bundle={}, hashcode={}, index={}", + sortedBrokers, bundle, hashcode, index); + } + return sortedBrokers.get(index); + } catch (Throwable e) { + // theoretically this logic branch should not be executed + log.error("Bundle format of {} is invalid", bundle, e); + return sortedBrokers.get(Math.abs(bundle.hashCode()) % sortedBrokers.size()); + } + } +} 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 d6c079ab85ce7..edd7080ee2e18 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 @@ -279,7 +279,21 @@ public void initialize(final PulsarService pulsar) { () -> LoadManagerShared.refreshBrokerToFailureDomainMap(pulsar, brokerToFailureDomainMap)); }); - loadSheddingPipeline.add(createLoadSheddingStrategy()); + if (placementStrategy instanceof LoadSheddingStrategy) { + // if the placement strategy is also a load shedding strategy + // we need to check two strategies are the same + if (!conf.getLoadBalancerLoadSheddingStrategy().equals( + conf.getLoadBalancerPlacementStrategy())) { + throw new IllegalArgumentException("The load shedding strategy: " + + conf.getLoadBalancerLoadSheddingStrategy() + + " can't work with the placement strategy: " + + conf.getLoadBalancerPlacementStrategy()); + } + // bind the load shedding strategy and the placement strategy + loadSheddingPipeline.add((LoadSheddingStrategy) placementStrategy); + } else { + loadSheddingPipeline.add(createLoadSheddingStrategy()); + } } public void handleDataNotification(Notification t) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerStrategyTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerStrategyTest.java index c64c9950a95a9..199981983758b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerStrategyTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/ModularLoadManagerStrategyTest.java @@ -19,6 +19,7 @@ package org.apache.pulsar.broker.loadbalance; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; import java.lang.reflect.Field; import java.util.Arrays; @@ -34,6 +35,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.loadbalance.impl.AvgShedder; import org.apache.pulsar.broker.loadbalance.impl.LeastLongTermMessageRate; import org.apache.pulsar.broker.loadbalance.impl.LeastResourceUsageWithWeight; import org.apache.pulsar.broker.loadbalance.impl.RoundRobinBrokerSelector; @@ -47,6 +49,47 @@ @Test(groups = "broker") public class ModularLoadManagerStrategyTest { + public void testAvgShedderWithPreassignedBroker() throws Exception { + ModularLoadManagerStrategy strategy = new AvgShedder(); + Field field = AvgShedder.class.getDeclaredField("bundleBrokerMap"); + field.setAccessible(true); + Map bundleBrokerMap = (Map) field.get(strategy); + BundleData bundleData = new BundleData(); + // assign bundle to broker1 in bundleBrokerMap. + bundleBrokerMap.put(bundleData, "1"); + assertEquals(strategy.selectBroker(Set.of("1", "2", "3"), bundleData, null, null), Optional.of("1")); + assertEquals(bundleBrokerMap.get(bundleData), "1"); + + // remove broker1 in candidates, only broker2 is candidate. + assertEquals(strategy.selectBroker(Set.of("2"), bundleData, null, null), Optional.of("2")); + assertEquals(bundleBrokerMap.get(bundleData), "2"); + } + + public void testAvgShedderWithoutPreassignedBroker() throws Exception { + ModularLoadManagerStrategy strategy = new AvgShedder(); + Field field = AvgShedder.class.getDeclaredField("bundleBrokerMap"); + field.setAccessible(true); + Map bundleBrokerMap = (Map) field.get(strategy); + BundleData bundleData = new BundleData(); + Set candidates = new HashSet<>(); + candidates.add("1"); + candidates.add("2"); + candidates.add("3"); + + // select broker from candidates randomly. + Optional selectedBroker = strategy.selectBroker(candidates, bundleData, null, null); + assertTrue(selectedBroker.isPresent()); + assertTrue(candidates.contains(selectedBroker.get())); + assertEquals(bundleBrokerMap.get(bundleData), selectedBroker.get()); + + // remove original broker in candidates + candidates.remove(selectedBroker.get()); + selectedBroker = strategy.selectBroker(candidates, bundleData, null, null); + assertTrue(selectedBroker.isPresent()); + assertTrue(candidates.contains(selectedBroker.get())); + assertEquals(bundleBrokerMap.get(bundleData), selectedBroker.get()); + } + // Test that least long term message rate works correctly. public void testLeastLongTermMessageRate() { BundleData bundleData = new BundleData(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/AvgShedderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/AvgShedderTest.java new file mode 100644 index 0000000000000..215e3d766a927 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/loadbalance/impl/AvgShedderTest.java @@ -0,0 +1,283 @@ +/* + * 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.impl; + +import com.google.common.collect.Multimap; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.loadbalance.LoadData; +import org.apache.pulsar.policies.data.loadbalancer.BrokerData; +import org.apache.pulsar.policies.data.loadbalancer.BundleData; +import org.apache.pulsar.policies.data.loadbalancer.LocalBrokerData; +import org.apache.pulsar.policies.data.loadbalancer.ResourceUsage; +import org.apache.pulsar.policies.data.loadbalancer.TimeAverageBrokerData; +import org.apache.pulsar.policies.data.loadbalancer.TimeAverageMessageData; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.fail; + +@Test(groups = "broker") +public class AvgShedderTest { + private AvgShedder avgShedder; + private final ServiceConfiguration conf; + + public AvgShedderTest() { + conf = new ServiceConfiguration(); + } + + @BeforeMethod + public void setup() { + avgShedder = new AvgShedder(); + } + + private BrokerData initBrokerData() { + LocalBrokerData localBrokerData = new LocalBrokerData(); + localBrokerData.setCpu(new ResourceUsage()); + localBrokerData.setMemory(new ResourceUsage()); + localBrokerData.setBandwidthIn(new ResourceUsage()); + localBrokerData.setBandwidthOut(new ResourceUsage()); + BrokerData brokerData = new BrokerData(localBrokerData); + TimeAverageBrokerData timeAverageBrokerData = new TimeAverageBrokerData(); + brokerData.setTimeAverageData(timeAverageBrokerData); + return brokerData; + } + + @Test + public void testHitHighThreshold() { + LoadData loadData = new LoadData(); + BrokerData brokerData1 = initBrokerData(); + BrokerData brokerData2 = initBrokerData(); + BrokerData brokerData3 = initBrokerData(); + loadData.getBrokerData().put("broker1", brokerData1); + loadData.getBrokerData().put("broker2", brokerData2); + loadData.getBrokerData().put("broker3", brokerData3); + // AvgShedder will distribute the load evenly between the highest and lowest brokers + conf.setMaxUnloadPercentage(0.5); + + // Set the high threshold to 40% and hit count high threshold to 2 + int hitCountForHighThreshold = 2; + conf.setLoadBalancerAvgShedderHighThreshold(40); + conf.setLoadBalancerAvgShedderHitCountHighThreshold(hitCountForHighThreshold); + brokerData1.getLocalData().setCpu(new ResourceUsage(80, 100)); + brokerData2.getLocalData().setCpu(new ResourceUsage(30, 100)); + brokerData1.getLocalData().setMsgRateIn(10000); + brokerData1.getLocalData().setMsgRateOut(10000); + brokerData2.getLocalData().setMsgRateIn(1000); + brokerData2.getLocalData().setMsgRateOut(1000); + + // broker3 is in the middle + brokerData3.getLocalData().setCpu(new ResourceUsage(50, 100)); + brokerData3.getLocalData().setMsgRateIn(5000); + brokerData3.getLocalData().setMsgRateOut(5000); + + // expect to shed bundles with message rate(in+out) ((10000+10000)-(1000+1000))/2 = 9000 + // each bundle with 450 msg rate in and 450 msg rate out + // so 9000/(450+450)=10 bundles will be shed + for (int i = 0; i < 11; i++) { + brokerData1.getLocalData().getBundles().add("bundle-" + i); + BundleData bundle = new BundleData(); + TimeAverageMessageData timeAverageMessageData = new TimeAverageMessageData(); + timeAverageMessageData.setMsgRateIn(450); + timeAverageMessageData.setMsgRateOut(450); + // as AvgShedder map BundleData to broker, the hashCode of different BundleData should be different + // so we need to set some different fields to make the hashCode different + timeAverageMessageData.setNumSamples(i); + bundle.setShortTermData(timeAverageMessageData); + loadData.getBundleData().put("bundle-" + i, bundle); + } + + // do shedding for the first time, expect to shed nothing because hit count is not enough + Multimap bundlesToUnload = avgShedder.findBundlesForUnloading(loadData, conf); + assertEquals(bundlesToUnload.size(), 0); + + // do shedding for the second time, expect to shed 10 bundles + bundlesToUnload = avgShedder.findBundlesForUnloading(loadData, conf); + assertEquals(bundlesToUnload.size(), 10); + + // assert that all the bundles are shed from broker1 + for (String broker : bundlesToUnload.keys()) { + assertEquals(broker, "broker1"); + } + // assert that all the bundles are shed to broker2 + for (String bundle : bundlesToUnload.values()) { + BundleData bundleData = loadData.getBundleData().get(bundle); + assertEquals(avgShedder.selectBroker(loadData.getBrokerData().keySet(), bundleData, loadData, conf).get(), "broker2"); + } + } + + @Test + public void testHitLowThreshold() { + LoadData loadData = new LoadData(); + BrokerData brokerData1 = initBrokerData(); + BrokerData brokerData2 = initBrokerData(); + BrokerData brokerData3 = initBrokerData(); + loadData.getBrokerData().put("broker1", brokerData1); + loadData.getBrokerData().put("broker2", brokerData2); + loadData.getBrokerData().put("broker3", brokerData3); + // AvgShedder will distribute the load evenly between the highest and lowest brokers + conf.setMaxUnloadPercentage(0.5); + + // Set the low threshold to 20% and hit count low threshold to 6 + int hitCountForLowThreshold = 6; + conf.setLoadBalancerAvgShedderLowThreshold(20); + conf.setLoadBalancerAvgShedderHitCountLowThreshold(hitCountForLowThreshold); + brokerData1.getLocalData().setCpu(new ResourceUsage(60, 100)); + brokerData2.getLocalData().setCpu(new ResourceUsage(40, 100)); + brokerData1.getLocalData().setMsgRateIn(10000); + brokerData1.getLocalData().setMsgRateOut(10000); + brokerData2.getLocalData().setMsgRateIn(1000); + brokerData2.getLocalData().setMsgRateOut(1000); + + // broker3 is in the middle + brokerData3.getLocalData().setCpu(new ResourceUsage(50, 100)); + brokerData3.getLocalData().setMsgRateIn(5000); + brokerData3.getLocalData().setMsgRateOut(5000); + + // expect to shed bundles with message rate(in+out) ((10000+10000)-(1000+1000))/2 = 9000 + // each bundle with 450 msg rate in and 450 msg rate out + // so 9000/(450+450)=10 bundles will be shed + for (int i = 0; i < 11; i++) { + brokerData1.getLocalData().getBundles().add("bundle-" + i); + BundleData bundle = new BundleData(); + TimeAverageMessageData timeAverageMessageData = new TimeAverageMessageData(); + timeAverageMessageData.setMsgRateIn(450); + timeAverageMessageData.setMsgRateOut(450); + // as AvgShedder map BundleData to broker, the hashCode of different BundleData should be different + // so we need to set some different fields to make the hashCode different + timeAverageMessageData.setNumSamples(i); + bundle.setShortTermData(timeAverageMessageData); + loadData.getBundleData().put("bundle-" + i, bundle); + } + + // do shedding for (lowCountForHighThreshold - 1) times, expect to shed nothing because hit count is not enough + Multimap bundlesToUnload; + for (int i = 0; i < hitCountForLowThreshold - 1; i++) { + bundlesToUnload = avgShedder.findBundlesForUnloading(loadData, conf); + assertEquals(bundlesToUnload.size(), 0); + } + + // do shedding for the last time, expect to shed 10 bundles + bundlesToUnload = avgShedder.findBundlesForUnloading(loadData, conf); + assertEquals(bundlesToUnload.size(), 10); + + // assert that all the bundles are shed from broker1 + for (String broker : bundlesToUnload.keys()) { + assertEquals(broker, "broker1"); + } + // assert that all the bundles are shed to broker2 + for (String bundle : bundlesToUnload.values()) { + BundleData bundleData = loadData.getBundleData().get(bundle); + assertEquals(avgShedder.selectBroker(loadData.getBrokerData().keySet(), bundleData, loadData, conf).get(), "broker2"); + } + } + + @Test + public void testSheddingMultiplePairs() { + LoadData loadData = new LoadData(); + BrokerData brokerData1 = initBrokerData(); + BrokerData brokerData2 = initBrokerData(); + BrokerData brokerData3 = initBrokerData(); + BrokerData brokerData4 = initBrokerData(); + loadData.getBrokerData().put("broker1", brokerData1); + loadData.getBrokerData().put("broker2", brokerData2); + loadData.getBrokerData().put("broker3", brokerData3); + loadData.getBrokerData().put("broker4", brokerData4); + // AvgShedder will distribute the load evenly between the highest and lowest brokers + conf.setMaxUnloadPercentage(0.5); + + // Set the high threshold to 40% and hit count high threshold to 2 + int hitCountForHighThreshold = 2; + conf.setLoadBalancerAvgShedderHighThreshold(40); + conf.setLoadBalancerAvgShedderHitCountHighThreshold(hitCountForHighThreshold); + + // pair broker1 and broker2 + brokerData1.getLocalData().setCpu(new ResourceUsage(80, 100)); + brokerData2.getLocalData().setCpu(new ResourceUsage(30, 100)); + brokerData1.getLocalData().setMsgRateIn(10000); + brokerData1.getLocalData().setMsgRateOut(10000); + brokerData2.getLocalData().setMsgRateIn(1000); + brokerData2.getLocalData().setMsgRateOut(1000); + + // pair broker3 and broker4 + brokerData3.getLocalData().setCpu(new ResourceUsage(75, 100)); + brokerData3.getLocalData().setMsgRateIn(10000); + brokerData3.getLocalData().setMsgRateOut(10000); + brokerData4.getLocalData().setCpu(new ResourceUsage(35, 100)); + brokerData4.getLocalData().setMsgRateIn(1000); + brokerData4.getLocalData().setMsgRateOut(1000); + + // expect to shed bundles with message rate(in+out) ((10000+10000)-(1000+1000))/2 = 9000 + // each bundle with 450 msg rate in and 450 msg rate out + // so 9000/(450+450)=10 bundles will be shed + for (int i = 0; i < 11; i++) { + brokerData1.getLocalData().getBundles().add("bundle1-" + i); + brokerData3.getLocalData().getBundles().add("bundle3-" + i); + + BundleData bundle = new BundleData(); + TimeAverageMessageData timeAverageMessageData = new TimeAverageMessageData(); + timeAverageMessageData.setMsgRateIn(450); + timeAverageMessageData.setMsgRateOut(450); + // as AvgShedder map BundleData to broker, the hashCode of different BundleData should be different + // so we need to set some different fields to make the hashCode different + timeAverageMessageData.setNumSamples(i); + bundle.setShortTermData(timeAverageMessageData); + loadData.getBundleData().put("bundle1-" + i, bundle); + + bundle = new BundleData(); + timeAverageMessageData = new TimeAverageMessageData(); + timeAverageMessageData.setMsgRateIn(450); + timeAverageMessageData.setMsgRateOut(450); + timeAverageMessageData.setNumSamples(i+11); + bundle.setShortTermData(timeAverageMessageData); + loadData.getBundleData().put("bundle3-" + i, bundle); + } + + // do shedding for the first time, expect to shed nothing because hit count is not enough + Multimap bundlesToUnload = avgShedder.findBundlesForUnloading(loadData, conf); + assertEquals(bundlesToUnload.size(), 0); + + // do shedding for the second time, expect to shed 10*2=20 bundles + bundlesToUnload = avgShedder.findBundlesForUnloading(loadData, conf); + assertEquals(bundlesToUnload.size(), 20); + + // assert that half of the bundles are shed from broker1, and the other half are shed from broker3 + for (String broker : bundlesToUnload.keys()) { + if (broker.equals("broker1")) { + assertEquals(bundlesToUnload.get(broker).size(), 10); + } else if (broker.equals("broker3")) { + assertEquals(bundlesToUnload.get(broker).size(), 10); + } else { + fail(); + } + } + + // assert that all the bundles from broker1 are shed to broker2, and all the bundles from broker3 are shed to broker4 + for (String bundle : bundlesToUnload.values()) { + BundleData bundleData = loadData.getBundleData().get(bundle); + if (bundle.startsWith("bundle1-")) { + assertEquals(avgShedder.selectBroker(loadData.getBrokerData().keySet(), bundleData, loadData, conf).get(), "broker2"); + } else if (bundle.startsWith("bundle3-")) { + assertEquals(avgShedder.selectBroker(loadData.getBrokerData().keySet(), bundleData, loadData, conf).get(), "broker4"); + } else { + fail(); + } + } + } +} From 082dfb5206be7fe0750720fa617f8281356fd7cb Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 22 Jul 2024 17:40:30 +0800 Subject: [PATCH 626/699] [improve] [broker] Improve CPU resources usege of TopicName Cache (#23052) Co-authored-by: Zixuan Liu (cherry picked from commit 81aed6c75eba99fb62172b986b0c59e693e6f4b9) (cherry picked from commit 5a839582ac08eb3be110de9b81f1cc7d245d4018) --- conf/broker.conf | 8 +++++ .../pulsar/broker/ServiceConfiguration.java | 15 +++++++++ .../pulsar/broker/service/BrokerService.java | 10 ++++++ .../pulsar/broker/PulsarServiceTest.java | 2 ++ .../pulsar/broker/service/StandaloneTest.java | 2 ++ .../naming/ServiceConfigurationTest.java | 13 ++++++++ .../configurations/pulsar_broker_test.conf | 2 ++ .../pulsar_broker_test_standalone.conf | 2 ++ .../pulsar/common/naming/TopicName.java | 33 +++++++++---------- 9 files changed, 70 insertions(+), 17 deletions(-) diff --git a/conf/broker.conf b/conf/broker.conf index 670e05d75b497..77020fd5c9994 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -159,6 +159,14 @@ skipBrokerShutdownOnOOM=false # Factory class-name to create topic with custom workflow topicFactoryClassName= +# Max capacity of the topic name cache. -1 means unlimited cache; 0 means broker will clear all cache +# per "maxSecondsToClearTopicNameCache", it does not mean broker will not cache TopicName. +topicNameCacheMaxCapacity=100000 + +# A Specifies the minimum number of seconds that the topic name stays in memory, to avoid clear cache frequently when +# there are too many topics are in use. +maxSecondsToClearTopicNameCache=7200 + # Enable backlog quota check. Enforces action on topic when the quota is reached backlogQuotaCheckEnabled=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 f54de8e8a1cd6..8e71c1dec578d 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 @@ -588,6 +588,21 @@ The delayed message index time step(in seconds) in per bucket snapshot segment, ) private boolean backlogQuotaCheckEnabled = true; + @FieldContext( + dynamic = true, + category = CATEGORY_POLICIES, + doc = "Max capacity of the topic name cache. -1 means unlimited cache; 0 means broker will clear all cache" + + " per maxSecondsToClearTopicNameCache, it does not mean broker will not cache TopicName." + ) + private int topicNameCacheMaxCapacity = 100_000; + + @FieldContext( + category = CATEGORY_POLICIES, + doc = "A Specifies the minimum number of seconds that the topic name stays in memory, to avoid clear cache" + + " frequently when there are too many topics are in use." + ) + private int maxSecondsToClearTopicNameCache = 3600 * 2; + @FieldContext( category = CATEGORY_POLICIES, doc = "Whether to enable precise time based backlog quota check. " 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 f62173f1acea6..410918a7ac7e7 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 @@ -582,6 +582,16 @@ public void start() throws Exception { this.updateBrokerDispatchThrottlingMaxRate(); this.startCheckReplicationPolicies(); this.startDeduplicationSnapshotMonitor(); + this.startClearInvalidateTopicNameCacheTask(); + } + + protected void startClearInvalidateTopicNameCacheTask() { + final int maxSecondsToClearTopicNameCache = pulsar.getConfiguration().getMaxSecondsToClearTopicNameCache(); + inactivityMonitor.scheduleAtFixedRate( + () -> TopicName.clearIfReachedMaxCapacity(pulsar.getConfiguration().getTopicNameCacheMaxCapacity()), + maxSecondsToClearTopicNameCache, + maxSecondsToClearTopicNameCache, + TimeUnit.SECONDS); } protected void startStatsUpdater(int statsUpdateInitialDelayInSecs, int statsUpdateFrequencyInSecs) { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java index 3e0887646e119..a515890dd3061 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/PulsarServiceTest.java @@ -56,6 +56,8 @@ protected void doInitConf() throws Exception { super.doInitConf(); conf.setBrokerServicePortTls(Optional.of(0)); conf.setWebServicePortTls(Optional.of(0)); + conf.setTopicNameCacheMaxCapacity(5000); + conf.setMaxSecondsToClearTopicNameCache(5); if (useStaticPorts) { conf.setBrokerServicePortTls(Optional.of(6651)); conf.setBrokerServicePort(Optional.of(6660)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java index 5307e1a9ee874..67d188efd2550 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java @@ -56,5 +56,7 @@ public void testAdvertised() throws Exception { assertNull(standalone.getConfig().getAdvertisedAddress()); assertEquals(standalone.getConfig().getAdvertisedListeners(), "internal:pulsar://192.168.1.11:6660,internal:pulsar+ssl://192.168.1.11:6651"); + assertEquals(standalone.getConfig().getMaxSecondsToClearTopicNameCache(), 1); + assertEquals(standalone.getConfig().getTopicNameCacheMaxCapacity(), 200); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java index 55971c15adf68..ae13afb19344b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java @@ -73,6 +73,8 @@ public void testInit() throws Exception { assertEquals(config.getManagedLedgerDataReadPriority(), "bookkeeper-first"); assertEquals(config.getBacklogQuotaDefaultLimitGB(), 0.05); assertEquals(config.getHttpMaxRequestHeaderSize(), 1234); + assertEquals(config.getMaxSecondsToClearTopicNameCache(), 1); + assertEquals(config.getTopicNameCacheMaxCapacity(), 200); OffloadPoliciesImpl offloadPolicies = OffloadPoliciesImpl.create(config.getProperties()); assertEquals(offloadPolicies.getManagedLedgerOffloadedReadPriority().getValue(), "bookkeeper-first"); } @@ -370,4 +372,15 @@ public void testAllowAutoTopicCreationType() throws Exception { conf = PulsarConfigurationLoader.create(properties, ServiceConfiguration.class); assertEquals(conf.getAllowAutoTopicCreationType(), TopicType.NON_PARTITIONED); } + + @Test + public void testTopicNameCacheConfiguration() throws Exception { + ServiceConfiguration conf; + final Properties properties = new Properties(); + properties.setProperty("maxSecondsToClearTopicNameCache", "2"); + properties.setProperty("topicNameCacheMaxCapacity", "100"); + conf = PulsarConfigurationLoader.create(properties, ServiceConfiguration.class); + assertEquals(conf.getMaxSecondsToClearTopicNameCache(), 2); + assertEquals(conf.getTopicNameCacheMaxCapacity(), 100); + } } diff --git a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf index 36f5869d73de6..551a9c88757a4 100644 --- a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf +++ b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf @@ -103,3 +103,5 @@ transactionPendingAckBatchedWriteEnabled=true transactionPendingAckBatchedWriteMaxRecords=44 transactionPendingAckBatchedWriteMaxSize=55 transactionPendingAckBatchedWriteMaxDelayInMillis=66 +topicNameCacheMaxCapacity=200 +maxSecondsToClearTopicNameCache=1 diff --git a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf index 0748418be6390..e9aeed1a34da9 100644 --- a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf +++ b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf @@ -94,3 +94,5 @@ brokerDeleteInactiveTopicsMode=delete_when_subscriptions_caught_up supportedNamespaceBundleSplitAlgorithms=[range_equally_divide] defaultNamespaceBundleSplitAlgorithm=topic_count_equally_divide maxMessagePublishBufferSizeInMB=-1 +topicNameCacheMaxCapacity=200 +maxSecondsToClearTopicNameCache=1 \ No newline at end of file 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 2298a8d06a880..d2b60d9256e9c 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 @@ -19,16 +19,11 @@ package org.apache.pulsar.common.naming; import com.google.common.base.Splitter; -import com.google.common.cache.CacheBuilder; -import com.google.common.cache.CacheLoader; -import com.google.common.cache.LoadingCache; -import com.google.common.util.concurrent.UncheckedExecutionException; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.util.List; import java.util.Objects; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; +import java.util.concurrent.ConcurrentHashMap; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.common.util.Codec; @@ -54,13 +49,17 @@ public class TopicName implements ServiceUnitId { private final int partitionIndex; - private static final LoadingCache cache = CacheBuilder.newBuilder().maximumSize(100000) - .expireAfterAccess(30, TimeUnit.MINUTES).build(new CacheLoader() { - @Override - public TopicName load(String name) throws Exception { - return new TopicName(name); - } - }); + private static final ConcurrentHashMap cache = new ConcurrentHashMap<>(); + + public static void clearIfReachedMaxCapacity(int maxCapacity) { + if (maxCapacity < 0) { + // Unlimited cache. + return; + } + if (cache.size() > maxCapacity) { + cache.clear(); + } + } public static TopicName get(String domain, NamespaceName namespaceName, String topic) { String name = domain + "://" + namespaceName.toString() + '/' + topic; @@ -79,11 +78,11 @@ public static TopicName get(String domain, String tenant, String cluster, String } public static TopicName get(String topic) { - try { - return cache.get(topic); - } catch (ExecutionException | UncheckedExecutionException e) { - throw (RuntimeException) e.getCause(); + TopicName tp = cache.get(topic); + if (tp != null) { + return tp; } + return cache.computeIfAbsent(topic, k -> new TopicName(k)); } public static TopicName getPartitionedTopicName(String topic) { From d63205c5e9a7f80ed17ad2aa39d84a6ea4e83ed0 Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Mon, 22 Jul 2024 20:16:12 +0800 Subject: [PATCH 627/699] fix code style (cherry picked from commit 18970684b172fc52a0eb795bca4ccb1959243144) --- .../main/java/org/apache/pulsar/common/naming/TopicName.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 d2b60d9256e9c..20b71eb189bd9 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 @@ -22,8 +22,8 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.util.List; import java.util.Objects; -import java.util.regex.Pattern; import java.util.concurrent.ConcurrentHashMap; +import java.util.regex.Pattern; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.common.util.Codec; From 40a797c15f0ce69f4e76eb0eacd25ef069719cd1 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 22 Jul 2024 08:45:23 +0800 Subject: [PATCH 628/699] [improve] [broker] high CPU usage caused by list topics under namespace (#23049) (cherry picked from commit 3e4f338e91877fb2e4592aa9abc3aced6d4e50c7) (cherry picked from commit 3f7206ca7857bed61a90941dc663c14dd93a0e75) --- .../broker/namespace/NamespaceService.java | 23 +++++++++++++++++++ .../pulsar/broker/service/ServerCnx.java | 4 ++-- .../pulsar/broker/service/ServerCnxTest.java | 2 ++ 3 files changed, 27 insertions(+), 2 deletions(-) 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 f977b6e561969..31529a22b51b0 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 @@ -37,6 +37,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -52,6 +53,7 @@ import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.collections4.ListUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.mutable.MutableBoolean; import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.PulsarService; @@ -100,6 +102,7 @@ import org.apache.pulsar.common.policies.data.Policies; import org.apache.pulsar.common.policies.data.stats.TopicStatsImpl; import org.apache.pulsar.common.policies.impl.NamespaceIsolationPolicies; +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; @@ -159,6 +162,9 @@ public class NamespaceService implements AutoCloseable { .register(); + private ConcurrentHashMap>> inProgressQueryUserTopics = + new ConcurrentHashMap<>(); + /** * Default constructor. */ @@ -1452,6 +1458,23 @@ public CompletableFuture> getListOfTopics(NamespaceName namespaceNa } } + public CompletableFuture> getListOfUserTopics(NamespaceName namespaceName, Mode mode) { + String key = String.format("%s://%s", mode, namespaceName); + final MutableBoolean initializedByCurrentThread = new MutableBoolean(); + CompletableFuture> queryRes = inProgressQueryUserTopics.computeIfAbsent(key, k -> { + initializedByCurrentThread.setTrue(); + return getListOfTopics(namespaceName, mode).thenApplyAsync(list -> { + return TopicList.filterSystemTopic(list); + }, pulsar.getExecutor()); + }); + if (initializedByCurrentThread.getValue()) { + queryRes.whenComplete((ignore, ex) -> { + inProgressQueryUserTopics.remove(key, queryRes); + }); + } + return queryRes; + } + public CompletableFuture> getAllPartitions(NamespaceName namespaceName) { return getPartitions(namespaceName, TopicDomain.persistent) .thenCombine(getPartitions(namespaceName, TopicDomain.non_persistent), 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 3f7c8b9c20a22..b36624b3c4491 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 @@ -2348,11 +2348,11 @@ protected void handleGetTopicsOfNamespace(CommandGetTopicsOfNamespace commandGet if (lookupSemaphore.tryAcquire()) { isNamespaceOperationAllowed(namespaceName, NamespaceOperation.GET_TOPICS).thenApply(isAuthorized -> { if (isAuthorized) { - getBrokerService().pulsar().getNamespaceService().getListOfTopics(namespaceName, mode) + getBrokerService().pulsar().getNamespaceService().getListOfUserTopics(namespaceName, mode) .thenAccept(topics -> { boolean filterTopics = false; // filter system topic - List filteredTopics = TopicList.filterSystemTopic(topics); + List filteredTopics = topics; if (enableSubscriptionPatternEvaluation && topicsPattern.isPresent()) { if (topicsPattern.get().length() <= maxSubscriptionPatternLength) { 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 626bce380b9de..3971cc5eda53c 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 @@ -229,6 +229,8 @@ public void setup() throws Exception { doReturn(CompletableFuture.completedFuture(true)).when(namespaceService).checkTopicOwnership(any()); doReturn(CompletableFuture.completedFuture(topics)).when(namespaceService).getListOfTopics( NamespaceName.get("use", "ns-abc"), CommandGetTopicsOfNamespace.Mode.ALL); + doReturn(CompletableFuture.completedFuture(topics)).when(namespaceService).getListOfUserTopics( + NamespaceName.get("use", "ns-abc"), CommandGetTopicsOfNamespace.Mode.ALL); doReturn(CompletableFuture.completedFuture(topics)).when(namespaceService).getListOfPersistentTopics( NamespaceName.get("use", "ns-abc")); From 86d4ea0cdc54ed24bd3516dbd5e4b74507facc57 Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Sat, 27 Jul 2024 11:43:43 -0700 Subject: [PATCH 629/699] [fix][broker][branch-3.0] Do not try to clean owned bundles from inactive source brokers (ExtensibleLoadManagerImpl only) (#23064) (#23077) (cherry picked from commit 55d32f2a9495e60ca5526499309582055e734e37) --- .../extensions/ExtensibleLoadManagerImpl.java | 27 ++- .../channel/ServiceUnitStateChannelImpl.java | 195 +++++++----------- .../pulsar/broker/service/BrokerService.java | 9 + .../channel/ServiceUnitStateChannelTest.java | 68 ++++-- 4 files changed, 156 insertions(+), 143 deletions(-) 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 fba0289367e7d..dc57a923c7adc 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 @@ -122,6 +122,9 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager { private static final String ELECTION_ROOT = "/loadbalance/extension/leader"; + private static final Set INTERNAL_TOPICS = + Set.of(BROKER_LOAD_DATA_STORE_TOPIC, TOP_BUNDLES_LOAD_DATA_STORE_TOPIC, TOPIC); + private PulsarService pulsar; private ServiceConfiguration conf; @@ -774,7 +777,8 @@ public void close() throws PulsarServerException { } public static boolean isInternalTopic(String topic) { - return topic.startsWith(TOPIC) + return INTERNAL_TOPICS.contains(topic) + || topic.startsWith(TOPIC) || topic.startsWith(BROKER_LOAD_DATA_STORE_TOPIC) || topic.startsWith(TOP_BUNDLES_LOAD_DATA_STORE_TOPIC); } @@ -932,5 +936,26 @@ public void disableBroker() throws Exception { serviceUnitStateChannel.cleanOwnerships(); leaderElectionService.close(); brokerRegistry.unregister(); + // 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(); + } + + private void closeInternalTopics() { + List> futures = new ArrayList<>(); + for (String name : INTERNAL_TOPICS) { + futures.add(pulsar.getBrokerService().getTopicIfExists(name) + .thenAccept(topicOptional -> topicOptional.ifPresent(topic -> topic.close(true))) + .exceptionally(__ -> { + log.warn("Failed to close internal topic:{}", name); + return null; + })); + } + try { + FutureUtil.waitForAll(futures) + .get(pulsar.getConfiguration().getNamespaceBundleUnloadingTimeoutMs(), TimeUnit.MILLISECONDS); + } catch (Throwable e) { + log.warn("Failed to wait for closing internal topics", e); + } } } 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 03c77033b0470..02e641f69a745 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 @@ -611,20 +611,13 @@ public CompletableFuture publishAssignEventAsync(String serviceUnit, Str } private CompletableFuture publishOverrideEventAsync(String serviceUnit, - ServiceUnitStateData orphanData, ServiceUnitStateData override) { if (!validateChannelState(Started, true)) { throw new IllegalStateException("Invalid channel state:" + channelState.name()); } EventType eventType = EventType.Override; eventCounters.get(eventType).getTotal().incrementAndGet(); - return pubAsync(serviceUnit, override).whenComplete((__, e) -> { - if (e != null) { - eventCounters.get(eventType).getFailure().incrementAndGet(); - log.error("Failed to override serviceUnit:{} from orphanData:{} to overrideData:{}", - serviceUnit, orphanData, override, e); - } - }).thenApply(__ -> null); + return pubAsync(serviceUnit, override).thenApply(__ -> null); } public CompletableFuture publishUnloadEventAsync(Unload unload) { @@ -1257,45 +1250,51 @@ private void scheduleCleanup(String broker, long delayInSecs) { broker, delayInSecs, cleanupJobs.size()); } - - private ServiceUnitStateData getOverrideInactiveBrokerStateData(ServiceUnitStateData orphanData, - Optional selectedBroker, - String inactiveBroker) { - - - if (selectedBroker.isEmpty()) { - return new ServiceUnitStateData(Free, null, inactiveBroker, - true, getNextVersionId(orphanData)); - } - - if (orphanData.state() == Splitting) { - return new ServiceUnitStateData(Splitting, orphanData.dstBroker(), selectedBroker.get(), - Map.copyOf(orphanData.splitServiceUnitToDestBroker()), - true, getNextVersionId(orphanData)); - } else { - return new ServiceUnitStateData(Owned, selectedBroker.get(), inactiveBroker, - true, getNextVersionId(orphanData)); - } - } - private void overrideOwnership(String serviceUnit, ServiceUnitStateData orphanData, String inactiveBroker) { - Optional selectedBroker = selectBroker(serviceUnit, inactiveBroker); - if (selectedBroker.isEmpty()) { - log.warn("Empty selected broker for ownership serviceUnit:{} orphanData:{}." + final var version = getNextVersionId(orphanData); + try { + selectBroker(serviceUnit, inactiveBroker) + .thenApply(selectedOpt -> + selectedOpt.map(selectedBroker -> { + if (orphanData.state() == Splitting) { + // if Splitting, set orphan.dstBroker() as dst to indicate where it was from. + // (The src broker runs handleSplitEvent.) + return new ServiceUnitStateData(Splitting, orphanData.dstBroker(), selectedBroker, + Map.copyOf(orphanData.splitServiceUnitToDestBroker()), true, version); + } else if (orphanData.state() == Owned) { + // if Owned, set orphan.dstBroker() as source to clean it up in case it is still + // alive. + return new ServiceUnitStateData(Owned, selectedBroker, + selectedBroker.equals(orphanData.dstBroker()) ? null : + orphanData.dstBroker(), + true, version); + } else { + // if Assigning or Releasing, set orphan.sourceBroker() as source + // to clean it up in case it is still alive. + return new ServiceUnitStateData(Owned, selectedBroker, + selectedBroker.equals(orphanData.sourceBroker()) ? null : + orphanData.sourceBroker(), + true, version); + } + // If no broker is selected(available), free the ownership. + // If the previous owner is still active, it will close the bundle(topic) ownership. + }).orElseGet(() -> new ServiceUnitStateData(Free, null, + orphanData.state() == Owned ? orphanData.dstBroker() : orphanData.sourceBroker(), + true, + version))) + .thenCompose(override -> { + log.info( + "Overriding inactiveBroker:{}, ownership serviceUnit:{} from orphanData:{} to " + + "overrideData:{}", + inactiveBroker, serviceUnit, orphanData, override); + return publishOverrideEventAsync(serviceUnit, override); + }).get(config.getMetadataStoreOperationTimeoutSeconds(), SECONDS); + } catch (Throwable e) { + log.error( + "Failed to override inactiveBroker:{} ownership serviceUnit:{} orphanData:{}. " + "totalCleanupErrorCnt:{}", - serviceUnit, orphanData, totalCleanupErrorCnt.incrementAndGet()); + inactiveBroker, serviceUnit, orphanData, totalCleanupErrorCnt.incrementAndGet(), e); } - var override = getOverrideInactiveBrokerStateData(orphanData, selectedBroker, inactiveBroker); - log.info("Overriding ownership serviceUnit:{} from orphanData:{} to overrideData:{}", - serviceUnit, orphanData, override); - publishOverrideEventAsync(serviceUnit, orphanData, override) - .exceptionally(e -> { - log.error( - "Failed to override the ownership serviceUnit:{} orphanData:{}. " - + "Failed to publish override event. totalCleanupErrorCnt:{}", - serviceUnit, orphanData, totalCleanupErrorCnt.incrementAndGet()); - return null; - }); } private void waitForCleanups(String broker, boolean excludeSystemTopics, int maxWaitTimeInMillis) { @@ -1411,61 +1410,14 @@ private synchronized void doCleanup(String broker) { } - private Optional selectBroker(String serviceUnit, String inactiveBroker) { - try { - return loadManager.selectAsync( - LoadManagerShared.getNamespaceBundle(pulsar, serviceUnit), - Set.of(inactiveBroker), LookupOptions.builder().build()) - .get(inFlightStateWaitingTimeInMillis, MILLISECONDS); - } catch (Throwable e) { - log.error("Failed to select a broker for serviceUnit:{}", serviceUnit); - } - return Optional.empty(); - } - - private Optional getRollForwardStateData(String serviceUnit, - String inactiveBroker, - long nextVersionId) { - Optional selectedBroker = selectBroker(serviceUnit, inactiveBroker); - if (selectedBroker.isEmpty()) { - return Optional.empty(); - } - return Optional.of(new ServiceUnitStateData(Owned, selectedBroker.get(), true, nextVersionId)); + private CompletableFuture> selectBroker(String serviceUnit, String inactiveBroker) { + return getLoadManager().selectAsync( + LoadManagerShared.getNamespaceBundle(pulsar, serviceUnit), + inactiveBroker == null ? Set.of() : Set.of(inactiveBroker), + LookupOptions.builder().build()); } - private Optional getOverrideInFlightStateData( - String serviceUnit, ServiceUnitStateData orphanData, - Set availableBrokers) { - long nextVersionId = getNextVersionId(orphanData); - var state = orphanData.state(); - switch (state) { - case Assigning: { - return getRollForwardStateData(serviceUnit, orphanData.dstBroker(), nextVersionId); - } - case Splitting: { - return Optional.of(new ServiceUnitStateData(Splitting, - orphanData.dstBroker(), orphanData.sourceBroker(), - Map.copyOf(orphanData.splitServiceUnitToDestBroker()), - true, nextVersionId)); - } - case Releasing: { - if (availableBrokers.contains(orphanData.sourceBroker())) { - // rollback to the src - return Optional.of(new ServiceUnitStateData(Owned, orphanData.sourceBroker(), true, nextVersionId)); - } else { - return getRollForwardStateData(serviceUnit, orphanData.sourceBroker(), nextVersionId); - } - } - default: { - var msg = String.format("Failed to get the overrideStateData from serviceUnit=%s, orphanData=%s", - serviceUnit, orphanData); - log.error(msg); - throw new IllegalStateException(msg); - } - } - } - @VisibleForTesting protected void monitorOwnerships(List brokers) { if (!isChannelOwner()) { @@ -1492,7 +1444,7 @@ protected void monitorOwnerships(List brokers) { long startTime = System.nanoTime(); Set inactiveBrokers = new HashSet<>(); Set activeBrokers = new HashSet<>(brokers); - Map orphanServiceUnits = new HashMap<>(); + Map timedOutInFlightStateServiceUnits = new HashMap<>(); int serviceUnitTombstoneCleanupCnt = 0; int orphanServiceUnitCleanupCnt = 0; long totalCleanupErrorCntStart = totalCleanupErrorCnt.get(); @@ -1504,20 +1456,27 @@ protected void monitorOwnerships(List brokers) { String srcBroker = stateData.sourceBroker(); var state = stateData.state(); - if (isActiveState(state) && StringUtils.isNotBlank(srcBroker) && !activeBrokers.contains(srcBroker)) { + if (state == Owned && (StringUtils.isBlank(dstBroker) || !activeBrokers.contains(dstBroker))) { + inactiveBrokers.add(dstBroker); + continue; + } + + if (isInFlightState(state) && StringUtils.isNotBlank(srcBroker) && !activeBrokers.contains(srcBroker)) { inactiveBrokers.add(srcBroker); continue; } - if (isActiveState(state) && StringUtils.isNotBlank(dstBroker) && !activeBrokers.contains(dstBroker)) { + if (isInFlightState(state) && StringUtils.isNotBlank(dstBroker) && !activeBrokers.contains(dstBroker)) { inactiveBrokers.add(dstBroker); continue; } - if (isActiveState(state) && isInFlightState(state) + + if (isInFlightState(state) && now - stateData.timestamp() > inFlightStateWaitingTimeInMillis) { - orphanServiceUnits.put(serviceUnit, stateData); + timedOutInFlightStateServiceUnits.put(serviceUnit, stateData); continue; } + if (!isActiveState(state) && now - stateData.timestamp() > semiTerminalStateWaitingTimeInMillis) { log.info("Found semi-terminal states to tombstone" + " serviceUnit:{}, stateData:{}", serviceUnit, stateData); @@ -1533,37 +1492,21 @@ protected void monitorOwnerships(List brokers) { } } - // Skip cleaning orphan bundles if inactiveBrokers exist. This is a bigger problem. + if (!inactiveBrokers.isEmpty()) { for (String inactiveBroker : inactiveBrokers) { handleBrokerDeletionEvent(inactiveBroker); } - } else if (!orphanServiceUnits.isEmpty()) { - for (var etr : orphanServiceUnits.entrySet()) { + } + + // timedOutInFlightStateServiceUnits are the in-flight ones although their src and dst brokers are known to + // be active. + if (!timedOutInFlightStateServiceUnits.isEmpty()) { + for (var etr : timedOutInFlightStateServiceUnits.entrySet()) { var orphanServiceUnit = etr.getKey(); var orphanData = etr.getValue(); - var overrideData = getOverrideInFlightStateData( - orphanServiceUnit, orphanData, activeBrokers); - if (overrideData.isPresent()) { - log.info("Overriding in-flight state ownership serviceUnit:{} " - + "from orphanData:{} to overrideData:{}", - orphanServiceUnit, orphanData, overrideData); - publishOverrideEventAsync(orphanServiceUnit, orphanData, overrideData.get()) - .whenComplete((__, e) -> { - if (e != null) { - log.error("Failed cleaning the ownership orphanServiceUnit:{}, orphanData:{}, " - + "cleanupErrorCnt:{}.", - orphanServiceUnit, orphanData, - totalCleanupErrorCnt.incrementAndGet() - totalCleanupErrorCntStart, e); - } - }); - orphanServiceUnitCleanupCnt++; - } else { - log.warn("Failed get the overrideStateData from orphanServiceUnit:{}, orphanData:{}," - + " cleanupErrorCnt:{}. will retry..", - orphanServiceUnit, orphanData, - totalCleanupErrorCnt.incrementAndGet() - totalCleanupErrorCntStart); - } + overrideOwnership(orphanServiceUnit, orphanData, null); + orphanServiceUnitCleanupCnt++; } } 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 410918a7ac7e7..9023a1ffdf255 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 @@ -2294,6 +2294,15 @@ private CompletableFuture unloadServiceUnit(NamespaceBundle serviceUnit topics.forEach((name, topicFuture) -> { TopicName topicName = TopicName.get(name); if (serviceUnit.includes(topicName)) { + if (ExtensibleLoadManagerImpl.isLoadManagerExtensionEnabled(pulsar) + && ExtensibleLoadManagerImpl.isInternalTopic(topicName.toString())) { + if (ExtensibleLoadManagerImpl.debug(pulsar.getConfiguration(), log)) { + log.info("[{}] Skip unloading ExtensibleLoadManager internal topics. Such internal topic " + + "should be closed when shutting down the broker.", topicName); + } + return; + } + // Topic needs to be unloaded log.info("[{}] Unloading topic", topicName); if (topicFuture.isCompletedExceptionally()) { 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 0b7bcb34d440e..03c9bed2d128f 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 @@ -108,6 +108,7 @@ public class ServiceUnitStateChannelTest extends MockedPulsarServiceBaseTest { private ServiceUnitStateChannel channel2; private String brokerId1; private String brokerId2; + private String brokerId3; private String bundle; private String bundle1; private String bundle2; @@ -159,6 +160,7 @@ protected void setup() throws Exception { FieldUtils.readDeclaredField(channel1, "brokerId", true); brokerId2 = (String) FieldUtils.readDeclaredField(channel2, "brokerId", true); + brokerId3 = "broker-3"; bundle = "public/default/0x00000000_0xffffffff"; bundle1 = "public/default/0x00000000_0xfffffff0"; @@ -1232,7 +1234,8 @@ public void splitTestWhenProducerFails() var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; - + doReturn(CompletableFuture.completedFuture(Optional.of(brokerId1))) + .when(loadManager).selectAsync(any(), any(), any()); waitUntilStateWithMonitor(leader, bundle, Init); waitUntilStateWithMonitor(channel1, bundle, Init); waitUntilStateWithMonitor(channel2, bundle, Init); @@ -1423,6 +1426,8 @@ public void splitAndRetryFailureTest() throws Exception { assertEquals(3, count.get()); }); var leader = channel1.isChannelOwnerAsync().get() ? channel1 : channel2; + doReturn(CompletableFuture.completedFuture(Optional.of(brokerId1))) + .when(loadManager).selectAsync(any(), any(), any()); ((ServiceUnitStateChannelImpl) leader) .monitorOwnerships(List.of(brokerId1, brokerId2)); @@ -1566,26 +1571,40 @@ public void testOverrideOrphanStateData() String broker = brokerId1; // test override states - String releasingBundle = "public/releasing/0xfffffff0_0xffffffff"; + String releasingBundle1 = "public/releasing1/0xfffffff0_0xffffffff"; + String releasingBundle2 = "public/releasing2/0xfffffff0_0xffffffff"; String splittingBundle = bundle; - String assigningBundle = "public/assigning/0xfffffff0_0xffffffff"; + String assigningBundle1 = "public/assigning1/0xfffffff0_0xffffffff"; + String assigningBundle2 = "public/assigning2/0xfffffff0_0xffffffff"; String freeBundle = "public/free/0xfffffff0_0xffffffff"; String deletedBundle = "public/deleted/0xfffffff0_0xffffffff"; - String ownedBundle = "public/owned/0xfffffff0_0xffffffff"; - overrideTableViews(releasingBundle, - new ServiceUnitStateData(Releasing, null, broker, 1)); + String ownedBundle1 = "public/owned1/0xfffffff0_0xffffffff"; + String ownedBundle2 = "public/owned2SourceBundle/0xfffffff0_0xffffffff"; + String ownedBundle3 = "public/owned3/0xfffffff0_0xffffffff"; + String inactiveBroker = "broker-inactive-1"; + overrideTableViews(releasingBundle1, + new ServiceUnitStateData(Releasing, broker, brokerId2, 1)); + overrideTableViews(releasingBundle2, + new ServiceUnitStateData(Releasing, brokerId2, brokerId3, 1)); overrideTableViews(splittingBundle, new ServiceUnitStateData(Splitting, null, broker, Map.of(childBundle1Range, Optional.empty(), childBundle2Range, Optional.empty()), 1)); - overrideTableViews(assigningBundle, + overrideTableViews(assigningBundle1, new ServiceUnitStateData(Assigning, broker, null, 1)); + overrideTableViews(assigningBundle2, + new ServiceUnitStateData(Assigning, broker, brokerId2, 1)); overrideTableViews(freeBundle, new ServiceUnitStateData(Free, null, broker, 1)); overrideTableViews(deletedBundle, new ServiceUnitStateData(Deleted, null, broker, 1)); - overrideTableViews(ownedBundle, + overrideTableViews(ownedBundle1, new ServiceUnitStateData(Owned, broker, null, 1)); + overrideTableViews(ownedBundle2, + new ServiceUnitStateData(Owned, broker, inactiveBroker, 1)); + overrideTableViews(ownedBundle3, + new ServiceUnitStateData(Owned, inactiveBroker, broker, 1)); + // test stable metadata state doReturn(CompletableFuture.completedFuture(Optional.of(brokerId2))) @@ -1595,16 +1614,33 @@ public void testOverrideOrphanStateData() FieldUtils.writeDeclaredField(followerChannel, "inFlightStateWaitingTimeInMillis", -1, true); ((ServiceUnitStateChannelImpl) leaderChannel) - .monitorOwnerships(List.of(brokerId1, brokerId2)); + .monitorOwnerships(List.of(brokerId1, brokerId2, "broker-3")); - waitUntilNewOwner(channel2, releasingBundle, broker); - waitUntilNewOwner(channel2, childBundle11, broker); - waitUntilNewOwner(channel2, childBundle12, broker); - waitUntilNewOwner(channel2, assigningBundle, brokerId2); - waitUntilNewOwner(channel2, ownedBundle, broker); - assertEquals(Optional.empty(), channel2.getOwnerAsync(freeBundle).get()); - assertTrue(channel2.getOwnerAsync(deletedBundle).isCompletedExceptionally()); + ServiceUnitStateChannel finalLeaderChannel = leaderChannel; + Awaitility.await().atMost(5, TimeUnit.SECONDS).until(() -> getCleanupJobs(finalLeaderChannel).isEmpty()); + + + waitUntilNewOwner(channel2, releasingBundle1, brokerId2); + waitUntilNewOwner(channel2, releasingBundle2, brokerId2); assertTrue(channel2.getOwnerAsync(splittingBundle).get().isEmpty()); + waitUntilNewOwner(channel2, childBundle11, brokerId2); + waitUntilNewOwner(channel2, childBundle12, brokerId2); + waitUntilNewOwner(channel2, assigningBundle1, brokerId2); + waitUntilNewOwner(channel2, assigningBundle2, brokerId2); + assertTrue(channel2.getOwnerAsync(freeBundle).get().isEmpty()); + assertTrue(channel2.getOwnerAsync(deletedBundle).isCompletedExceptionally()); + waitUntilNewOwner(channel2, ownedBundle1, broker); + waitUntilNewOwner(channel2, ownedBundle2, broker); + waitUntilNewOwner(channel2, ownedBundle3, brokerId2); + + validateMonitorCounters(leaderChannel, + 1, + 0, + 6, + 0, + 1, + 0, + 0); // clean-up FieldUtils.writeDeclaredField(channel1, From 8124284a3283e8c042c1c988832ea4a2782b1aff Mon Sep 17 00:00:00 2001 From: congbo <39078850+congbobo184@users.noreply.github.com> Date: Mon, 22 Jul 2024 16:43:38 +0800 Subject: [PATCH 630/699] [improve][broker] GetPartitionMetadata fail also can produce messages (#23050) ### Motivation GetPartitionMetadata fail also can produce messages - 'autoUpdatePartitionsInterval' will get partition metadata and will regularly detect partition changes - if GetPartitionMetadata will return ServiceNotReady, client receive ServiceNotReady will close cnx - if close the current cnx, all producers and consumers witch use this cnx will close and reconnect (https://github.com/apache/pulsar/blob/5c6602cbb3660a696bf960f2847aac1a2ae037d2/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java#L323-L345) - this will influence a lot of producers and consumers and if current time the zk not available and bundle cache not exist the topic's bundle metadata, the client can't send messages to broker because the producer lookup will fail ### Modifications GetPartitionMetadata return MetadataError when throw MetadataStoreException (cherry picked from commit 6fa3bcfe082e6662733928aa49bee1fcce217c80) (cherry picked from commit 3b3e90b7909a6e4c943c5765f6a990781689253c) --- .../pulsar/broker/service/ServerCnx.java | 5 +- .../broker/zookeeper/ZKReconnectTest.java | 87 +++++++++++++++++++ 2 files changed, 91 insertions(+), 1 deletion(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ZKReconnectTest.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 b36624b3c4491..e7a318e86c57e 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 @@ -172,6 +172,7 @@ import org.apache.pulsar.common.util.netty.NettyChannelUtil; import org.apache.pulsar.common.util.netty.NettyFutureUtil; import org.apache.pulsar.functions.utils.Exceptions; +import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.transaction.coordinator.TransactionCoordinatorID; import org.apache.pulsar.transaction.coordinator.exceptions.CoordinatorException; import org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStore; @@ -631,7 +632,9 @@ protected void handlePartitionMetadataRequest(CommandPartitionedTopicMetadata pa log.warn("Failed to get Partitioned Metadata [{}] {}: {}", remoteAddress, topicName, ex.getMessage(), ex); ServerError error = ServerError.ServiceNotReady; - if (ex instanceof RestException restException){ + if (ex instanceof MetadataStoreException) { + error = ServerError.MetadataError; + } else if (ex instanceof RestException restException){ int responseCode = restException.getResponse().getStatus(); if (responseCode == NOT_FOUND.getStatusCode()){ error = ServerError.TopicNotFound; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ZKReconnectTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ZKReconnectTest.java new file mode 100644 index 0000000000000..7b9e4beec6bdf --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/zookeeper/ZKReconnectTest.java @@ -0,0 +1,87 @@ +/* + * 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.zookeeper; + +import com.google.common.collect.Sets; +import org.apache.pulsar.broker.MetadataSessionExpiredPolicy; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.zookeeper.KeeperException; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.util.concurrent.TimeUnit; + + +@Test +public class ZKReconnectTest extends MockedPulsarServiceBaseTest { + + @BeforeMethod(alwaysRun = true) + @Override + protected void setup() throws Exception { + this.conf.setZookeeperSessionExpiredPolicy(MetadataSessionExpiredPolicy.reconnect); + this.internalSetup(); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); + admin.tenants().createTenant("public", + new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("test"))); + admin.namespaces().createNamespace("public/default"); + admin.namespaces().setNamespaceReplicationClusters("public/default", Sets.newHashSet("test")); + } + + @Test + public void testGetPartitionMetadataFailAlsoCanProduceMessage() throws Exception { + + pulsarClient = PulsarClient.builder(). + serviceUrl(pulsar.getBrokerServiceUrl()) + .build(); + + String topic = "testGetPartitionMetadataFailAlsoCanProduceMessage"; + admin.topics().createPartitionedTopic(topic, 5); + Producer producer = pulsarClient.newProducer() + .autoUpdatePartitionsInterval(1, TimeUnit.SECONDS).topic(topic).create(); + + this.mockZooKeeper.setAlwaysFail(KeeperException.Code.SESSIONEXPIRED); + + // clear cache + pulsar.getPulsarResources().getNamespaceResources().getPartitionedTopicResources() + .getCache().delete("/admin/partitioned-topics/public/default/persistent" + + "/testGetPartitionMetadataFailAlsoCanProduceMessage"); + pulsar.getNamespaceService().getOwnershipCache().invalidateLocalOwnerCache(); + + // autoUpdatePartitions 1 second + TimeUnit.SECONDS.sleep(3); + + // also can send message + producer.send("test".getBytes()); + this.mockZooKeeper.unsetAlwaysFail(); + producer.send("test".getBytes()); + producer.close(); + } + + + @AfterMethod(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + this.internalCleanup(); + } +} From 4b715b61152640d36b9ea97a53279c4eb15641bf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Wed, 24 Jul 2024 14:40:03 +0800 Subject: [PATCH 631/699] [fix][broker] Handle BucketDelayedDeliveryTracker recover failed (#22735) (cherry picked from commit 1c53841cc7f585bdd8ff6702d74f37491d8cc9c6) (cherry picked from commit 257ca7f6286dd39009b675f794e324adafcde985) --- .../BucketDelayedDeliveryTrackerFactory.java | 30 ++- .../delayed/DelayedDeliveryTracker.java | 47 ++++ ...InMemoryDelayedDeliveryTrackerFactory.java | 19 ++ .../bucket/BucketDelayedDeliveryTracker.java | 35 ++- ...ecoverDelayedDeliveryTrackerException.java | 25 ++ .../pulsar/broker/service/BrokerService.java | 28 +- ...PersistentDispatcherMultipleConsumers.java | 11 +- .../DelayedDeliveryTrackerFactoryTest.java | 242 ++++++++++++++++++ .../BucketDelayedDeliveryTrackerTest.java | 6 +- 9 files changed, 420 insertions(+), 23 deletions(-) create mode 100644 pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/RecoverDelayedDeliveryTrackerException.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTrackerFactoryTest.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 33076fd51a8e9..69a08bd2be4a9 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 @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.delayed; +import com.google.common.annotations.VisibleForTesting; import io.netty.util.HashedWheelTimer; import io.netty.util.Timer; import io.netty.util.concurrent.DefaultThreadFactory; @@ -33,10 +34,15 @@ import org.apache.pulsar.broker.delayed.bucket.BookkeeperBucketSnapshotStorage; import org.apache.pulsar.broker.delayed.bucket.BucketDelayedDeliveryTracker; 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.common.util.FutureUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class BucketDelayedDeliveryTrackerFactory implements DelayedDeliveryTrackerFactory { + private static final Logger log = LoggerFactory.getLogger(BucketDelayedDeliveryTrackerFactory.class); BucketSnapshotStorage bucketSnapshotStorage; @@ -73,8 +79,28 @@ public void initialize(PulsarService pulsarService) throws Exception { @Override public DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers dispatcher) { - return new BucketDelayedDeliveryTracker(dispatcher, timer, tickTimeMillis, isDelayedDeliveryDeliverAtTimeStrict, - bucketSnapshotStorage, delayedDeliveryMinIndexCountPerBucket, + String topicName = dispatcher.getTopic().getName(); + String subscriptionName = dispatcher.getSubscription().getName(); + BrokerService brokerService = dispatcher.getTopic().getBrokerService(); + DelayedDeliveryTracker tracker; + + try { + tracker = newTracker0(dispatcher); + } catch (RecoverDelayedDeliveryTrackerException ex) { + log.warn("Failed to recover BucketDelayedDeliveryTracker, fallback to InMemoryDelayedDeliveryTracker." + + " topic {}, subscription {}", topicName, subscriptionName, ex); + // If failed to create BucketDelayedDeliveryTracker, fallback to InMemoryDelayedDeliveryTracker + brokerService.initializeFallbackDelayedDeliveryTrackerFactory(); + tracker = brokerService.getFallbackDelayedDeliveryTrackerFactory().newTracker(dispatcher); + } + return tracker; + } + + @VisibleForTesting + BucketDelayedDeliveryTracker newTracker0(PersistentDispatcherMultipleConsumers dispatcher) + throws RecoverDelayedDeliveryTrackerException { + return new BucketDelayedDeliveryTracker(dispatcher, timer, tickTimeMillis, + isDelayedDeliveryDeliverAtTimeStrict, bucketSnapshotStorage, delayedDeliveryMinIndexCountPerBucket, TimeUnit.SECONDS.toMillis(delayedDeliveryMaxTimeStepPerBucketSnapshotSegmentSeconds), delayedDeliveryMaxIndexesPerBucketSnapshotSegment, delayedDeliveryMaxNumBuckets); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java index 78229fef25a5a..893a4e59c3cde 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/DelayedDeliveryTracker.java @@ -85,4 +85,51 @@ public interface DelayedDeliveryTracker extends AutoCloseable { * Close the subscription tracker and release all resources. */ void close(); + + DelayedDeliveryTracker DISABLE = new DelayedDeliveryTracker() { + @Override + public boolean addMessage(long ledgerId, long entryId, long deliveryAt) { + return false; + } + + @Override + public boolean hasMessageAvailable() { + return false; + } + + @Override + public long getNumberOfDelayedMessages() { + return 0; + } + + @Override + public long getBufferMemoryUsage() { + return 0; + } + + @Override + public NavigableSet getScheduledMessages(int maxMessages) { + return null; + } + + @Override + public boolean shouldPauseAllDeliveries() { + return false; + } + + @Override + public void resetTickTime(long tickTime) { + + } + + @Override + public CompletableFuture clear() { + return null; + } + + @Override + public void close() { + + } + }; } 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 e7dc3f18f4630..179cf74db4179 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 @@ -18,6 +18,7 @@ */ package org.apache.pulsar.broker.delayed; +import com.google.common.annotations.VisibleForTesting; import io.netty.util.HashedWheelTimer; import io.netty.util.Timer; import io.netty.util.concurrent.DefaultThreadFactory; @@ -25,8 +26,11 @@ import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.service.persistent.PersistentDispatcherMultipleConsumers; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class InMemoryDelayedDeliveryTrackerFactory implements DelayedDeliveryTrackerFactory { + private static final Logger log = LoggerFactory.getLogger(InMemoryDelayedDeliveryTrackerFactory.class); private Timer timer; @@ -48,6 +52,21 @@ public void initialize(PulsarService pulsarService) { @Override public DelayedDeliveryTracker newTracker(PersistentDispatcherMultipleConsumers dispatcher) { + String topicName = dispatcher.getTopic().getName(); + String subscriptionName = dispatcher.getSubscription().getName(); + DelayedDeliveryTracker tracker = DelayedDeliveryTracker.DISABLE; + try { + tracker = newTracker0(dispatcher); + } catch (Exception e) { + // it should never go here + log.warn("Failed to create InMemoryDelayedDeliveryTracker, topic {}, subscription {}", + topicName, subscriptionName, e); + } + return tracker; + } + + @VisibleForTesting + InMemoryDelayedDeliveryTracker newTracker0(PersistentDispatcherMultipleConsumers 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 f98c9e000f150..1cbae674bd512 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 @@ -105,22 +105,24 @@ public class BucketDelayedDeliveryTracker extends AbstractDelayedDeliveryTracker private CompletableFuture pendingLoad = null; public BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, - Timer timer, long tickTimeMillis, - boolean isDelayedDeliveryDeliverAtTimeStrict, - BucketSnapshotStorage bucketSnapshotStorage, - long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegmentInMillis, - int maxIndexesPerBucketSnapshotSegment, int maxNumBuckets) { + Timer timer, long tickTimeMillis, + boolean isDelayedDeliveryDeliverAtTimeStrict, + BucketSnapshotStorage bucketSnapshotStorage, + long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegmentInMillis, + int maxIndexesPerBucketSnapshotSegment, int maxNumBuckets) + throws RecoverDelayedDeliveryTrackerException { this(dispatcher, timer, tickTimeMillis, Clock.systemUTC(), isDelayedDeliveryDeliverAtTimeStrict, bucketSnapshotStorage, minIndexCountPerBucket, timeStepPerBucketSnapshotSegmentInMillis, maxIndexesPerBucketSnapshotSegment, maxNumBuckets); } public BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispatcher, - Timer timer, long tickTimeMillis, Clock clock, - boolean isDelayedDeliveryDeliverAtTimeStrict, - BucketSnapshotStorage bucketSnapshotStorage, - long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegmentInMillis, - int maxIndexesPerBucketSnapshotSegment, int maxNumBuckets) { + Timer timer, long tickTimeMillis, Clock clock, + boolean isDelayedDeliveryDeliverAtTimeStrict, + BucketSnapshotStorage bucketSnapshotStorage, + long minIndexCountPerBucket, long timeStepPerBucketSnapshotSegmentInMillis, + int maxIndexesPerBucketSnapshotSegment, int maxNumBuckets) + throws RecoverDelayedDeliveryTrackerException { super(dispatcher, timer, tickTimeMillis, clock, isDelayedDeliveryDeliverAtTimeStrict); this.minIndexCountPerBucket = minIndexCountPerBucket; this.timeStepPerBucketSnapshotSegmentInMillis = timeStepPerBucketSnapshotSegmentInMillis; @@ -133,10 +135,17 @@ public BucketDelayedDeliveryTracker(PersistentDispatcherMultipleConsumers dispat new MutableBucket(dispatcher.getName(), dispatcher.getCursor(), FutureUtil.Sequencer.create(), bucketSnapshotStorage); this.stats = new BucketDelayedMessageIndexStats(); - this.numberDelayedMessages = recoverBucketSnapshot(); + + // Close the tracker if failed to recover. + try { + this.numberDelayedMessages = recoverBucketSnapshot(); + } catch (RecoverDelayedDeliveryTrackerException e) { + close(); + throw e; + } } - private synchronized long recoverBucketSnapshot() throws RuntimeException { + private synchronized long recoverBucketSnapshot() throws RecoverDelayedDeliveryTrackerException { ManagedCursor cursor = this.lastMutableBucket.getCursor(); Map cursorProperties = cursor.getCursorProperties(); if (MapUtils.isEmpty(cursorProperties)) { @@ -181,7 +190,7 @@ private synchronized long recoverBucketSnapshot() throws RuntimeException { if (e instanceof InterruptedException) { Thread.currentThread().interrupt(); } - throw new RuntimeException(e); + throw new RecoverDelayedDeliveryTrackerException(e); } for (Map.Entry, CompletableFuture>> entry : futures.entrySet()) { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/RecoverDelayedDeliveryTrackerException.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/RecoverDelayedDeliveryTrackerException.java new file mode 100644 index 0000000000000..71a851100fe4e --- /dev/null +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/delayed/bucket/RecoverDelayedDeliveryTrackerException.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.broker.delayed.bucket; + +public class RecoverDelayedDeliveryTrackerException extends Exception { + public RecoverDelayedDeliveryTrackerException(Throwable cause) { + super(cause); + } +} 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 9023a1ffdf255..8fffce88fb961 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 @@ -104,6 +104,7 @@ import org.apache.pulsar.broker.cache.BundlesQuotas; import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.delayed.DelayedDeliveryTrackerLoader; +import org.apache.pulsar.broker.delayed.InMemoryDelayedDeliveryTrackerFactory; import org.apache.pulsar.broker.intercept.BrokerInterceptor; import org.apache.pulsar.broker.intercept.ManagedLedgerInterceptorImpl; import org.apache.pulsar.broker.loadbalance.LoadManager; @@ -272,10 +273,11 @@ public class BrokerService implements Closeable { private final AtomicBoolean blockedDispatcherOnHighUnackedMsgs = new AtomicBoolean(false); private final ConcurrentOpenHashSet blockedDispatchers; private final ReadWriteLock lock = new ReentrantReadWriteLock(); - - @Getter @VisibleForTesting private final DelayedDeliveryTrackerFactory delayedDeliveryTrackerFactory; + // InMemoryDelayedDeliveryTrackerFactory is for the purpose of + // fallback if recover BucketDelayedDeliveryTracker failed. + private volatile DelayedDeliveryTrackerFactory fallbackDelayedDeliveryTrackerFactory; private final ServerBootstrap defaultServerBootstrap; private final List protocolHandlersWorkerGroups = new ArrayList<>(); @@ -884,6 +886,9 @@ public CompletableFuture closeAsync() { pulsarStats.close(); try { delayedDeliveryTrackerFactory.close(); + if (fallbackDelayedDeliveryTrackerFactory != null) { + fallbackDelayedDeliveryTrackerFactory.close(); + } } catch (Exception e) { log.warn("Error in closing delayedDeliveryTrackerFactory", e); } @@ -3442,6 +3447,25 @@ public void unblockDispatchersOnUnAckMessages(List pair = + mockDelayedDeliveryTrackerFactoryAndDispatcher(); + BrokerService brokerService = pair.getLeft(); + PersistentDispatcherMultipleConsumers dispatcher = pair.getRight(); + + // Since Mocked BucketDelayedDeliveryTrackerFactory.newTracker0() throws RecoverDelayedDeliveryTrackerException, + // the factory should be fallback to InMemoryDelayedDeliveryTrackerFactory + @Cleanup + DelayedDeliveryTracker tracker = brokerService.getDelayedDeliveryTrackerFactory().newTracker(dispatcher); + Assert.assertTrue(tracker instanceof InMemoryDelayedDeliveryTracker); + + DelayedDeliveryTrackerFactory fallbackFactory = brokerService.getFallbackDelayedDeliveryTrackerFactory(); + Assert.assertTrue(fallbackFactory instanceof InMemoryDelayedDeliveryTrackerFactory); + } + + + private Pair mockDelayedDeliveryTrackerFactoryAndDispatcher() + throws Exception { + BrokerService brokerService = Mockito.spy(pulsar.getBrokerService()); + + // Mock dispatcher + PersistentDispatcherMultipleConsumers dispatcher = Mockito.mock(PersistentDispatcherMultipleConsumers.class); + Mockito.doReturn("test").when(dispatcher).getName(); + // Mock BucketDelayedDeliveryTrackerFactory + @Cleanup + BucketDelayedDeliveryTrackerFactory factory = new BucketDelayedDeliveryTrackerFactory(); + factory = Mockito.spy(factory); + factory.initialize(pulsar); + Mockito.doThrow(new RecoverDelayedDeliveryTrackerException(new RuntimeException())) + .when(factory).newTracker0(Mockito.eq(dispatcher)); + // Mock brokerService + Mockito.doReturn(factory).when(brokerService).getDelayedDeliveryTrackerFactory(); + // Mock topic and subscription + PersistentTopic topic = Mockito.mock(PersistentTopic.class); + Mockito.doReturn(brokerService).when(topic).getBrokerService(); + Subscription subscription = Mockito.mock(Subscription.class); + Mockito.doReturn("topic").when(topic).getName(); + Mockito.doReturn("sub").when(subscription).getName(); + Mockito.doReturn(topic).when(dispatcher).getTopic(); + Mockito.doReturn(subscription).when(dispatcher).getSubscription(); + + return Pair.of(brokerService, dispatcher); + } + + @Test + public void testFallbackToInMemoryTrackerFactoryFailed() throws Exception { + Pair pair = + mockDelayedDeliveryTrackerFactoryAndDispatcher(); + BrokerService brokerService = pair.getLeft(); + PersistentDispatcherMultipleConsumers dispatcher = pair.getRight(); + + // Mock InMemoryDelayedDeliveryTrackerFactory + @Cleanup + InMemoryDelayedDeliveryTrackerFactory factory = new InMemoryDelayedDeliveryTrackerFactory(); + factory = Mockito.spy(factory); + factory.initialize(pulsar); + // Mock InMemoryDelayedDeliveryTrackerFactory.newTracker0() throws RuntimeException + Mockito.doThrow(new RuntimeException()).when(factory).newTracker0(Mockito.eq(dispatcher)); + + // Mock brokerService to return mocked InMemoryDelayedDeliveryTrackerFactory + Mockito.doAnswer(inv -> null).when(brokerService).initializeFallbackDelayedDeliveryTrackerFactory(); + Mockito.doReturn(factory).when(brokerService).getFallbackDelayedDeliveryTrackerFactory(); + + // Since Mocked BucketDelayedDeliveryTrackerFactory.newTracker0() throws RecoverDelayedDeliveryTrackerException, + // and Mocked InMemoryDelayedDeliveryTrackerFactory.newTracker0() throws RuntimeException, + // the tracker instance should be DelayedDeliveryTracker.DISABLE + @Cleanup + DelayedDeliveryTracker tracker = brokerService.getDelayedDeliveryTrackerFactory().newTracker(dispatcher); + Assert.assertEquals(tracker, DelayedDeliveryTracker.DISABLE); + } + + // 1. Create BucketDelayedDeliveryTracker failed, fallback to InMemoryDelayedDeliveryTracker, + // 2. Publish delay messages + @Test(timeOut = 60_000) + public void testPublishDelayMessagesAndCreateBucketDelayDeliveryTrackerFailed() throws Exception { + String topicName = "persistent://public/default/" + UUID.randomUUID(); + + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .topic(topicName) + .enableBatching(false) + .create(); + + // Mock BucketDelayedDeliveryTrackerFactory.newTracker0() throws RecoverDelayedDeliveryTrackerException + PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get(); + topic = Mockito.spy(topic); + BrokerService brokerService = Mockito.spy(pulsar.getBrokerService()); + BucketDelayedDeliveryTrackerFactory factory = + (BucketDelayedDeliveryTrackerFactory) Mockito.spy(brokerService.getDelayedDeliveryTrackerFactory()); + Mockito.doThrow(new RecoverDelayedDeliveryTrackerException(new RuntimeException())) + .when(factory).newTracker0(Mockito.any()); + Mockito.doReturn(factory).when(brokerService).getDelayedDeliveryTrackerFactory(); + + // Return mocked BrokerService + Mockito.doReturn(brokerService).when(topic).getBrokerService(); + + // Set Mocked topic to BrokerService + Field topics = BrokerService.class.getDeclaredField("topics"); + topics.setAccessible(true); + @SuppressWarnings("unchecked") + ConcurrentOpenHashMap>> topicMap = + (ConcurrentOpenHashMap>>) topics.get(brokerService); + topicMap.put(topicName, CompletableFuture.completedFuture(Optional.of(topic))); + + // Create consumer + Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .topic(topicName) + .subscriptionName("sub") + .subscriptionType(SubscriptionType.Shared) + .messageListener((c, msg) -> { + try { + c.acknowledge(msg); + } catch (PulsarClientException e) { + throw new RuntimeException(e); + } + }) + .subscribe(); + + PersistentSubscription subscription = topic.getSubscription("sub"); + Dispatcher dispatcher = subscription.getDispatcher(); + Assert.assertTrue(dispatcher instanceof PersistentDispatcherMultipleConsumers); + + // Publish a delay message to initialize DelayedDeliveryTracker + producer.newMessage().value("test").deliverAfter(10_000, TimeUnit.MILLISECONDS).send(); + + // Get DelayedDeliveryTracker from Dispatcher + PersistentDispatcherMultipleConsumers dispatcher0 = (PersistentDispatcherMultipleConsumers) dispatcher; + Field trackerField = + PersistentDispatcherMultipleConsumers.class.getDeclaredField("delayedDeliveryTracker"); + trackerField.setAccessible(true); + + AtomicReference> reference = new AtomicReference<>(); + // Wait until DelayedDeliveryTracker is initialized + Awaitility.await().atMost(Duration.ofSeconds(20)).until(() -> { + @SuppressWarnings("unchecked") + Optional optional = + (Optional) trackerField.get(dispatcher0); + if (optional.isPresent()) { + reference.set(optional); + return true; + } + return false; + }); + + Optional optional = reference.get(); + Assert.assertTrue(optional.get() instanceof InMemoryDelayedDeliveryTracker); + + // Mock DelayedDeliveryTracker and Count the number of addMessage() calls + AtomicInteger counter = new AtomicInteger(0); + InMemoryDelayedDeliveryTracker tracker = (InMemoryDelayedDeliveryTracker) optional.get(); + tracker = Mockito.spy(tracker); + Mockito.doAnswer(inv -> { + counter.incrementAndGet(); + return inv.callRealMethod(); + }).when(tracker).addMessage(Mockito.anyLong(), Mockito.anyLong(), Mockito.anyLong()); + // Set Mocked InMemoryDelayedDeliveryTracker back to Dispatcher + trackerField.set(dispatcher0, Optional.of(tracker)); + + // Publish 10 delay messages, so the counter should be 10 + for (int i = 0; i < 10; i++) { + producer.newMessage().value("test") + .deliverAfter(10_000, TimeUnit.MILLISECONDS).send(); + } + + try { + Awaitility.await().atMost(Duration.ofSeconds(20)).until(() -> counter.get() == 10); + } finally { + consumer.close(); + } + } +} 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 1e3e72aa0ec44..eef10e49cbaf5 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 @@ -182,7 +182,7 @@ public void testContainsMessage(BucketDelayedDeliveryTracker tracker) { } @Test(dataProvider = "delayedTracker", invocationCount = 10) - public void testRecoverSnapshot(BucketDelayedDeliveryTracker tracker) { + public void testRecoverSnapshot(BucketDelayedDeliveryTracker tracker) throws Exception { for (int i = 1; i <= 100; i++) { tracker.addMessage(i, i, i * 10); } @@ -265,7 +265,7 @@ public void testRoaringBitmapSerialize() { } @Test(dataProvider = "delayedTracker") - public void testMergeSnapshot(final BucketDelayedDeliveryTracker tracker) { + public void testMergeSnapshot(final BucketDelayedDeliveryTracker tracker) throws Exception { for (int i = 1; i <= 110; i++) { tracker.addMessage(i, i, i * 10); Awaitility.await().untilAsserted(() -> { @@ -318,7 +318,7 @@ public void testMergeSnapshot(final BucketDelayedDeliveryTracker tracker) { } @Test(dataProvider = "delayedTracker") - public void testWithBkException(final BucketDelayedDeliveryTracker tracker) { + public void testWithBkException(final BucketDelayedDeliveryTracker tracker) throws Exception { MockBucketSnapshotStorage mockBucketSnapshotStorage = (MockBucketSnapshotStorage) bucketSnapshotStorage; mockBucketSnapshotStorage.injectCreateException( new BucketSnapshotPersistenceException("Bookie operation timeout, op: Create entry")); From c2fda68b477bab5bed3c052bb5a40b5f3b66f55f Mon Sep 17 00:00:00 2001 From: Hideaki Oguni <22386882+izumo27@users.noreply.github.com> Date: Mon, 29 Jul 2024 16:29:59 +0900 Subject: [PATCH 632/699] [fix][client] Fix negative acknowledgement by messageId (#23060) (cherry picked from commit d4bbf10f58771e2d43e576dc3422e502834b1de4) (cherry picked from commit 02f3ecc1d20cda17edd4c308f401b3c15463753c) --- .../apache/pulsar/client/impl/NegativeAcksTest.java | 13 ++++++++----- .../org/apache/pulsar/client/impl/ConsumerImpl.java | 2 +- 2 files changed, 9 insertions(+), 6 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 a6b77a1c72775..09b9b8b2fabdb 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 @@ -134,7 +134,7 @@ public void testNegativeAcks(boolean batching, boolean usePartitions, Subscripti Set sentMessages = new HashSet<>(); final int N = 10; - for (int i = 0; i < N; i++) { + for (int i = 0; i < N * 2; i++) { String value = "test-" + i; producer.sendAsync(value); sentMessages.add(value); @@ -145,6 +145,11 @@ public void testNegativeAcks(boolean batching, boolean usePartitions, Subscripti Message msg = consumer.receive(); consumer.negativeAcknowledge(msg); } + for (int i = 0; i < N; i++) { + Message msg = consumer.receive(); + consumer.negativeAcknowledge(msg.getMessageId()); + } + assertTrue(consumer instanceof ConsumerBase); assertEquals(((ConsumerBase) consumer).getUnAckedMessageTracker().size(), 0); @@ -152,7 +157,7 @@ public void testNegativeAcks(boolean batching, boolean usePartitions, Subscripti Set receivedMessages = new HashSet<>(); // All the messages should be received again - for (int i = 0; i < N; i++) { + for (int i = 0; i < N * 2; i++) { Message msg = consumer.receive(); receivedMessages.add(msg.getValue()); consumer.acknowledge(msg); @@ -310,9 +315,7 @@ public void testNegativeAcksDeleteFromUnackedTracker() throws Exception { assertEquals(unAckedMessageTracker.size(), 0); negativeAcksTracker.close(); // negative batch message id - unAckedMessageTracker.add(batchMessageId); - unAckedMessageTracker.add(batchMessageId2); - unAckedMessageTracker.add(batchMessageId3); + unAckedMessageTracker.add(messageId); consumer.negativeAcknowledge(batchMessageId); consumer.negativeAcknowledge(batchMessageId2); consumer.negativeAcknowledge(batchMessageId3); 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 75326336b4446..844f7f54a62a8 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 @@ -762,7 +762,7 @@ public void negativeAcknowledge(MessageId messageId) { negativeAcksTracker.add(messageId); // Ensure the message is not redelivered for ack-timeout, since we did receive an "ack" - unAckedMessageTracker.remove(messageId); + unAckedMessageTracker.remove(MessageIdAdvUtils.discardBatch(messageId)); } @Override From f5da2bf701f4bb070a98fb5ac81fd52c8e1b4384 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 29 Jul 2024 18:47:33 +0800 Subject: [PATCH 633/699] [fix] [broker] Remove blocking calls from Subscription.getStats (#23088) (cherry picked from commit e59cd05881bff11e4b127ed3496a02a0ce697fb7) (cherry picked from commit 9ea2a68cd560b08875ddcefc9a8f31536a293dcf) --- .../persistent/PersistentSubscription.java | 53 ++++++++---- .../service/persistent/PersistentTopic.java | 82 ++++++++++--------- 2 files changed, 80 insertions(+), 55 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 c5eeab63ccd95..97fa497dda9c8 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 @@ -32,6 +32,8 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -1125,8 +1127,29 @@ public long estimateBacklogSize() { return cursor.getEstimatedSizeSinceMarkDeletePosition(); } + /** + * @deprecated please call {@link #getStatsAsync(Boolean, boolean, boolean)}. + */ + @Deprecated public SubscriptionStatsImpl getStats(Boolean getPreciseBacklog, boolean subscriptionBacklogSize, boolean getEarliestTimeInBacklog) { + // So far, there is no case hits this check. + if (getEarliestTimeInBacklog) { + throw new IllegalArgumentException("Calling the sync method subscription.getStats with" + + " getEarliestTimeInBacklog, it may encountered a deadlock error."); + } + // The method "getStatsAsync" will be a sync method if the param "isGetEarliestTimeInBacklog" is false. + try { + return getStatsAsync(getPreciseBacklog, subscriptionBacklogSize, false).get(5, TimeUnit.SECONDS); + } catch (InterruptedException | ExecutionException | TimeoutException e) { + // This error will never occur. + throw new RuntimeException(e); + } + } + + public CompletableFuture getStatsAsync(Boolean getPreciseBacklog, + boolean subscriptionBacklogSize, + boolean getEarliestTimeInBacklog) { SubscriptionStatsImpl subStats = new SubscriptionStatsImpl(); subStats.lastExpireTimestamp = lastExpireTimestamp; subStats.lastConsumedFlowTimestamp = lastConsumedFlowTimestamp; @@ -1197,21 +1220,6 @@ public SubscriptionStatsImpl getStats(Boolean getPreciseBacklog, boolean subscri } else { subStats.backlogSize = -1; } - if (getEarliestTimeInBacklog) { - if (subStats.msgBacklog > 0) { - ManagedLedgerImpl managedLedger = ((ManagedLedgerImpl) cursor.getManagedLedger()); - PositionImpl markDeletedPosition = (PositionImpl) cursor.getMarkDeletedPosition(); - long result = 0; - try { - result = managedLedger.getEarliestMessagePublishTimeOfPos(markDeletedPosition).get(); - } catch (InterruptedException | ExecutionException e) { - result = -1; - } - subStats.earliestMsgPublishTimeInBacklog = result; - } else { - subStats.earliestMsgPublishTimeInBacklog = -1; - } - } subStats.msgBacklogNoDelayed = subStats.msgBacklog - subStats.msgDelayed; subStats.msgRateExpired = expiryMonitor.getMessageExpiryRate(); subStats.totalMsgExpired = expiryMonitor.getTotalMessageExpired(); @@ -1236,7 +1244,20 @@ public SubscriptionStatsImpl getStats(Boolean getPreciseBacklog, boolean subscri subStats.nonContiguousDeletedMessagesRanges = cursor.getTotalNonContiguousDeletedMessagesRange(); subStats.nonContiguousDeletedMessagesRangesSerializedSize = cursor.getNonContiguousDeletedMessagesRangeSerializedSize(); - return subStats; + if (!getEarliestTimeInBacklog) { + return CompletableFuture.completedFuture(subStats); + } + if (subStats.msgBacklog > 0) { + ManagedLedgerImpl managedLedger = ((ManagedLedgerImpl) cursor.getManagedLedger()); + PositionImpl markDeletedPosition = (PositionImpl) cursor.getMarkDeletedPosition(); + return managedLedger.getEarliestMessagePublishTimeOfPos(markDeletedPosition).thenApply(v -> { + subStats.earliestMsgPublishTimeInBacklog = v; + return subStats; + }); + } else { + subStats.earliestMsgPublishTimeInBacklog = -1; + return CompletableFuture.completedFuture(subStats); + } } @Override 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 d4167afbda18e..e1ff7621168e0 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 @@ -2434,7 +2434,6 @@ public TopicStatsImpl getStats(boolean getPreciseBacklog, boolean subscriptionBa public CompletableFuture asyncGetStats(boolean getPreciseBacklog, boolean subscriptionBacklogSize, boolean getEarliestTimeInBacklog) { - CompletableFuture statsFuture = new CompletableFuture<>(); TopicStatsImpl stats = new TopicStatsImpl(); ObjectObjectHashMap remotePublishersStats = new ObjectObjectHashMap<>(); @@ -2463,29 +2462,6 @@ public CompletableFuture asyncGetStats(boolean getPreciseBacklog stats.abortedTxnCount = txnBuffer.getAbortedTxnCount(); stats.committedTxnCount = txnBuffer.getCommittedTxnCount(); - subscriptions.forEach((name, subscription) -> { - SubscriptionStatsImpl subStats = - subscription.getStats(getPreciseBacklog, subscriptionBacklogSize, getEarliestTimeInBacklog); - - stats.msgRateOut += subStats.msgRateOut; - stats.msgThroughputOut += subStats.msgThroughputOut; - stats.bytesOutCounter += subStats.bytesOutCounter; - stats.msgOutCounter += subStats.msgOutCounter; - stats.subscriptions.put(name, subStats); - stats.nonContiguousDeletedMessagesRanges += subStats.nonContiguousDeletedMessagesRanges; - stats.nonContiguousDeletedMessagesRangesSerializedSize += - subStats.nonContiguousDeletedMessagesRangesSerializedSize; - stats.delayedMessageIndexSizeInBytes += subStats.delayedMessageIndexSizeInBytes; - - subStats.bucketDelayedIndexStats.forEach((k, v) -> { - TopicMetricBean topicMetricBean = - stats.bucketDelayedIndexStats.computeIfAbsent(k, __ -> new TopicMetricBean()); - topicMetricBean.name = v.name; - topicMetricBean.labelsAndValues = v.labelsAndValues; - topicMetricBean.value += v.value; - }); - }); - replicators.forEach((cluster, replicator) -> { ReplicatorStatsImpl replicatorStats = replicator.getStats(); @@ -2535,21 +2511,49 @@ public CompletableFuture asyncGetStats(boolean getPreciseBacklog return compactionRecord; }); - if (getEarliestTimeInBacklog && stats.backlogSize != 0) { - ledger.getEarliestMessagePublishTimeInBacklog().whenComplete((earliestTime, e) -> { - if (e != null) { - log.error("[{}] Failed to get earliest message publish time in backlog", topic, e); - statsFuture.completeExceptionally(e); - } else { - stats.earliestMsgPublishTimeInBacklogs = earliestTime; - statsFuture.complete(stats); - } - }); - } else { - statsFuture.complete(stats); - } - - return statsFuture; + Map> subscriptionFutures = new HashMap<>(); + subscriptions.forEach((name, subscription) -> { + subscriptionFutures.put(name, subscription.getStatsAsync(getPreciseBacklog, subscriptionBacklogSize, + getEarliestTimeInBacklog)); + }); + return FutureUtil.waitForAll(subscriptionFutures.values()).thenCompose(ignore -> { + for (Map.Entry> e : subscriptionFutures.entrySet()) { + String name = e.getKey(); + SubscriptionStatsImpl subStats = e.getValue().join(); + stats.msgRateOut += subStats.msgRateOut; + stats.msgThroughputOut += subStats.msgThroughputOut; + stats.bytesOutCounter += subStats.bytesOutCounter; + stats.msgOutCounter += subStats.msgOutCounter; + stats.subscriptions.put(name, subStats); + stats.nonContiguousDeletedMessagesRanges += subStats.nonContiguousDeletedMessagesRanges; + stats.nonContiguousDeletedMessagesRangesSerializedSize += + subStats.nonContiguousDeletedMessagesRangesSerializedSize; + stats.delayedMessageIndexSizeInBytes += subStats.delayedMessageIndexSizeInBytes; + + subStats.bucketDelayedIndexStats.forEach((k, v) -> { + TopicMetricBean topicMetricBean = + stats.bucketDelayedIndexStats.computeIfAbsent(k, __ -> new TopicMetricBean()); + topicMetricBean.name = v.name; + topicMetricBean.labelsAndValues = v.labelsAndValues; + topicMetricBean.value += v.value; + }); + } + if (getEarliestTimeInBacklog && stats.backlogSize != 0) { + CompletableFuture finalRes = ledger.getEarliestMessagePublishTimeInBacklog() + .thenApply((earliestTime) -> { + stats.earliestMsgPublishTimeInBacklogs = earliestTime; + return stats; + }); + // print error log. + finalRes.exceptionally(ex -> { + log.error("[{}] Failed to get earliest message publish time in backlog", topic, ex); + return null; + }); + return finalRes; + } else { + return CompletableFuture.completedFuture(stats); + } + }); } private Optional getCompactorMXBean() { From 453d96476ac799cdaeceed6b8963964f65b2ae0a Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 29 Jul 2024 18:54:12 +0800 Subject: [PATCH 634/699] [improve] [broker] Check max producers/consumers limitation first before other ops to save resources (#23074) (cherry picked from commit 679a3d49eefc2a82bbeba085c258b1f2b751f28a) (cherry picked from commit f08a34c8b7adc9ac454cb1aded6529f70f849ee7) --- .../pulsar/broker/service/AbstractTopic.java | 20 +++-- .../pulsar/broker/service/ServerCnx.java | 19 ++++ .../pulsar/broker/admin/AdminApi2Test.java | 77 ++++++++++++---- .../broker/service/PersistentTopicTest.java | 45 ---------- .../pulsar/client/api/MaxProducerTest.java | 88 +++++++++++++++++++ 5 files changed, 181 insertions(+), 68 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/MaxProducerTest.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 ff6fa1100607a..d437cb5929c7f 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 @@ -459,8 +459,18 @@ private PublishRate publishRateInBroker(ServiceConfiguration config) { return new PublishRate(config.getMaxPublishRatePerTopicInMessages(), config.getMaxPublishRatePerTopicInBytes()); } + public boolean isProducersExceeded(String producerName) { + String replicatorPrefix = brokerService.getPulsar().getConfig().getReplicatorPrefix() + "."; + boolean isRemote = producerName.startsWith(replicatorPrefix); + return isProducersExceeded(isRemote); + } + protected boolean isProducersExceeded(Producer producer) { - if (isSystemTopic() || producer.isRemote()) { + return isProducersExceeded(producer.isRemote()); + } + + protected boolean isProducersExceeded(boolean isRemote) { + if (isSystemTopic() || isRemote) { return false; } Integer maxProducers = topicPolicies.getMaxProducersPerTopic().get(); @@ -511,7 +521,7 @@ public int getNumberOfSameAddressProducers(final String clientAddress) { return count; } - protected boolean isConsumersExceededOnTopic() { + public boolean isConsumersExceededOnTopic() { if (isSystemTopic()) { return false; } @@ -967,12 +977,6 @@ protected void checkTopicFenced() throws BrokerServiceException { } protected CompletableFuture internalAddProducer(Producer producer) { - if (isProducersExceeded(producer)) { - log.warn("[{}] Attempting to add producer to topic which reached max producers limit", topic); - return CompletableFuture.failedFuture(new BrokerServiceException.ProducerBusyException( - "Topic '" + topic + "' reached max producers limit")); - } - if (isSameAddressProducersExceeded(producer)) { log.warn("[{}] Attempting to add producer to topic which reached max same address producers limit", topic); return CompletableFuture.failedFuture(new BrokerServiceException.ProducerBusyException( 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 e7a318e86c57e..2a8b14d19b4a4 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 @@ -1275,6 +1275,16 @@ protected void handleSubscribe(final CommandSubscribe subscribe) { "Topic " + topicName + " does not exist")); } final Topic topic = optTopic.get(); + // Check max consumer limitation to avoid unnecessary ops wasting resources. For example: + // the new consumer reached max producer limitation, but pulsar did schema check first, + // it would waste CPU. + if (((AbstractTopic) topic).isConsumersExceededOnTopic()) { + log.warn("[{}] Attempting to add consumer to topic which reached max" + + " consumers limit", topic); + Throwable t = + new ConsumerBusyException("Topic reached max consumers limit"); + return FutureUtil.failedFuture(t); + } return service.isAllowAutoSubscriptionCreationAsync(topicName) .thenCompose(isAllowedAutoSubscriptionCreation -> { boolean rejectSubscriptionIfDoesNotExist = isDurable @@ -1487,6 +1497,15 @@ protected void handleProducer(final CommandProducer cmdProducer) { } service.getOrCreateTopic(topicName.toString()).thenCompose((Topic topic) -> { + // Check max producer limitation to avoid unnecessary ops wasting resources. For example: the new + // producer reached max producer limitation, but pulsar did schema check first, it would waste CPU + if (((AbstractTopic) topic).isProducersExceeded(producerName)) { + log.warn("[{}] Attempting to add producer to topic which reached max producers limit", topic); + String errorMsg = "Topic '" + topicName.toString() + "' reached max producers limit"; + Throwable t = new BrokerServiceException.ProducerBusyException(errorMsg); + return CompletableFuture.failedFuture(t); + } + // Before creating producer, check if backlog quota exceeded // on topic for size based limit and time based limit CompletableFuture backlogQuotaCheckFuture = CompletableFuture.allOf( 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 30e4875c170be..0254cf06f4ead 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 @@ -21,6 +21,8 @@ import static java.util.concurrent.TimeUnit.MINUTES; import static org.apache.commons.lang3.StringUtils.isBlank; import static org.apache.pulsar.broker.BrokerTestUtil.newUniqueName; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -51,6 +53,7 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import javax.ws.rs.NotAcceptableException; import javax.ws.rs.core.Response.Status; import lombok.AllArgsConstructor; @@ -69,6 +72,7 @@ import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl; import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerWrapper; import org.apache.pulsar.broker.loadbalance.impl.SimpleLoadManagerImpl; +import org.apache.pulsar.broker.service.AbstractTopic; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -126,7 +130,13 @@ import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.policies.data.TopicType; import org.apache.pulsar.common.policies.data.impl.BacklogQuotaImpl; +import org.apache.pulsar.common.protocol.schema.SchemaData; +import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.AfterMethod; @@ -2868,34 +2878,40 @@ public void testMaxProducersPerTopicUnlimited() throws Exception { final String myNamespace = newUniqueName(defaultTenant + "/ns"); admin.namespaces().createNamespace(myNamespace, Set.of("test")); final String topic = "persistent://" + myNamespace + "/testMaxProducersPerTopicUnlimited"; + admin.topics().createNonPartitionedTopic(topic); + AtomicInteger schemaOpsCounter = injectSchemaCheckCounterForTopic(topic); //the policy is set to 0, so there will be no restrictions admin.namespaces().setMaxProducersPerTopic(myNamespace, 0); Awaitility.await().until(() -> admin.namespaces().getMaxProducersPerTopic(myNamespace) == 0); - List> producers = new ArrayList<>(); + List> producers = new ArrayList<>(); for (int i = 0; i < maxProducersPerTopic + 1; i++) { - Producer producer = pulsarClient.newProducer().topic(topic).create(); + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); producers.add(producer); } + assertEquals(schemaOpsCounter.get(), maxProducersPerTopic + 1); admin.namespaces().removeMaxProducersPerTopic(myNamespace); Awaitility.await().until(() -> admin.namespaces().getMaxProducersPerTopic(myNamespace) == null); + try { @Cleanup - Producer producer = pulsarClient.newProducer().topic(topic).create(); + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); fail("should fail"); } catch (PulsarClientException e) { String expectMsg = "Topic '" + topic + "' reached max producers limit"; assertTrue(e.getMessage().contains(expectMsg)); + assertEquals(schemaOpsCounter.get(), maxProducersPerTopic + 1); } //set the limit to 3 admin.namespaces().setMaxProducersPerTopic(myNamespace, 3); Awaitility.await().until(() -> admin.namespaces().getMaxProducersPerTopic(myNamespace) == 3); // should success - Producer producer = pulsarClient.newProducer().topic(topic).create(); + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); producers.add(producer); + assertEquals(schemaOpsCounter.get(), maxProducersPerTopic + 2); try { @Cleanup Producer producer1 = pulsarClient.newProducer().topic(topic).create(); @@ -2903,14 +2919,39 @@ public void testMaxProducersPerTopicUnlimited() throws Exception { } catch (PulsarClientException e) { String expectMsg = "Topic '" + topic + "' reached max producers limit"; assertTrue(e.getMessage().contains(expectMsg)); + assertEquals(schemaOpsCounter.get(), maxProducersPerTopic + 2); } //clean up - for (Producer tempProducer : producers) { + for (Producer tempProducer : producers) { tempProducer.close(); } } + private AtomicInteger injectSchemaCheckCounterForTopic(String topicName) { + ConcurrentOpenHashMap>> topics = + WhiteboxImpl.getInternalState(pulsar.getBrokerService(), "topics"); + AbstractTopic topic = (AbstractTopic) topics.get(topicName).join().get(); + AbstractTopic spyTopic = Mockito.spy(topic); + AtomicInteger counter = new AtomicInteger(); + doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + counter.incrementAndGet(); + return invocation.callRealMethod(); + } + }).when(spyTopic).addSchema(any(SchemaData.class)); + doAnswer(new Answer() { + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + counter.incrementAndGet(); + return invocation.callRealMethod(); + } + }).when(spyTopic).addSchemaIfIdleOrCheckCompatible(any(SchemaData.class)); + topics.put(topicName, CompletableFuture.completedFuture(Optional.of(spyTopic))); + return counter; + } + @Test public void testMaxConsumersPerTopicUnlimited() throws Exception { restartClusterAfterTest(); @@ -2922,49 +2963,55 @@ public void testMaxConsumersPerTopicUnlimited() throws Exception { final String myNamespace = newUniqueName(defaultTenant + "/ns"); admin.namespaces().createNamespace(myNamespace, Set.of("test")); final String topic = "persistent://" + myNamespace + "/testMaxConsumersPerTopicUnlimited"; + admin.topics().createNonPartitionedTopic(topic); + AtomicInteger schemaOpsCounter = injectSchemaCheckCounterForTopic(topic); assertNull(admin.namespaces().getMaxConsumersPerTopic(myNamespace)); //the policy is set to 0, so there will be no restrictions admin.namespaces().setMaxConsumersPerTopic(myNamespace, 0); Awaitility.await().until(() -> admin.namespaces().getMaxConsumersPerTopic(myNamespace) == 0); - List> consumers = new ArrayList<>(); + List> consumers = new ArrayList<>(); for (int i = 0; i < maxConsumersPerTopic + 1; i++) { - Consumer consumer = - pulsarClient.newConsumer().subscriptionName(UUID.randomUUID().toString()).topic(topic).subscribe(); + Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .subscriptionName(UUID.randomUUID().toString()).topic(topic).subscribe(); consumers.add(consumer); } + assertEquals(schemaOpsCounter.get(), maxConsumersPerTopic + 2); admin.namespaces().removeMaxConsumersPerTopic(myNamespace); Awaitility.await().until(() -> admin.namespaces().getMaxConsumersPerTopic(myNamespace) == null); try { @Cleanup - Consumer subscribe = - pulsarClient.newConsumer().subscriptionName(UUID.randomUUID().toString()).topic(topic).subscribe(); + Consumer subscribe = pulsarClient.newConsumer(Schema.STRING) + .subscriptionName(UUID.randomUUID().toString()).topic(topic).subscribe(); fail("should fail"); } catch (PulsarClientException e) { assertTrue(e.getMessage().contains("Topic reached max consumers limit")); + assertEquals(schemaOpsCounter.get(), maxConsumersPerTopic + 2); } //set the limit to 3 admin.namespaces().setMaxConsumersPerTopic(myNamespace, 3); Awaitility.await().until(() -> admin.namespaces().getMaxConsumersPerTopic(myNamespace) == 3); // should success - Consumer consumer = - pulsarClient.newConsumer().subscriptionName(UUID.randomUUID().toString()).topic(topic).subscribe(); + Consumer consumer = pulsarClient.newConsumer(Schema.STRING) + .subscriptionName(UUID.randomUUID().toString()).topic(topic).subscribe(); consumers.add(consumer); + assertEquals(schemaOpsCounter.get(), maxConsumersPerTopic + 3); try { @Cleanup - Consumer subscribe = - pulsarClient.newConsumer().subscriptionName(UUID.randomUUID().toString()).topic(topic).subscribe(); + Consumer subscribe = pulsarClient.newConsumer(Schema.STRING) + .subscriptionName(UUID.randomUUID().toString()).topic(topic).subscribe(); fail("should fail"); } catch (PulsarClientException e) { assertTrue(e.getMessage().contains("Topic reached max consumers limit")); + assertEquals(schemaOpsCounter.get(), maxConsumersPerTopic + 3); } //clean up - for (Consumer subConsumer : consumers) { + for (Consumer subConsumer : consumers) { subConsumer.close(); } } 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 bf6065a8b9e3a..123a7e5372dc3 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 @@ -502,51 +502,6 @@ public void testProducerOverwrite() { topic.getProducers().values().forEach(producer -> Assert.assertEquals(producer.getEpoch(), 3)); } - private void testMaxProducers() { - PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, brokerService); - topic.initialize().join(); - String role = "appid1"; - // 1. add producer1 - Producer producer = new Producer(topic, serverCnx, 1 /* producer id */, "prod-name1", role, - false, null, SchemaVersion.Latest, 0, false, ProducerAccessMode.Shared, Optional.empty(), true); - topic.addProducer(producer, new CompletableFuture<>()); - assertEquals(topic.getProducers().size(), 1); - - // 2. add producer2 - Producer producer2 = new Producer(topic, serverCnx, 2 /* producer id */, "prod-name2", role, - false, null, SchemaVersion.Latest, 0, false, ProducerAccessMode.Shared, Optional.empty(), true); - topic.addProducer(producer2, new CompletableFuture<>()); - assertEquals(topic.getProducers().size(), 2); - - // 3. add producer3 but reached maxProducersPerTopic - try { - Producer producer3 = new Producer(topic, serverCnx, 3 /* producer id */, "prod-name3", role, - false, null, SchemaVersion.Latest, 0, false, ProducerAccessMode.Shared, Optional.empty(), true); - topic.addProducer(producer3, new CompletableFuture<>()).join(); - fail("should have failed"); - } catch (Exception e) { - assertEquals(e.getCause().getClass(), BrokerServiceException.ProducerBusyException.class); - } - } - - @Test - public void testMaxProducersForBroker() { - // set max clients - pulsarTestContext.getConfig().setMaxProducersPerTopic(2); - testMaxProducers(); - } - - @Test - public void testMaxProducersForNamespace() throws Exception { - // set max clients - Policies policies = new Policies(); - policies.max_producers_per_topic = 2; - pulsarTestContext.getPulsarResources().getNamespaceResources() - .createPolicies(TopicName.get(successTopicName).getNamespaceObject(), - policies); - testMaxProducers(); - } - private Producer getMockedProducerWithSpecificAddress(Topic topic, long producerId, InetAddress address) { final String producerNameBase = "producer"; final String role = "appid1"; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MaxProducerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MaxProducerTest.java new file mode 100644 index 0000000000000..a34b05280c4f5 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MaxProducerTest.java @@ -0,0 +1,88 @@ +/* + * 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.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import java.util.ArrayList; +import java.util.List; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker-api") +public class MaxProducerTest 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(); + } + + @Override + protected void doInitConf() throws Exception { + super.doInitConf(); + conf.setMaxProducersPerTopic(2); + } + + @Test + public void testMaxProducersForBroker() throws Exception { + testMaxProducers(2); + } + + @Test + public void testMaxProducersForNamespace() throws Exception { + // set max clients + admin.namespaces().setMaxProducersPerTopic("public/default", 3); + testMaxProducers(3); + } + + private void testMaxProducers(int maxProducerExpected) throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + admin.topics().createNonPartitionedTopic(topicName); + + List> producers = new ArrayList<>(); + for (int i = 0; i < maxProducerExpected; i++) { + producers.add(pulsarClient.newProducer().topic(topicName).create()); + } + + try { + pulsarClient.newProducer().topic(topicName).create(); + fail("should have failed"); + } catch (Exception e) { + assertTrue(e instanceof PulsarClientException.ProducerBusyException); + } + + // cleanup. + for (org.apache.pulsar.client.api.Producer p : producers) { + p.close(); + } + admin.topics().delete(topicName, false); + } +} From de5ccbdde6991cbfd78a2127191546b868d3109b Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Mon, 29 Jul 2024 14:43:52 +0300 Subject: [PATCH 635/699] [improve][ci] Switch to use DEVELOCITY_ACCESS_KEY from GRADLE_ENTERPRISE_ACCESS_KEY (#23090) (cherry picked from commit a0bdf4585feb78b69179b260199dd9aaea6d793f) (cherry picked from commit 98978962e5c7c51862c166fa42ca8e4b2fe0a40e) --- .github/workflows/ci-maven-cache-update.yaml | 2 +- .../workflows/ci-owasp-dependency-check.yaml | 2 +- .github/workflows/pulsar-ci-flaky.yaml | 2 +- .github/workflows/pulsar-ci.yaml | 18 +++++++++--------- .mvn/extensions.xml | 2 +- 5 files changed, 13 insertions(+), 13 deletions(-) diff --git a/.github/workflows/ci-maven-cache-update.yaml b/.github/workflows/ci-maven-cache-update.yaml index bb2c6efa33c10..d37794c2e54a1 100644 --- a/.github/workflows/ci-maven-cache-update.yaml +++ b/.github/workflows/ci-maven-cache-update.yaml @@ -49,7 +49,7 @@ jobs: name: Update Maven dependency cache for ${{ matrix.name }} env: JOB_NAME: Update Maven dependency cache for ${{ matrix.name }} - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} 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 a70f4a82ff1af..a1c6dd594d3a2 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 }} - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} 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 27d2e9895ef7e..4731381857d2f 100644 --- a/.github/workflows/pulsar-ci-flaky.yaml +++ b/.github/workflows/pulsar-ci-flaky.yaml @@ -95,7 +95,7 @@ jobs: env: JOB_NAME: Flaky tests suite COLLECT_COVERAGE: "${{ needs.preconditions.outputs.collect_coverage }}" - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} runs-on: ubuntu-22.04 timeout-minutes: 100 if: ${{ needs.preconditions.outputs.docs_only != 'true' }} diff --git a/.github/workflows/pulsar-ci.yaml b/.github/workflows/pulsar-ci.yaml index 71e51af50f159..494ac771b0420 100644 --- a/.github/workflows/pulsar-ci.yaml +++ b/.github/workflows/pulsar-ci.yaml @@ -96,7 +96,7 @@ jobs: name: Build and License check env: JOB_NAME: Build and License check - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} runs-on: ubuntu-22.04 timeout-minutes: 60 if: ${{ needs.preconditions.outputs.docs_only != 'true' }} @@ -172,7 +172,7 @@ jobs: env: JOB_NAME: CI - Unit - ${{ matrix.name }} COLLECT_COVERAGE: "${{ needs.preconditions.outputs.collect_coverage }}" - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} runs-on: ubuntu-22.04 timeout-minutes: ${{ matrix.timeout || 60 }} needs: ['preconditions', 'build-and-license-check'] @@ -391,7 +391,7 @@ jobs: needs: ['preconditions', 'build-and-license-check'] if: ${{ needs.preconditions.outputs.docs_only != 'true'}} env: - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} steps: - name: checkout uses: actions/checkout@v4 @@ -465,7 +465,7 @@ jobs: env: JOB_NAME: CI - Integration - ${{ matrix.name }} PULSAR_TEST_IMAGE_NAME: apachepulsar/java-test-image:latest - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} strategy: fail-fast: false matrix: @@ -731,7 +731,7 @@ jobs: needs: ['preconditions', 'build-and-license-check'] if: ${{ needs.preconditions.outputs.docs_only != 'true' }} env: - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} steps: - name: checkout uses: actions/checkout@v4 @@ -841,7 +841,7 @@ jobs: env: JOB_NAME: CI - System - ${{ matrix.name }} PULSAR_TEST_IMAGE_NAME: apachepulsar/pulsar-test-latest-version:latest - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} strategy: fail-fast: false matrix: @@ -1072,7 +1072,7 @@ jobs: env: JOB_NAME: CI Flaky - System - ${{ matrix.name }} PULSAR_TEST_IMAGE_NAME: apachepulsar/pulsar-test-latest-version:latest - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} strategy: fail-fast: false matrix: @@ -1214,7 +1214,7 @@ jobs: needs: ['preconditions', 'integration-tests'] if: ${{ needs.preconditions.outputs.docs_only != 'true' }} env: - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} steps: - name: checkout uses: actions/checkout@v4 @@ -1249,7 +1249,7 @@ jobs: needs: [ 'preconditions', 'integration-tests' ] if: ${{ needs.preconditions.outputs.need_owasp == 'true' }} env: - GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + DEVELOCITY_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} NIST_NVD_API_KEY: ${{ secrets.NIST_NVD_API_KEY }} steps: - name: checkout diff --git a/.mvn/extensions.xml b/.mvn/extensions.xml index 4a2117925f163..eb998dc3471b8 100644 --- a/.mvn/extensions.xml +++ b/.mvn/extensions.xml @@ -24,7 +24,7 @@ com.gradle develocity-maven-extension - 1.21.4 + 1.21.6 com.gradle From 148077cf6314ce394cf00abb9113aaaed744831d Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 29 Jul 2024 23:06:45 +0800 Subject: [PATCH 636/699] [fix] [broker] Internal reader of __change_events can not started after metadata store session rebuilt (#23018) (cherry picked from commit b955c6520d8db948048a1b2dc548a001ee396076) (cherry picked from commit 0e59678194c36ef1a4f443ac18dba08ae8a1c372) --- .../extensions/ExtensibleLoadManagerImpl.java | 3 +- .../impl/ModularLoadManagerImpl.java | 3 +- .../pulsar/broker/service/Ipv4Proxy.java | 197 ++++++++++++ .../broker/service/NetworkErrorTestBase.java | 297 ++++++++++++++++++ .../broker/service/ZkSessionExpireTest.java | 184 +++++++++++ .../metadata/impl/AbstractMetadataStore.java | 10 + 6 files changed, 692 insertions(+), 2 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/Ipv4Proxy.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/NetworkErrorTestBase.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ZkSessionExpireTest.java 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 dc57a923c7adc..e0ed8c9198d57 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 @@ -125,7 +125,8 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager { private static final Set INTERNAL_TOPICS = Set.of(BROKER_LOAD_DATA_STORE_TOPIC, TOP_BUNDLES_LOAD_DATA_STORE_TOPIC, TOPIC); - private PulsarService pulsar; + @VisibleForTesting + protected PulsarService pulsar; private ServiceConfiguration conf; 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 edd7080ee2e18..fb961f5076fb1 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 @@ -169,8 +169,9 @@ public class ModularLoadManagerImpl implements ModularLoadManager { // Policies used to determine which brokers are available for particular namespaces. private SimpleResourceAllocationPolicies policies; + @VisibleForTesting // Pulsar service used to initialize this. - private PulsarService pulsar; + protected PulsarService pulsar; // Executor service used to update broker data. private final ExecutorService executors; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/Ipv4Proxy.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/Ipv4Proxy.java new file mode 100644 index 0000000000000..a84dab4d17dff --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/Ipv4Proxy.java @@ -0,0 +1,197 @@ +/* + * 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 io.netty.bootstrap.Bootstrap; +import io.netty.bootstrap.ServerBootstrap; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelOption; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.SocketChannel; +import io.netty.channel.socket.nio.NioServerSocketChannel; +import io.netty.handler.logging.LogLevel; +import io.netty.handler.logging.LoggingHandler; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import lombok.Getter; + +public class Ipv4Proxy { + @Getter + private final int localPort; + private final String backendServerHost; + private final int backendServerPort; + private final EventLoopGroup serverGroup = new NioEventLoopGroup(1); + private final EventLoopGroup workerGroup = new NioEventLoopGroup(); + private ChannelFuture localServerChannel; + private ServerBootstrap serverBootstrap = new ServerBootstrap(); + private List frontChannels = Collections.synchronizedList(new ArrayList<>()); + private AtomicBoolean rejectAllConnections = new AtomicBoolean(); + + public Ipv4Proxy(int localPort, String backendServerHost, int backendServerPort) { + this.localPort = localPort; + this.backendServerHost = backendServerHost; + this.backendServerPort = backendServerPort; + } + + public synchronized void startup() throws InterruptedException { + localServerChannel = serverBootstrap.group(serverGroup, workerGroup) + .channel(NioServerSocketChannel.class) + .handler(new LoggingHandler(LogLevel.INFO)) + .childHandler(new ChannelInitializer() { + @Override + protected void initChannel(SocketChannel ch) { + ch.pipeline().addLast(new FrontendHandler()); + } + }).childOption(ChannelOption.AUTO_READ, false) + .bind(localPort).sync(); + } + + public synchronized void stop() throws InterruptedException{ + localServerChannel.channel().close().sync(); + serverGroup.shutdownGracefully(); + workerGroup.shutdownGracefully(); + } + + private static void closeOnFlush(Channel ch) { + if (ch.isActive()) { + ch.writeAndFlush(Unpooled.EMPTY_BUFFER).addListener(ChannelFutureListener.CLOSE); + } + } + + public void disconnectFrontChannels() throws InterruptedException { + for (Channel channel : frontChannels) { + channel.close(); + } + } + + public void rejectAllConnections() throws InterruptedException { + rejectAllConnections.set(true); + } + + public void unRejectAllConnections() throws InterruptedException { + rejectAllConnections.set(false); + } + + private class FrontendHandler extends ChannelInboundHandlerAdapter { + + private Channel backendChannel; + + @Override + public void channelActive(ChannelHandlerContext ctx) { + if (rejectAllConnections.get()) { + ctx.close(); + return; + } + final Channel frontendChannel = ctx.channel(); + frontChannels.add(frontendChannel); + Bootstrap backendBootstrap = new Bootstrap(); + backendBootstrap.group(frontendChannel.eventLoop()) + .channel(ctx.channel().getClass()) + .handler(new BackendHandler(frontendChannel)) + .option(ChannelOption.AUTO_READ, false); + ChannelFuture backendChannelFuture = + backendBootstrap.connect(Ipv4Proxy.this.backendServerHost, Ipv4Proxy.this.backendServerPort); + backendChannel = backendChannelFuture.channel(); + backendChannelFuture.addListener((ChannelFutureListener) future -> { + if (future.isSuccess()) { + frontendChannel.read(); + } else { + frontChannels.remove(frontendChannel); + frontendChannel.close(); + } + }); + } + + @Override + public void channelRead(final ChannelHandlerContext ctx, Object msg) { + if (backendChannel.isActive()) { + backendChannel.writeAndFlush(msg).addListener((ChannelFutureListener) future -> { + if (future.isSuccess()) { + ctx.channel().read(); + } else { + future.channel().close(); + } + }); + } + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) { + frontChannels.remove(ctx.channel()); + if (backendChannel != null) { + closeOnFlush(backendChannel); + } + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + cause.printStackTrace(); + closeOnFlush(ctx.channel()); + } + } + + private class BackendHandler extends ChannelInboundHandlerAdapter { + + private final Channel frontendChannel; + + public BackendHandler(Channel inboundChannel) { + this.frontendChannel = inboundChannel; + } + + @Override + public void channelActive(ChannelHandlerContext ctx) { + if (!frontendChannel.isActive()) { + closeOnFlush(ctx.channel()); + } else { + ctx.read(); + } + } + + @Override + public void channelRead(final ChannelHandlerContext ctx, Object msg) { + frontendChannel.writeAndFlush(msg).addListener((ChannelFutureListener) future -> { + if (future.isSuccess()) { + ctx.channel().read(); + } else { + future.channel().close(); + } + }); + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) { + closeOnFlush(frontendChannel); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { + cause.printStackTrace(); + closeOnFlush(ctx.channel()); + } + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/NetworkErrorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/NetworkErrorTestBase.java new file mode 100644 index 0000000000000..36f8cb4761248 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/NetworkErrorTestBase.java @@ -0,0 +1,297 @@ +/* + * 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.collect.Sets; +import java.io.IOException; +import java.net.ServerSocket; +import java.net.URL; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.StringUtils; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.loadbalance.extensions.ExtensibleLoadManagerImpl; +import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl; +import org.apache.pulsar.broker.namespace.LookupOptions; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.ClientBuilder; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.common.naming.ServiceUnitId; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.policies.data.TopicType; +import org.apache.pulsar.tests.TestRetrySupport; +import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; +import org.apache.pulsar.zookeeper.ZookeeperServerTest; +import org.awaitility.reflect.WhiteboxImpl; + +@Slf4j +public abstract class NetworkErrorTestBase extends TestRetrySupport { + + protected final String defaultTenant = "public"; + protected final String defaultNamespace = defaultTenant + "/default"; + protected final String cluster1 = "r1"; + protected URL url1; + protected URL urlTls1; + protected URL url2; + protected URL urlTls2; + protected ServiceConfiguration config1 = new ServiceConfiguration(); + protected ServiceConfiguration config2 = new ServiceConfiguration(); + protected ZookeeperServerTest brokerConfigZk1; + protected Ipv4Proxy metadataZKProxy; + protected LocalBookkeeperEnsemble bkEnsemble1; + protected PulsarService pulsar1; + protected PulsarService pulsar2; + protected BrokerService broker1; + protected BrokerService broker2; + protected PulsarAdmin admin1; + protected PulsarAdmin admin2; + protected PulsarClient client1; + protected PulsarClient client2; + + private final static AtomicReference preferBroker = new AtomicReference<>(); + + protected void startZKAndBK() throws Exception { + // Start ZK & BK. + bkEnsemble1 = new LocalBookkeeperEnsemble(3, 0, () -> 0); + bkEnsemble1.start(); + + metadataZKProxy = new Ipv4Proxy(getOneFreePort(), "127.0.0.1", bkEnsemble1.getZookeeperPort()); + metadataZKProxy.startup(); + } + + protected void startBrokers() throws Exception { + // Start brokers. + setConfigDefaults(config1, cluster1, metadataZKProxy.getLocalPort()); + pulsar1 = new PulsarService(config1); + pulsar1.start(); + broker1 = pulsar1.getBrokerService(); + url1 = new URL(pulsar1.getWebServiceAddress()); + urlTls1 = new URL(pulsar1.getWebServiceAddressTls()); + + setConfigDefaults(config2, cluster1, bkEnsemble1.getZookeeperPort()); + pulsar2 = new PulsarService(config2); + pulsar2.start(); + broker2 = pulsar2.getBrokerService(); + url2 = new URL(pulsar2.getWebServiceAddress()); + urlTls2 = new URL(pulsar2.getWebServiceAddressTls()); + + log.info("broker-1: {}, broker-2: {}", broker1.getListenPort(), broker2.getListenPort()); + } + + protected int getOneFreePort() throws IOException { + ServerSocket serverSocket = new ServerSocket(0); + int port = serverSocket.getLocalPort(); + serverSocket.close(); + return port; + } + + protected void startAdminClient() throws Exception { + admin1 = PulsarAdmin.builder().serviceHttpUrl(url1.toString()).build(); + admin2 = PulsarAdmin.builder().serviceHttpUrl(url2.toString()).build(); + } + + protected void startPulsarClient() throws Exception{ + ClientBuilder clientBuilder1 = PulsarClient.builder().serviceUrl(url1.toString()); + client1 = initClient(clientBuilder1); + ClientBuilder clientBuilder2 = PulsarClient.builder().serviceUrl(url2.toString()); + client2 = initClient(clientBuilder2); + } + + protected void createDefaultTenantsAndClustersAndNamespace() throws Exception { + admin1.clusters().createCluster(cluster1, ClusterData.builder() + .serviceUrl(url1.toString()) + .serviceUrlTls(urlTls1.toString()) + .brokerServiceUrl(pulsar1.getBrokerServiceUrl()) + .brokerServiceUrlTls(pulsar1.getBrokerServiceUrlTls()) + .brokerClientTlsEnabled(false) + .build()); + admin1.tenants().createTenant(defaultTenant, new TenantInfoImpl(Collections.emptySet(), + Sets.newHashSet(cluster1))); + admin1.namespaces().createNamespace(defaultNamespace, Sets.newHashSet(cluster1)); + } + + @Override + protected void setup() throws Exception { + incrementSetupNumber(); + + log.info("--- Starting OneWayReplicatorTestBase::setup ---"); + + startZKAndBK(); + + startBrokers(); + + startAdminClient(); + + createDefaultTenantsAndClustersAndNamespace(); + + startPulsarClient(); + + Thread.sleep(100); + log.info("--- OneWayReplicatorTestBase::setup completed ---"); + } + + protected void setConfigDefaults(ServiceConfiguration config, String clusterName, int zkPort) { + config.setClusterName(clusterName); + config.setAdvertisedAddress("localhost"); + config.setWebServicePort(Optional.of(0)); + config.setWebServicePortTls(Optional.of(0)); + config.setMetadataStoreUrl("zk:127.0.0.1:" + zkPort); + config.setConfigurationMetadataStoreUrl("zk:127.0.0.1:" + zkPort + "/config_meta"); + config.setBrokerDeleteInactiveTopicsEnabled(false); + config.setBrokerDeleteInactiveTopicsFrequencySeconds(60); + config.setBrokerShutdownTimeoutMs(0L); + config.setLoadBalancerOverrideBrokerNicSpeedGbps(Optional.of(1.0d)); + config.setBrokerServicePort(Optional.of(0)); + config.setBrokerServicePortTls(Optional.of(0)); + config.setBacklogQuotaCheckIntervalInSeconds(5); + config.setDefaultNumberOfNamespaceBundles(1); + config.setAllowAutoTopicCreationType(TopicType.NON_PARTITIONED); + config.setEnableReplicatedSubscriptions(true); + config.setReplicatedSubscriptionsSnapshotFrequencyMillis(1000); + config.setLoadBalancerSheddingEnabled(false); + config.setForceDeleteNamespaceAllowed(true); + config.setLoadManagerClassName(PreferBrokerModularLoadManager.class.getName()); + config.setMetadataStoreSessionTimeoutMillis(5000); + } + + @Override + protected void cleanup() throws Exception { + // shutdown. + markCurrentSetupNumberCleaned(); + log.info("--- Shutting down ---"); + + // Stop brokers. + if (client1 != null) { + client1.close(); + client1 = null; + } + if (admin1 != null) { + admin1.close(); + admin1 = null; + } + if (client2 != null) { + client2.close(); + client2 = null; + } + if (admin2 != null) { + admin2.close(); + admin2 = null; + } + if (pulsar1 != null) { + pulsar1.close(); + pulsar1 = null; + } + if (pulsar2 != null) { + pulsar2.close(); + pulsar2 = null; + } + + // Stop ZK and BK. + if (bkEnsemble1 != null) { + bkEnsemble1.stop(); + bkEnsemble1 = null; + } + if (metadataZKProxy != null) { + metadataZKProxy.stop(); + } + if (brokerConfigZk1 != null) { + brokerConfigZk1.stop(); + brokerConfigZk1 = null; + } + + // Reset configs. + config1 = new ServiceConfiguration(); + preferBroker.set(null); + } + + protected PulsarClient initClient(ClientBuilder clientBuilder) throws Exception { + return clientBuilder.build(); + } + + protected static class PreferBrokerModularLoadManager extends ModularLoadManagerImpl { + + @Override + public String setNamespaceBundleAffinity(String bundle, String broker) { + if (StringUtils.isNotBlank(broker)) { + return broker; + } + Set availableBrokers = NetworkErrorTestBase.getAvailableBrokers(super.pulsar); + String prefer = preferBroker.get(); + if (availableBrokers.contains(prefer)) { + return prefer; + } else { + return null; + } + } + } + + protected static class PreferExtensibleLoadManager extends ExtensibleLoadManagerImpl { + + @Override + public CompletableFuture> selectAsync(ServiceUnitId bundle, + Set excludeBrokerSet, + LookupOptions options) { + Set availableBrokers = NetworkErrorTestBase.getAvailableBrokers(super.pulsar); + String prefer = preferBroker.get(); + if (availableBrokers.contains(prefer)) { + return CompletableFuture.completedFuture(Optional.of(prefer)); + } else { + return super.selectAsync(bundle, excludeBrokerSet, options); + } + } + } + + public void setPreferBroker(PulsarService target) { + for (PulsarService pulsar : Arrays.asList(pulsar1, pulsar2)) { + for (String broker : getAvailableBrokers(pulsar)) { + if (broker.endsWith(target.getBrokerListenPort().orElse(-1) + "") + || broker.endsWith(target.getListenPortHTTPS().orElse(-1) + "") + || broker.endsWith(target.getListenPortHTTP().orElse(-1) + "") + || broker.endsWith(target.getBrokerListenPortTls().orElse(-1) + "")) { + preferBroker.set(broker); + } + } + } + } + + public static Set getAvailableBrokers(PulsarService pulsar) { + Object loadManagerWrapper = pulsar.getLoadManager().get(); + Object loadManager = WhiteboxImpl.getInternalState(loadManagerWrapper, "loadManager"); + if (loadManager instanceof ModularLoadManagerImpl) { + return ((ModularLoadManagerImpl) loadManager).getAvailableBrokers(); + } else if (loadManager instanceof ExtensibleLoadManagerImpl) { + return new HashSet<>(((ExtensibleLoadManagerImpl) loadManager).getBrokerRegistry() + .getAvailableBrokersAsync().join()); + } else { + throw new RuntimeException("Not support for the load manager: " + loadManager.getClass().getName()); + } + } + + public void clearPreferBroker() { + preferBroker.set(null); + } +} 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 new file mode 100644 index 0000000000000..143557b008b23 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ZkSessionExpireTest.java @@ -0,0 +1,184 @@ +/* + * 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 static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import java.time.Duration; +import java.util.Optional; +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.broker.ServiceConfiguration; +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.client.impl.ProducerImpl; +import org.awaitility.Awaitility; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +@Slf4j +@Test(groups = "broker") +public class ZkSessionExpireTest extends NetworkErrorTestBase { + + private java.util.function.Consumer settings; + + @AfterMethod(alwaysRun = true) + @Override + public void cleanup() throws Exception { + super.cleanup(); + } + + private void setupWithSettings(java.util.function.Consumer settings) throws Exception { + this.settings = settings; + super.setup(); + } + + protected void setConfigDefaults(ServiceConfiguration config, String clusterName, int zkPort) { + super.setConfigDefaults(config, clusterName, zkPort); + settings.accept(config); + } + + @DataProvider(name = "settings") + public Object[][] settings() { + return new Object[][]{ + {false, NetworkErrorTestBase.PreferBrokerModularLoadManager.class}, + {true, NetworkErrorTestBase.PreferBrokerModularLoadManager.class} + // Create a separate PR to add this test case. + // {true, NetworkErrorTestBase.PreferExtensibleLoadManager.class}. + }; + } + + @Test(timeOut = 600 * 1000, dataProvider = "settings") + public void testTopicUnloadAfterSessionRebuild(boolean enableSystemTopic, Class loadManager) throws Exception { + // Setup. + setupWithSettings(config -> { + config.setManagedLedgerMaxEntriesPerLedger(1); + config.setSystemTopicEnabled(enableSystemTopic); + config.setTopicLevelPoliciesEnabled(enableSystemTopic); + if (loadManager != null) { + config.setLoadManagerClassName(loadManager.getName()); + } + }); + + // Init topic. + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + defaultNamespace + "/tp"); + admin1.topics().createNonPartitionedTopic(topicName); + admin1.topics().createSubscription(topicName, "s1", MessageId.earliest); + + // Inject a prefer mechanism, so that all topics will be assigned to broker1, which can be injected a ZK + // session expire error. + setPreferBroker(pulsar1); + admin1.namespaces().unload(defaultNamespace); + admin2.namespaces().unload(defaultNamespace); + + // Confirm all brokers registered. + Awaitility.await().untilAsserted(() -> { + assertEquals(getAvailableBrokers(pulsar1).size(), 2); + assertEquals(getAvailableBrokers(pulsar2).size(), 2); + }); + + // Load up a topic, and it will be assigned to broker1. + ProducerImpl p1 = (ProducerImpl) client1.newProducer(Schema.STRING).topic(topicName) + .sendTimeout(10, TimeUnit.SECONDS).create(); + Topic broker1Topic1 = pulsar1.getBrokerService().getTopic(topicName, false).join().get(); + assertNotNull(broker1Topic1); + clearPreferBroker(); + + // Inject a ZK session expire error, and wait for broker1 to offline. + metadataZKProxy.rejectAllConnections(); + metadataZKProxy.disconnectFrontChannels(); + Awaitility.await().untilAsserted(() -> { + assertEquals(getAvailableBrokers(pulsar2).size(), 1); + }); + + // Send messages continuously. + // Verify: the topic was transferred to broker2. + CompletableFuture broker1Send1 = p1.sendAsync("broker1_msg1"); + Producer p2 = client2.newProducer(Schema.STRING).topic(topicName) + .sendTimeout(10, TimeUnit.SECONDS).create(); + CompletableFuture broker2Send1 = p2.sendAsync("broker2_msg1"); + Awaitility.await().untilAsserted(() -> { + CompletableFuture> future = pulsar2.getBrokerService().getTopic(topicName, false); + assertNotNull(future); + assertTrue(future.isDone() && !future.isCompletedExceptionally()); + Optional optional = future.join(); + assertTrue(optional != null && !optional.isEmpty()); + }); + + // Both two brokers assumed they are the owner of the topic. + Topic broker1Topic2 = pulsar1.getBrokerService().getTopic(topicName, false).join().get(); + Topic broker2Topic2 = pulsar2.getBrokerService().getTopic(topicName, false).join().get(); + assertNotNull(broker1Topic2); + assertNotNull(broker2Topic2); + + // Send messages continuously. + // Publishing to broker-1 will fail. + // Publishing to broker-2 will success. + CompletableFuture broker1Send2 = p1.sendAsync("broker1_msg2"); + CompletableFuture broker2Send2 = p2.sendAsync("broker2_msg2"); + try { + broker1Send1.join(); + broker1Send2.join(); + p1.getClientCnx(); + fail("expected a publish error"); + } catch (Exception ex) { + // Expected. + } + broker2Send1.join(); + broker2Send2.join(); + + // Broker rebuild ZK session. + metadataZKProxy.unRejectAllConnections(); + Awaitility.await().untilAsserted(() -> { + assertEquals(getAvailableBrokers(pulsar1).size(), 2); + assertEquals(getAvailableBrokers(pulsar2).size(), 2); + }); + + // Verify: the topic on broker-1 will be unloaded. + // 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()); + }); + Topic broker2Topic3 = pulsar2.getBrokerService().getTopic(topicName, false).join().get(); + assertNotNull(broker2Topic3); + + // Send messages continuously. + // Verify: p1.send will success(it will connect to broker-2). + // Verify: p2.send will success. + CompletableFuture broker1Send3 = p1.sendAsync("broker1_msg3"); + CompletableFuture broker2Send3 = p2.sendAsync("broker2_msg3"); + broker1Send3.join(); + broker2Send3.join(); + + long msgBacklog = admin2.topics().getStats(topicName).getSubscriptions().get("s1").getMsgBacklog(); + log.info("msgBacklog: {}", msgBacklog); + + // cleanup. + p1.close(); + p2.close(); + admin2.topics().delete(topicName, false); + } +} 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 d099d79d05c4d..8d7743dd791f1 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 @@ -487,6 +487,16 @@ public void registerSessionListener(Consumer listener) { protected void receivedSessionEvent(SessionEvent event) { isConnected = event.isConnected(); + + // Clear cache after session expired. + if (event == SessionEvent.SessionReestablished || event == SessionEvent.Reconnected) { + for (MetadataCacheImpl metadataCache : metadataCaches) { + metadataCache.invalidateAll(); + } + invalidateAll(); + } + + // Notice listeners. try { executor.execute(() -> { sessionListeners.forEach(l -> { From f5eebae98737601adc222d8b3e78fc61c52464bf Mon Sep 17 00:00:00 2001 From: Baodi Shi Date: Mon, 29 Jul 2024 22:02:29 +0800 Subject: [PATCH 637/699] [improve][broker] Optimize the performance of individual acknowledgments (#23072) (cherry picked from commit 77b6378ae8b9ac83962f71063ad44d6ac57f8e32) (cherry picked from commit 487913eef153003c2369fb71e91e2c3373142836) --- .../pulsar/broker/service/Consumer.java | 151 ++++++++---------- 1 file changed, 69 insertions(+), 82 deletions(-) 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 6687d984be24d..8493a687a6c1f 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 @@ -43,6 +43,7 @@ import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.commons.lang3.tuple.MutablePair; +import org.apache.commons.lang3.tuple.Pair; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationDataSubscription; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; @@ -506,14 +507,16 @@ public CompletableFuture messageAcked(CommandAck ack) { //this method is for individual ack not carry the transaction private CompletableFuture individualAckNormal(CommandAck ack, Map properties) { - List positionsAcked = new ArrayList<>(); + List> positionsAcked = new ArrayList<>(); long totalAckCount = 0; for (int i = 0; i < ack.getMessageIdsCount(); i++) { MessageIdData msgId = ack.getMessageIdAt(i); PositionImpl position; - long ackedCount = 0; - long batchSize = getBatchSize(msgId); - Consumer ackOwnerConsumer = getAckOwnerConsumer(msgId.getLedgerId(), msgId.getEntryId()); + Pair ackOwnerConsumerAndBatchSize = + getAckOwnerConsumerAndBatchSize(msgId.getLedgerId(), msgId.getEntryId()); + Consumer ackOwnerConsumer = ackOwnerConsumerAndBatchSize.getLeft(); + long ackedCount; + long batchSize = ackOwnerConsumerAndBatchSize.getRight(); if (msgId.getAckSetsCount() > 0) { long[] ackSets = new long[msgId.getAckSetsCount()]; for (int j = 0; j < msgId.getAckSetsCount(); j++) { @@ -532,28 +535,32 @@ private CompletableFuture individualAckNormal(CommandAck ack, Map completableFuture = new CompletableFuture<>(); completableFuture.complete(totalAckCount); if (isTransactionEnabled() && Subscription.isIndividualAckMode(subType)) { - completableFuture.whenComplete((v, e) -> positionsAcked.forEach(position -> { + completableFuture.whenComplete((v, e) -> positionsAcked.forEach(positionPair -> { + Consumer ackOwnerConsumer = positionPair.getLeft(); + Position position = positionPair.getRight(); //check if the position can remove from the consumer pending acks. // the bit set is empty in pending ack handle. if (((PositionImpl) position).getAckSet() != null) { if (((PersistentSubscription) subscription) .checkIsCanDeleteConsumerPendingAck((PositionImpl) position)) { - removePendingAcks((PositionImpl) position); + removePendingAcks(ackOwnerConsumer, (PositionImpl) position); } } })); @@ -565,7 +572,7 @@ private CompletableFuture individualAckNormal(CommandAck ack, Map individualAckWithTransaction(CommandAck ack) { // Individual ack - List> positionsAcked = new ArrayList<>(); + List>> positionsAcked = new ArrayList<>(); if (!isTransactionEnabled()) { return FutureUtil.failedFuture( new BrokerServiceException.NotAllowedException("Server don't support transaction ack!")); @@ -575,20 +582,23 @@ private CompletableFuture individualAckWithTransaction(CommandAck ack) { for (int i = 0; i < ack.getMessageIdsCount(); i++) { MessageIdData msgId = ack.getMessageIdAt(i); PositionImpl position = PositionImpl.get(msgId.getLedgerId(), msgId.getEntryId()); + Consumer ackOwnerConsumer = getAckOwnerConsumerAndBatchSize(msgId.getLedgerId(), + msgId.getEntryId()).getLeft(); // acked count at least one - long ackedCount = 0; - long batchSize = 0; + long ackedCount; + long batchSize; if (msgId.hasBatchSize()) { batchSize = msgId.getBatchSize(); // ack batch messages set ackeCount = batchSize ackedCount = msgId.getBatchSize(); - positionsAcked.add(new MutablePair<>(position, msgId.getBatchSize())); + positionsAcked.add(Pair.of(ackOwnerConsumer, new MutablePair<>(position, msgId.getBatchSize()))); } else { // ack no batch message set ackedCount = 1 + batchSize = 0; ackedCount = 1; - positionsAcked.add(new MutablePair<>(position, (int) batchSize)); + positionsAcked.add(Pair.of(ackOwnerConsumer, new MutablePair<>(position, (int) batchSize))); } - Consumer ackOwnerConsumer = getAckOwnerConsumer(msgId.getLedgerId(), msgId.getEntryId()); + if (msgId.getAckSetsCount() > 0) { long[] ackSets = new long[msgId.getAckSetsCount()]; for (int j = 0; j < msgId.getAckSetsCount(); j++) { @@ -600,7 +610,7 @@ private CompletableFuture individualAckWithTransaction(CommandAck ack) { addAndGetUnAckedMsgs(ackOwnerConsumer, -(int) ackedCount); - checkCanRemovePendingAcksAndHandle(position, msgId); + checkCanRemovePendingAcksAndHandle(ackOwnerConsumer, position, msgId); checkAckValidationError(ack, position); @@ -608,14 +618,16 @@ private CompletableFuture individualAckWithTransaction(CommandAck ack) { } CompletableFuture completableFuture = transactionIndividualAcknowledge(ack.getTxnidMostBits(), - ack.getTxnidLeastBits(), positionsAcked); + ack.getTxnidLeastBits(), positionsAcked.stream().map(Pair::getRight).collect(Collectors.toList())); if (Subscription.isIndividualAckMode(subType)) { completableFuture.whenComplete((v, e) -> - positionsAcked.forEach(positionLongMutablePair -> { + positionsAcked.forEach(positionPair -> { + Consumer ackOwnerConsumer = positionPair.getLeft(); + MutablePair positionLongMutablePair = positionPair.getRight(); if (positionLongMutablePair.getLeft().getAckSet() != null) { if (((PersistentSubscription) subscription) .checkIsCanDeleteConsumerPendingAck(positionLongMutablePair.left)) { - removePendingAcks(positionLongMutablePair.left); + removePendingAcks(ackOwnerConsumer, positionLongMutablePair.left); } } })); @@ -623,24 +635,6 @@ private CompletableFuture individualAckWithTransaction(CommandAck ack) { return completableFuture.thenApply(__ -> totalAckCount.sum()); } - private long getBatchSize(MessageIdData msgId) { - long batchSize = 1; - if (Subscription.isIndividualAckMode(subType)) { - LongPair longPair = pendingAcks.get(msgId.getLedgerId(), msgId.getEntryId()); - // Consumer may ack the msg that not belongs to it. - if (longPair == null) { - Consumer ackOwnerConsumer = getAckOwnerConsumer(msgId.getLedgerId(), msgId.getEntryId()); - longPair = ackOwnerConsumer.getPendingAcks().get(msgId.getLedgerId(), msgId.getEntryId()); - if (longPair != null) { - batchSize = longPair.first; - } - } else { - batchSize = longPair.first; - } - } - return batchSize; - } - private long getAckedCountForMsgIdNoAckSets(long batchSize, PositionImpl position, Consumer consumer) { if (isAcknowledgmentAtBatchIndexLevelEnabled && Subscription.isIndividualAckMode(subType)) { long[] cursorAckSet = getCursorAckSet(position); @@ -700,26 +694,39 @@ private void checkAckValidationError(CommandAck ack, PositionImpl position) { } } - private boolean checkCanRemovePendingAcksAndHandle(PositionImpl position, MessageIdData msgId) { + private boolean checkCanRemovePendingAcksAndHandle(Consumer ackOwnedConsumer, + PositionImpl position, MessageIdData msgId) { if (Subscription.isIndividualAckMode(subType) && msgId.getAckSetsCount() == 0) { - return removePendingAcks(position); + return removePendingAcks(ackOwnedConsumer, position); } return false; } - private Consumer getAckOwnerConsumer(long ledgerId, long entryId) { - Consumer ackOwnerConsumer = this; + /** + * Retrieves the acknowledgment owner consumer and batch size for the specified ledgerId and entryId. + * + * @param ledgerId The ID of the ledger. + * @param entryId The ID of the entry. + * @return Pair + */ + private Pair getAckOwnerConsumerAndBatchSize(long ledgerId, long entryId) { if (Subscription.isIndividualAckMode(subType)) { - if (!getPendingAcks().containsKey(ledgerId, entryId)) { + LongPair longPair = getPendingAcks().get(ledgerId, entryId); + if (longPair != null) { + return Pair.of(this, longPair.first); + } 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 && consumer.getPendingAcks().containsKey(ledgerId, entryId)) { - ackOwnerConsumer = consumer; - break; + if (consumer != this) { + longPair = consumer.getPendingAcks().get(ledgerId, entryId); + if (longPair != null) { + return Pair.of(consumer, longPair.first); + } } } } } - return ackOwnerConsumer; + return Pair.of(this, 1L); } private long[] getCursorAckSet(PositionImpl position) { @@ -971,44 +978,24 @@ public int hashCode() { * * @param position */ - private boolean removePendingAcks(PositionImpl position) { - Consumer ackOwnedConsumer = null; - if (pendingAcks.get(position.getLedgerId(), position.getEntryId()) == null) { - for (Consumer consumer : subscription.getConsumers()) { - if (!consumer.equals(this) && consumer.getPendingAcks().containsKey(position.getLedgerId(), - position.getEntryId())) { - ackOwnedConsumer = consumer; - break; - } - } - } else { - ackOwnedConsumer = this; + private boolean removePendingAcks(Consumer ackOwnedConsumer, PositionImpl position) { + if (!ackOwnedConsumer.getPendingAcks().remove(position.getLedgerId(), position.getEntryId())) { + // Message was already removed by the other consumer + return false; } - - // remove pending message from appropriate consumer and unblock unAckMsg-flow if requires - LongPair ackedPosition = ackOwnedConsumer != null - ? ackOwnedConsumer.getPendingAcks().get(position.getLedgerId(), position.getEntryId()) - : null; - if (ackedPosition != null) { - if (!ackOwnedConsumer.getPendingAcks().remove(position.getLedgerId(), position.getEntryId())) { - // Message was already removed by the other consumer - return false; - } - if (log.isDebugEnabled()) { - log.debug("[{}-{}] consumer {} received ack {}", topicName, subscription, consumerId, position); - } - // unblock consumer-throttling when limit check is disabled or receives half of maxUnackedMessages => - // consumer can start again consuming messages - int unAckedMsgs = UNACKED_MESSAGES_UPDATER.get(ackOwnedConsumer); - if ((((unAckedMsgs <= getMaxUnackedMessages() / 2) && ackOwnedConsumer.blockedConsumerOnUnackedMsgs) - && ackOwnedConsumer.shouldBlockConsumerOnUnackMsgs()) - || !shouldBlockConsumerOnUnackMsgs()) { - ackOwnedConsumer.blockedConsumerOnUnackedMsgs = false; - flowConsumerBlockedPermits(ackOwnedConsumer); - } - return true; + if (log.isDebugEnabled()) { + log.debug("[{}-{}] consumer {} received ack {}", topicName, subscription, consumerId, position); } - return false; + // unblock consumer-throttling when limit check is disabled or receives half of maxUnackedMessages => + // consumer can start again consuming messages + int unAckedMsgs = UNACKED_MESSAGES_UPDATER.get(ackOwnedConsumer); + if ((((unAckedMsgs <= getMaxUnackedMessages() / 2) && ackOwnedConsumer.blockedConsumerOnUnackedMsgs) + && ackOwnedConsumer.shouldBlockConsumerOnUnackMsgs()) + || !shouldBlockConsumerOnUnackMsgs()) { + ackOwnedConsumer.blockedConsumerOnUnackedMsgs = false; + flowConsumerBlockedPermits(ackOwnedConsumer); + } + return true; } public ConcurrentLongLongPairHashMap getPendingAcks() { From 1ed129df5e33f2151cfc2f2b4b37e4f489748210 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Thu, 1 Aug 2024 15:31:57 +0530 Subject: [PATCH 638/699] Updated license version for Datastax --- .../server/src/assemble/LICENSE.bin.txt | 46 +++++++++---------- pulsar-sql/presto-distribution/LICENSE | 2 +- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 11a3703e5a200..4102eab079775 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -578,29 +578,29 @@ Bouncy Castle License - org.bouncycastle-bcutil-jdk18on-1.78.1.jar Datastax - - com.datastax.oss-managed-ledger-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-auth-oidc-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-auth-sasl-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-pulsar-broker-common-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-pulsar-config-validation-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-pulsar-docs-tools-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-api-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-instance-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-local-runner-original-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-proto-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-runtime-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-secrets-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-utils-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-pulsar-functions-worker-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-pulsar-io-core-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-pulsar-metadata-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-pulsar-proxy-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-pulsar-testclient-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-pulsar-transaction-common-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-pulsar-transaction-coordinator-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-testmocks-3.1.4.4-SNAPSHOT.jar - - com.datastax.oss-jetcd-core-shaded-3.1.4.4-SNAPSHOT-shaded.jar + - com.datastax.oss-managed-ledger-3.1.4.4.jar + - com.datastax.oss-pulsar-broker-3.1.4.4.jar + - com.datastax.oss-pulsar-broker-auth-oidc-3.1.4.4.jar + - com.datastax.oss-pulsar-broker-auth-sasl-3.1.4.4.jar + - com.datastax.oss-pulsar-broker-common-3.1.4.4.jar + - com.datastax.oss-pulsar-config-validation-3.1.4.4.jar + - com.datastax.oss-pulsar-docs-tools-3.1.4.4.jar + - com.datastax.oss-pulsar-functions-api-3.1.4.4.jar + - com.datastax.oss-pulsar-functions-instance-3.1.4.4.jar + - com.datastax.oss-pulsar-functions-local-runner-original-3.1.4.4.jar + - com.datastax.oss-pulsar-functions-proto-3.1.4.4.jar + - com.datastax.oss-pulsar-functions-runtime-3.1.4.4.jar + - com.datastax.oss-pulsar-functions-secrets-3.1.4.4.jar + - com.datastax.oss-pulsar-functions-utils-3.1.4.4.jar + - com.datastax.oss-pulsar-functions-worker-3.1.4.4.jar + - com.datastax.oss-pulsar-io-core-3.1.4.4.jar + - com.datastax.oss-pulsar-metadata-3.1.4.4.jar + - com.datastax.oss-pulsar-proxy-3.1.4.4.jar + - com.datastax.oss-pulsar-testclient-3.1.4.4.jar + - com.datastax.oss-pulsar-transaction-common-3.1.4.4.jar + - com.datastax.oss-pulsar-transaction-coordinator-3.1.4.4.jar + - com.datastax.oss-testmocks-3.1.4.4.jar + - com.datastax.oss-jetcd-core-shaded-3.1.4.4-shaded.jar ------------------------ diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index a92059903e757..c25fbdce36241 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -272,7 +272,7 @@ The Apache Software License, Version 2.0 - grpc-stub-1.55.3.jar - grpc-util-1.60.0.jar * JEtcd - shaded - - jetcd-core-shaded-3.1.4.4-SNAPSHOT-shaded.jar + - jetcd-core-shaded-3.1.4.4-shaded.jar * Vertx - shaded * Joda Time - joda-time-2.10.10.jar From 3d5863ac9996d164ef90fe9dbefd908736aed6a9 Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Thu, 1 Aug 2024 15:42:46 +0530 Subject: [PATCH 639/699] Release 3.1.4.4 --- 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-experimental/pom.xml | 2 +- docker/pulsar/pom.xml | 2 +- jclouds-shaded/pom.xml | 2 +- jetcd-core-shaded/pom.xml | 2 +- managed-ledger/pom.xml | 2 +- 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-client-1x-base/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-1x/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-2x-shaded/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/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/hdfs2/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-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-sql/pom.xml | 2 +- pulsar-sql/presto-distribution/pom.xml | 2 +- pulsar-sql/presto-pulsar-plugin/pom.xml | 2 +- pulsar-sql/presto-pulsar/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 +- 135 files changed, 137 insertions(+), 137 deletions(-) diff --git a/bouncy-castle/bc/pom.xml b/bouncy-castle/bc/pom.xml index 0a8a4c69f12d9..44345c35edcad 100644 --- a/bouncy-castle/bc/pom.xml +++ b/bouncy-castle/bc/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.4.4-SNAPSHOT + 3.1.4.4 .. diff --git a/bouncy-castle/bcfips-include-test/pom.xml b/bouncy-castle/bcfips-include-test/pom.xml index 1aff106d86957..8678de5f6fe2e 100644 --- a/bouncy-castle/bcfips-include-test/pom.xml +++ b/bouncy-castle/bcfips-include-test/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.4.4-SNAPSHOT + 3.1.4.4 .. diff --git a/bouncy-castle/bcfips/pom.xml b/bouncy-castle/bcfips/pom.xml index 61b5bb0c18346..bda5a867c9c20 100644 --- a/bouncy-castle/bcfips/pom.xml +++ b/bouncy-castle/bcfips/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.4.4-SNAPSHOT + 3.1.4.4 .. diff --git a/bouncy-castle/pom.xml b/bouncy-castle/pom.xml index 962825d3976ad..615d8c53cbfc1 100644 --- a/bouncy-castle/pom.xml +++ b/bouncy-castle/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.4.4-SNAPSHOT + 3.1.4.4 .. diff --git a/buildtools/pom.xml b/buildtools/pom.xml index a53eea7f27dc9..18f4bb49dc3d6 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -31,12 +31,12 @@ com.datastax.oss buildtools - 3.1.4.4-SNAPSHOT + 3.1.4.4 jar Pulsar Build Tools - 2024-07-02T12:36:26Z + 2024-08-01T10:12:46Z 1.8 1.8 3.1.0 diff --git a/distribution/io/pom.xml b/distribution/io/pom.xml index 67a7ad016831d..ce25c4bfffb54 100644 --- a/distribution/io/pom.xml +++ b/distribution/io/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.4.4-SNAPSHOT + 3.1.4.4 .. diff --git a/distribution/offloaders/pom.xml b/distribution/offloaders/pom.xml index aa84f94b418f3..05dd2c767322e 100644 --- a/distribution/offloaders/pom.xml +++ b/distribution/offloaders/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.4.4-SNAPSHOT + 3.1.4.4 .. diff --git a/distribution/pom.xml b/distribution/pom.xml index 65be35feee8d2..cc246148b1546 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.4.4-SNAPSHOT + 3.1.4.4 .. diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index b822c728eba0b..4c7e6e0fae181 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.4.4-SNAPSHOT + 3.1.4.4 .. diff --git a/distribution/shell/pom.xml b/distribution/shell/pom.xml index 18eec73d9b143..a6a58ecdcf87a 100644 --- a/distribution/shell/pom.xml +++ b/distribution/shell/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.4.4-SNAPSHOT + 3.1.4.4 .. diff --git a/docker/pom.xml b/docker/pom.xml index d79b2730eed39..8edd5a61b9e42 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.4.4-SNAPSHOT + 3.1.4.4 docker-images Apache Pulsar :: Docker Images diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index 8579a911f553c..aa29973634fd3 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.4.4-SNAPSHOT + 3.1.4.4 4.0.0 pulsar-all-docker-image diff --git a/docker/pulsar-experimental/pom.xml b/docker/pulsar-experimental/pom.xml index c9752f9364b8f..6e995fc301f42 100644 --- a/docker/pulsar-experimental/pom.xml +++ b/docker/pulsar-experimental/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.4.4-SNAPSHOT + 3.1.4.4 4.0.0 pulsar-experimental-docker-image diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 2752f59f44916..d017c156a60e4 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.4.4-SNAPSHOT + 3.1.4.4 4.0.0 pulsar-docker-image diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml index 5bdaa9c3404b4..51918d89a4879 100644 --- a/jclouds-shaded/pom.xml +++ b/jclouds-shaded/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.4.4-SNAPSHOT + 3.1.4.4 .. diff --git a/jetcd-core-shaded/pom.xml b/jetcd-core-shaded/pom.xml index 1bb74bd902d3e..d4f7208b829a7 100644 --- a/jetcd-core-shaded/pom.xml +++ b/jetcd-core-shaded/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.4.4-SNAPSHOT + 3.1.4.4 jetcd-core-shaded diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index e45283576af72..0b00b64f28a62 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.4.4-SNAPSHOT + 3.1.4.4 .. diff --git a/pom.xml b/pom.xml index 4a8f3fda3adcb..f69b96dde72f0 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ com.datastax.oss pulsar - 3.1.4.4-SNAPSHOT + 3.1.4.4 Pulsar Pulsar is a distributed pub-sub messaging platform with a very @@ -95,7 +95,7 @@ flexible messaging model and an intuitive client API. UTF-8 UTF-8 - 2024-07-02T12:36:26Z + 2024-08-01T10:12:46Z true + + + + + + + + + + + + + + + + + + diff --git a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdBase.java b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdBase.java index 381bc8abcaa3f..dd2577e7864c1 100644 --- a/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdBase.java +++ b/pulsar-client-tools/src/main/java/org/apache/pulsar/admin/cli/CmdBase.java @@ -44,10 +44,10 @@ public abstract class CmdBase { private IUsageFormatter usageFormatter; /** - * Default read timeout in milliseconds. - * Used if not found from configuration data in {@link #getReadTimeoutMs()} + * Default request timeout in milliseconds. + * Used if not found from configuration data in {@link #getRequestTimeoutMs()} */ - private static final long DEFAULT_READ_TIMEOUT_MILLIS = 60000; + private static final long DEFAULT_REQUEST_TIMEOUT_MILLIS = 60000; @Parameter(names = { "--help", "-h" }, help = true, hidden = true) private boolean help = false; @@ -136,17 +136,17 @@ protected PulsarAdmin getAdmin() { return admin; } - protected long getReadTimeoutMs() { + protected long getRequestTimeoutMs() { PulsarAdmin pulsarAdmin = getAdmin(); if (pulsarAdmin instanceof PulsarAdminImpl) { - return ((PulsarAdminImpl) pulsarAdmin).getClientConfigData().getReadTimeoutMs(); + return ((PulsarAdminImpl) pulsarAdmin).getClientConfigData().getRequestTimeoutMs(); } - return DEFAULT_READ_TIMEOUT_MILLIS; + return DEFAULT_REQUEST_TIMEOUT_MILLIS; } protected T sync(Supplier> executor) throws PulsarAdminException { try { - return executor.get().get(getReadTimeoutMs(), TimeUnit.MILLISECONDS); + return executor.get().get(getRequestTimeoutMs(), TimeUnit.MILLISECONDS); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new PulsarAdminException(e); From fcb4809ce751458b1c6e0d2e766eb957dcd6567f Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Wed, 7 Aug 2024 13:52:55 +0300 Subject: [PATCH 657/699] [improve][fn] Add support for overriding additionalJavaRuntimeArguments with PF_additionalJavaRuntimeArguments env (#23130) (cherry picked from commit 8707fbe8351fb6ac4337fbd88d86eb32aff55b04) (cherry picked from commit be0fca6bb6db9a3fab7e5ebb6317f8a31cccb28a) --- docker/pulsar/scripts/gen-yml-from-env.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docker/pulsar/scripts/gen-yml-from-env.py b/docker/pulsar/scripts/gen-yml-from-env.py index aa40408ed5b1f..916b147f0cbba 100755 --- a/docker/pulsar/scripts/gen-yml-from-env.py +++ b/docker/pulsar/scripts/gen-yml-from-env.py @@ -50,6 +50,9 @@ 'brokerClientTlsProtocols', 'webServiceTlsCiphers', 'webServiceTlsProtocols', + 'additionalJavaRuntimeArguments', + 'additionalEnabledConnectorUrlPatterns', + 'additionalEnabledFunctionsUrlPatterns' ] PF_ENV_PREFIX = 'PF_' From 2455b1e5795b746d4dec125f3b971b9215871f6f Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Thu, 8 Aug 2024 01:19:38 -0700 Subject: [PATCH 658/699] [fix][broker] Fix the bug that elected leader thinks it's a follower (#23138) (cherry picked from commit 3560ddb64f44fb2a53d52ef3df0624bb9bda1af6) (cherry picked from commit 90d17e81d56a1cdf1f47edd77b9711d620d4bf7f) --- .../coordination/impl/LeaderElectionImpl.java | 15 ++++- .../pulsar/metadata/LeaderElectionTest.java | 2 + .../apache/pulsar/metadata/ZKSessionTest.java | 55 +++++++++++++++++++ 3 files changed, 69 insertions(+), 3 deletions(-) diff --git a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/LeaderElectionImpl.java b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/LeaderElectionImpl.java index 9e6a9b94c42a3..4fe01ec11e363 100644 --- a/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/LeaderElectionImpl.java +++ b/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/coordination/impl/LeaderElectionImpl.java @@ -133,8 +133,11 @@ private synchronized CompletableFuture handleExistingLeader // If the value is the same as our proposed value, it means this instance was the leader at some // point before. The existing value can either be for this same session or for a previous one. if (res.getStat().isCreatedBySelf()) { + log.info("Keeping the existing value {} for {} as it's from the same session stat={}", existingValue, + path, res.getStat()); // The value is still valid because it was created in the same session changeState(LeaderElectionState.Leading); + return CompletableFuture.completedFuture(LeaderElectionState.Leading); } else { // Since the value was created in a different session, it might be expiring. We need to delete it // and try the election again. @@ -257,7 +260,13 @@ public synchronized CompletableFuture asyncClose() { return CompletableFuture.completedFuture(null); } - return store.delete(path, version); + return store.delete(path, version) + .thenAccept(__ -> { + synchronized (LeaderElectionImpl.this) { + leaderElectionState = LeaderElectionState.NoLeader; + } + } + ); } @Override @@ -278,8 +287,8 @@ public Optional getLeaderValueIfPresent() { private void handleSessionNotification(SessionEvent event) { // Ensure we're only processing one session event at a time. sequencer.sequential(() -> FutureUtil.composeAsync(() -> { - if (event == SessionEvent.SessionReestablished) { - log.info("Revalidating leadership for {}", path); + if (event == SessionEvent.Reconnected || event == SessionEvent.SessionReestablished) { + log.info("Revalidating leadership for {}, event:{}", path, event); return elect().thenAccept(leaderState -> { log.info("Resynced leadership for {} - State: {}", path, leaderState); }).exceptionally(ex -> { diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/LeaderElectionTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/LeaderElectionTest.java index 6b4f74a30b563..4b48f3c20b02b 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/LeaderElectionTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/LeaderElectionTest.java @@ -69,6 +69,8 @@ public void basicTest(String provider, Supplier urlSupplier) throws Exce leaderElection.close(); + assertEquals(leaderElection.getState(), LeaderElectionState.NoLeader); + assertEquals(cache.get("/my/leader-election").join(), Optional.empty()); } diff --git a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/ZKSessionTest.java b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/ZKSessionTest.java index 36cb0f132ba58..02d65fd21ed5c 100644 --- a/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/ZKSessionTest.java +++ b/pulsar-metadata/src/test/java/org/apache/pulsar/metadata/ZKSessionTest.java @@ -27,6 +27,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import lombok.Cleanup; +import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.pulsar.metadata.api.MetadataStoreConfig; import org.apache.pulsar.metadata.api.coordination.CoordinationService; import org.apache.pulsar.metadata.api.coordination.LeaderElection; @@ -180,4 +181,58 @@ public void testReacquireLeadershipAfterSessionLost() throws Exception { .untilAsserted(()-> assertEquals(le1.getState(),LeaderElectionState.Leading)); assertTrue(store.get(path).join().isPresent()); } + + + @Test + public void testElectAfterReconnected() throws Exception { + // --- init + @Cleanup + MetadataStoreExtended store = MetadataStoreExtended.create(zks.getConnectionString(), + MetadataStoreConfig.builder() + .sessionTimeoutMillis(2_000) + .build()); + + + BlockingQueue sessionEvents = new LinkedBlockingQueue<>(); + store.registerSessionListener(sessionEvents::add); + BlockingQueue leaderElectionEvents = new LinkedBlockingQueue<>(); + String path = newKey(); + + @Cleanup + CoordinationService coordinationService = new CoordinationServiceImpl(store); + @Cleanup + LeaderElection le1 = coordinationService.getLeaderElection(String.class, path, + leaderElectionEvents::add); + + // --- test manual elect + String proposed = "value-1"; + le1.elect(proposed).join(); + assertEquals(le1.getState(), LeaderElectionState.Leading); + LeaderElectionState les = leaderElectionEvents.poll(5, TimeUnit.SECONDS); + assertEquals(les, LeaderElectionState.Leading); + + + // simulate no leader state + FieldUtils.writeDeclaredField(le1, "leaderElectionState", LeaderElectionState.NoLeader, true); + + // reconnect + zks.stop(); + + SessionEvent e = sessionEvents.poll(5, TimeUnit.SECONDS); + assertEquals(e, SessionEvent.ConnectionLost); + + zks.start(); + + + // --- test le1 can be leader + e = sessionEvents.poll(10, TimeUnit.SECONDS); + assertEquals(e, SessionEvent.Reconnected); + Awaitility.await().atMost(Duration.ofSeconds(15)) + .untilAsserted(()-> { + assertEquals(le1.getState(),LeaderElectionState.Leading); + }); // reacquire leadership + + + assertTrue(store.get(path).join().isPresent()); + } } From 5149aef4e23fd2a40a9ad4d4ebfeccd226860db7 Mon Sep 17 00:00:00 2001 From: Zixuan Liu Date: Mon, 18 Mar 2024 22:14:11 +0800 Subject: [PATCH 659/699] [improve][misc] Upgrade jersey to 2.41 (#22290) (cherry picked from commit 4ddcd497efa3fcaf377eec86090f98212932b6e5) --- .../server/src/assemble/LICENSE.bin.txt | 20 +++++++++---------- .../shell/src/assemble/LICENSE.bin.txt | 14 ++++++------- pom.xml | 2 +- pulsar-sql/presto-distribution/LICENSE | 20 +++++++++---------- pulsar-sql/presto-distribution/pom.xml | 2 +- 5 files changed, 29 insertions(+), 29 deletions(-) diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 5bc7175c0e95d..344e65aa4086a 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -543,16 +543,16 @@ CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt - org.glassfish.hk2-osgi-resource-locator-1.0.3.jar - org.glassfish.hk2.external-aopalliance-repackaged-2.6.1.jar * Jersey - - org.glassfish.jersey.containers-jersey-container-servlet-2.34.jar - - org.glassfish.jersey.containers-jersey-container-servlet-core-2.34.jar - - org.glassfish.jersey.core-jersey-client-2.34.jar - - org.glassfish.jersey.core-jersey-common-2.34.jar - - org.glassfish.jersey.core-jersey-server-2.34.jar - - org.glassfish.jersey.ext-jersey-entity-filtering-2.34.jar - - org.glassfish.jersey.media-jersey-media-json-jackson-2.34.jar - - org.glassfish.jersey.media-jersey-media-multipart-2.34.jar - - org.glassfish.jersey.inject-jersey-hk2-2.34.jar - * Mimepull -- org.jvnet.mimepull-mimepull-1.9.13.jar + - org.glassfish.jersey.containers-jersey-container-servlet-2.41.jar + - org.glassfish.jersey.containers-jersey-container-servlet-core-2.41.jar + - org.glassfish.jersey.core-jersey-client-2.41.jar + - org.glassfish.jersey.core-jersey-common-2.41.jar + - org.glassfish.jersey.core-jersey-server-2.41.jar + - org.glassfish.jersey.ext-jersey-entity-filtering-2.41.jar + - org.glassfish.jersey.media-jersey-media-json-jackson-2.41.jar + - org.glassfish.jersey.media-jersey-media-multipart-2.41.jar + - org.glassfish.jersey.inject-jersey-hk2-2.41.jar + * Mimepull -- org.jvnet.mimepull-mimepull-1.9.15.jar Eclipse Distribution License 1.0 -- ../licenses/LICENSE-EDL-1.0.txt * Jakarta Activation diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 233fc3d22b857..8e164473b7ecb 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -438,13 +438,13 @@ CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt - aopalliance-repackaged-2.6.1.jar - osgi-resource-locator-1.0.3.jar * Jersey - - jersey-client-2.34.jar - - jersey-common-2.34.jar - - jersey-entity-filtering-2.34.jar - - jersey-media-json-jackson-2.34.jar - - jersey-media-multipart-2.34.jar - - jersey-hk2-2.34.jar - * Mimepull -- mimepull-1.9.13.jar + - jersey-client-2.41.jar + - jersey-common-2.41.jar + - jersey-entity-filtering-2.41.jar + - jersey-media-json-jackson-2.41.jar + - jersey-media-multipart-2.41.jar + - jersey-hk2-2.41.jar + * Mimepull -- mimepull-1.9.15.jar Eclipse Distribution License 1.0 -- ../licenses/LICENSE-EDL-1.0.txt * Jakarta Activation diff --git a/pom.xml b/pom.xml index c664ea1aa2986..546daa7a7fad7 100644 --- a/pom.xml +++ b/pom.xml @@ -147,7 +147,7 @@ flexible messaging model and an intuitive client API. 0.0.21.Final 9.4.54.v20240208 2.5.2 - 2.34 + 2.41 1.10.50 0.16.0 4.5.8 diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index cb6c13a9053e2..4178f50b3680f 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -535,15 +535,15 @@ CDDL-1.1 -- licenses/LICENSE-CDDL-1.1.txt - aopalliance-repackaged-2.6.1.jar * Jersey - jaxrs-213.jar - - jersey-client-2.34.jar - - jersey-common-2.34.jar - - jersey-container-servlet-2.34.jar - - jersey-container-servlet-core-2.34.jar - - jersey-entity-filtering-2.34.jar - - jersey-hk2-2.34.jar - - jersey-media-json-jackson-2.34.jar - - jersey-media-multipart-2.34.jar - - jersey-server-2.34.jar + - jersey-client-2.41.jar + - jersey-common-2.41.jar + - jersey-container-servlet-2.41.jar + - jersey-container-servlet-core-2.41.jar + - jersey-entity-filtering-2.41.jar + - jersey-hk2-2.41.jar + - jersey-media-json-jackson-2.41.jar + - jersey-media-multipart-2.41.jar + - jersey-server-2.41.jar * JAXB - jaxb-api-2.3.1.jar - jaxb-runtime-2.3.4.jar @@ -560,7 +560,7 @@ CDDL-1.1 -- licenses/LICENSE-CDDL-1.1.txt * Logback Core Module - logback-core-1.2.3.jar * MIME Streaming Extension - - mimepull-1.9.13.jar + - mimepull-1.9.15.jar Eclipse Public License - v2.0 -- licenses/LICENSE-EPL-2.0.txt * jakarta.annotation-api-1.3.5.jar diff --git a/pulsar-sql/presto-distribution/pom.xml b/pulsar-sql/presto-distribution/pom.xml index 563bd172fbab8..f8f3420e49f6f 100644 --- a/pulsar-sql/presto-distribution/pom.xml +++ b/pulsar-sql/presto-distribution/pom.xml @@ -33,7 +33,7 @@ false - 2.34 + 2.41 2.6 0.0.12 3.0.5 From 62fbcc9645c23cd21b467110cb2fb5f7c8968080 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Thu, 8 Aug 2024 15:04:04 +0300 Subject: [PATCH 660/699] [improve][client] Add maxConnectionsPerHost and connectionMaxIdleSeconds to PulsarAdminBuilder (#22541) (cherry picked from commit 3e7dbb4957bf5daae59127cd66e4da3802072853) (cherry picked from commit 4c480fd4ec16af1ffeac5898b60415a503f6ed02) --- .../server/src/assemble/LICENSE.bin.txt | 20 +- .../shell/src/assemble/LICENSE.bin.txt | 13 +- pom.xml | 9 +- .../client/admin/PulsarAdminBuilder.java | 26 ++ pulsar-client-admin-shaded/pom.xml | 5 + .../client/admin/internal/FunctionsImpl.java | 70 ++-- .../client/admin/internal/PackagesImpl.java | 68 ++-- .../internal/PulsarAdminBuilderImpl.java | 24 ++ .../admin/internal/PulsarAdminImpl.java | 8 +- .../client/admin/internal/SinksImpl.java | 13 +- .../client/admin/internal/SourcesImpl.java | 13 +- .../internal/http/AsyncHttpConnector.java | 351 +++++++++++++----- .../http/AsyncHttpRequestExecutor.java | 48 +++ .../internal/PulsarAdminBuilderImplTest.java | 2 + .../internal/http/AsyncHttpConnectorTest.java | 200 ++++++++++ pulsar-client-all/pom.xml | 5 + .../pulsar/client/api/ClientBuilder.java | 2 + pulsar-client-shaded/pom.xml | 5 + .../pulsar/client/impl/ConnectionPool.java | 2 +- .../impl/conf/ClientConfigurationData.java | 4 +- .../client/impl/ClientBuilderImplTest.java | 2 +- pulsar-common/pom.xml | 5 + pulsar-sql/presto-distribution/LICENSE | 20 +- pulsar-sql/presto-distribution/pom.xml | 2 +- 24 files changed, 687 insertions(+), 230 deletions(-) create mode 100644 pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpRequestExecutor.java diff --git a/distribution/server/src/assemble/LICENSE.bin.txt b/distribution/server/src/assemble/LICENSE.bin.txt index 344e65aa4086a..bf740c4455df8 100644 --- a/distribution/server/src/assemble/LICENSE.bin.txt +++ b/distribution/server/src/assemble/LICENSE.bin.txt @@ -499,6 +499,8 @@ The Apache Software License, Version 2.0 - io.reactivex.rxjava3-rxjava-3.0.1.jar * RoaringBitmap - org.roaringbitmap-RoaringBitmap-1.2.0.jar + * Spotify completable-futures + - com.spotify-completable-futures-0.3.6.jar BSD 3-clause "New" or "Revised" License * Google auth library @@ -543,15 +545,15 @@ CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt - org.glassfish.hk2-osgi-resource-locator-1.0.3.jar - org.glassfish.hk2.external-aopalliance-repackaged-2.6.1.jar * Jersey - - org.glassfish.jersey.containers-jersey-container-servlet-2.41.jar - - org.glassfish.jersey.containers-jersey-container-servlet-core-2.41.jar - - org.glassfish.jersey.core-jersey-client-2.41.jar - - org.glassfish.jersey.core-jersey-common-2.41.jar - - org.glassfish.jersey.core-jersey-server-2.41.jar - - org.glassfish.jersey.ext-jersey-entity-filtering-2.41.jar - - org.glassfish.jersey.media-jersey-media-json-jackson-2.41.jar - - org.glassfish.jersey.media-jersey-media-multipart-2.41.jar - - org.glassfish.jersey.inject-jersey-hk2-2.41.jar + - org.glassfish.jersey.containers-jersey-container-servlet-2.42.jar + - org.glassfish.jersey.containers-jersey-container-servlet-core-2.42.jar + - org.glassfish.jersey.core-jersey-client-2.42.jar + - org.glassfish.jersey.core-jersey-common-2.42.jar + - org.glassfish.jersey.core-jersey-server-2.42.jar + - org.glassfish.jersey.ext-jersey-entity-filtering-2.42.jar + - org.glassfish.jersey.media-jersey-media-json-jackson-2.42.jar + - org.glassfish.jersey.media-jersey-media-multipart-2.42.jar + - org.glassfish.jersey.inject-jersey-hk2-2.42.jar * Mimepull -- org.jvnet.mimepull-mimepull-1.9.15.jar Eclipse Distribution License 1.0 -- ../licenses/LICENSE-EDL-1.0.txt diff --git a/distribution/shell/src/assemble/LICENSE.bin.txt b/distribution/shell/src/assemble/LICENSE.bin.txt index 8e164473b7ecb..728ad9f27fefa 100644 --- a/distribution/shell/src/assemble/LICENSE.bin.txt +++ b/distribution/shell/src/assemble/LICENSE.bin.txt @@ -409,6 +409,7 @@ The Apache Software License, Version 2.0 * Apache Avro - avro-1.11.3.jar - avro-protobuf-1.11.3.jar + * Spotify completable-futures -- completable-futures-0.3.6.jar BSD 3-clause "New" or "Revised" License * JSR305 -- jsr305-3.0.2.jar -- ../licenses/LICENSE-JSR305.txt @@ -438,12 +439,12 @@ CDDL-1.1 -- ../licenses/LICENSE-CDDL-1.1.txt - aopalliance-repackaged-2.6.1.jar - osgi-resource-locator-1.0.3.jar * Jersey - - jersey-client-2.41.jar - - jersey-common-2.41.jar - - jersey-entity-filtering-2.41.jar - - jersey-media-json-jackson-2.41.jar - - jersey-media-multipart-2.41.jar - - jersey-hk2-2.41.jar + - jersey-client-2.42.jar + - jersey-common-2.42.jar + - jersey-entity-filtering-2.42.jar + - jersey-media-json-jackson-2.42.jar + - jersey-media-multipart-2.42.jar + - jersey-hk2-2.42.jar * Mimepull -- mimepull-1.9.15.jar Eclipse Distribution License 1.0 -- ../licenses/LICENSE-EDL-1.0.txt diff --git a/pom.xml b/pom.xml index 546daa7a7fad7..143200dc1bd88 100644 --- a/pom.xml +++ b/pom.xml @@ -147,7 +147,7 @@ flexible messaging model and an intuitive client API. 0.0.21.Final 9.4.54.v20240208 2.5.2 - 2.41 + 2.42 1.10.50 0.16.0 4.5.8 @@ -255,6 +255,7 @@ flexible messaging model and an intuitive client API. 3.4.3 1.5.2-3 2.0.6 + 0.3.6 3.3.2 @@ -646,6 +647,12 @@ flexible messaging model and an intuitive client API. ${bookkeeper.version} + + com.spotify + completable-futures + ${completable-futures.version} + + org.rocksdb rocksdbjni diff --git a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java index 1260555a7c43f..55f2fa6ed849a 100644 --- a/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java +++ b/pulsar-client-admin-api/src/main/java/org/apache/pulsar/client/admin/PulsarAdminBuilder.java @@ -327,4 +327,30 @@ PulsarAdminBuilder authentication(String authPluginClassName, Map + * By default, the connection pool maintains up to 16 connections to a single host. This method allows you to + * modify this default behavior and limit the number of connections. + *

    + * This setting can be useful in scenarios where you want to limit the resources used by the client library, + * or control the level of parallelism for operations so that a single client does not overwhelm + * the Pulsar cluster with too many concurrent connections. + * + * @param maxConnectionsPerHost the maximum number of connections to establish per host. Set to <= 0 to disable + * the limit. + * @return the PulsarAdminBuilder instance, allowing for method chaining + */ + PulsarAdminBuilder maxConnectionsPerHost(int maxConnectionsPerHost); + + /** + * Sets the maximum idle time for a pooled connection. If a connection is idle for more than the specified + * amount of seconds, it will be released back to the connection pool. + * Defaults to 25 seconds. + * + * @param connectionMaxIdleSeconds the maximum idle time, in seconds, for a pooled connection + * @return the PulsarAdminBuilder instance + */ + PulsarAdminBuilder connectionMaxIdleSeconds(int connectionMaxIdleSeconds); } diff --git a/pulsar-client-admin-shaded/pom.xml b/pulsar-client-admin-shaded/pom.xml index 89934f7ac84ad..96f77bf18bf64 100644 --- a/pulsar-client-admin-shaded/pom.xml +++ b/pulsar-client-admin-shaded/pom.xml @@ -123,6 +123,7 @@ com.google.protobuf:protobuf-java com.google.guava:guava com.google.code.gson:gson + com.spotify:completable-futures com.fasterxml.jackson.*:* io.netty:* io.netty.incubator:* @@ -192,6 +193,10 @@ com.google.protobuf.* + + com.spotify.futures + org.apache.pulsar.shade.com.spotify.futures + com.fasterxml.jackson org.apache.pulsar.shade.com.fasterxml.jackson diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/FunctionsImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/FunctionsImpl.java index 97c42e5c1a95a..bfcc3fe39a444 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/FunctionsImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/FunctionsImpl.java @@ -22,7 +22,6 @@ import static org.asynchttpclient.Dsl.post; import static org.asynchttpclient.Dsl.put; import com.google.gson.Gson; -import io.netty.handler.codec.http.HttpHeaders; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -41,6 +40,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.client.admin.Functions; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.admin.internal.http.AsyncHttpRequestExecutor; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.common.functions.FunctionConfig; import org.apache.pulsar.common.functions.FunctionDefinition; @@ -54,10 +54,8 @@ import org.apache.pulsar.common.policies.data.FunctionStats; import org.apache.pulsar.common.policies.data.FunctionStatsImpl; import org.apache.pulsar.common.policies.data.FunctionStatus; -import org.asynchttpclient.AsyncHandler; -import org.asynchttpclient.AsyncHttpClient; +import org.asynchttpclient.AsyncCompletionHandlerBase; import org.asynchttpclient.HttpResponseBodyPart; -import org.asynchttpclient.HttpResponseStatus; import org.asynchttpclient.RequestBuilder; import org.asynchttpclient.request.body.multipart.ByteArrayPart; import org.asynchttpclient.request.body.multipart.FilePart; @@ -70,12 +68,14 @@ public class FunctionsImpl extends ComponentResource implements Functions { private final WebTarget functions; - private final AsyncHttpClient asyncHttpClient; + private final AsyncHttpRequestExecutor asyncHttpRequestExecutor; - public FunctionsImpl(WebTarget web, Authentication auth, AsyncHttpClient asyncHttpClient, long requestTimeoutMs) { + public FunctionsImpl(WebTarget web, Authentication auth, + AsyncHttpRequestExecutor asyncHttpRequestExecutor, + long requestTimeoutMs) { super(auth, requestTimeoutMs); this.functions = web.path("/admin/v3/functions"); - this.asyncHttpClient = asyncHttpClient; + this.asyncHttpRequestExecutor = asyncHttpRequestExecutor; } @Override @@ -171,8 +171,7 @@ public CompletableFuture createFunctionAsync(FunctionConfig functionConfig // If the function code is built in, we don't need to submit here builder.addBodyPart(new FilePart("data", new File(fileName), MediaType.APPLICATION_OCTET_STREAM)); } - asyncHttpClient.executeRequest(addAuthHeaders(functions, builder).build()) - .toCompletableFuture() + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(functions, builder).build()) .thenAccept(response -> { if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { future.completeExceptionally( @@ -263,8 +262,7 @@ public CompletableFuture updateFunctionAsync( builder.addBodyPart(new FilePart("data", new File(fileName), MediaType.APPLICATION_OCTET_STREAM)); } - asyncHttpClient.executeRequest(addAuthHeaders(functions, builder).build()) - .toCompletableFuture() + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(functions, builder).build()) .thenAccept(response -> { if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { future.completeExceptionally( @@ -464,7 +462,7 @@ public CompletableFuture uploadFunctionAsync(String sourceFile, String pat .addBodyPart(new FilePart("data", new File(sourceFile), MediaType.APPLICATION_OCTET_STREAM)) .addBodyPart(new StringPart("path", path, MediaType.TEXT_PLAIN)); - asyncHttpClient.executeRequest(addAuthHeaders(functions, builder).build()).toCompletableFuture() + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(functions, builder).build()) .thenAccept(response -> { if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { future.completeExceptionally( @@ -543,55 +541,31 @@ private CompletableFuture downloadFileAsync(String destinationPath, WebTar RequestBuilder builder = get(target.getUri().toASCIIString()); - CompletableFuture statusFuture = - asyncHttpClient.executeRequest(addAuthHeaders(functions, builder).build(), - new AsyncHandler() { - private HttpResponseStatus status; - - @Override - public State onStatusReceived(HttpResponseStatus responseStatus) throws Exception { - status = responseStatus; - if (status.getStatusCode() != Response.Status.OK.getStatusCode()) { - return State.ABORT; - } - return State.CONTINUE; - } - - @Override - public State onHeadersReceived(HttpHeaders headers) throws Exception { - return State.CONTINUE; - } + CompletableFuture responseFuture = + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(functions, builder).build(), + () -> new AsyncCompletionHandlerBase() { @Override public State onBodyPartReceived(HttpResponseBodyPart bodyPart) throws Exception { os.write(bodyPart.getBodyByteBuffer()); return State.CONTINUE; } + }); - @Override - public HttpResponseStatus onCompleted() throws Exception { - return status; - } - - @Override - public void onThrowable(Throwable t) { - } - }).toCompletableFuture(); - - statusFuture - .whenComplete((status, throwable) -> { + responseFuture + .whenComplete((response, throwable) -> { try { os.close(); } catch (IOException e) { future.completeExceptionally(getApiException(e)); } }) - .thenAccept(status -> { - if (status.getStatusCode() < 200 || status.getStatusCode() >= 300) { + .thenAccept(response -> { + if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { future.completeExceptionally( getApiException(Response - .status(status.getStatusCode()) - .entity(status.getStatusText()) + .status(response.getStatusCode()) + .entity(response.getStatusText()) .build())); } else { future.complete(null); @@ -700,7 +674,7 @@ public CompletableFuture putFunctionStateAsync( .path("state").path(state.getKey()).getUri().toASCIIString()); builder.addBodyPart(new StringPart("state", objectWriter() .writeValueAsString(state), MediaType.APPLICATION_JSON)); - asyncHttpClient.executeRequest(addAuthHeaders(functions, builder).build()) + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(functions, builder).build()) .toCompletableFuture() .thenAccept(response -> { if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { @@ -740,7 +714,7 @@ public CompletableFuture updateOnWorkerLeaderAsync(String tenant, String n .addBodyPart(new ByteArrayPart("functionMetaData", functionMetaData)) .addBodyPart(new StringPart("delete", Boolean.toString(delete))); - asyncHttpClient.executeRequest(addAuthHeaders(functions, builder).build()) + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(functions, builder).build()) .toCompletableFuture() .thenAccept(response -> { if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PackagesImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PackagesImpl.java index d69bef448c12e..2b8efc3b97c8c 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PackagesImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PackagesImpl.java @@ -20,7 +20,6 @@ import static org.asynchttpclient.Dsl.get; import com.google.gson.Gson; -import io.netty.handler.codec.http.HttpHeaders; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -36,15 +35,14 @@ import javax.ws.rs.core.Response; import org.apache.pulsar.client.admin.Packages; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.admin.internal.http.AsyncHttpRequestExecutor; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.packages.management.core.common.PackageMetadata; import org.apache.pulsar.packages.management.core.common.PackageName; -import org.asynchttpclient.AsyncHandler; -import org.asynchttpclient.AsyncHttpClient; +import org.asynchttpclient.AsyncCompletionHandlerBase; import org.asynchttpclient.Dsl; import org.asynchttpclient.HttpResponseBodyPart; -import org.asynchttpclient.HttpResponseStatus; import org.asynchttpclient.RequestBuilder; import org.asynchttpclient.request.body.multipart.FilePart; import org.asynchttpclient.request.body.multipart.StringPart; @@ -55,11 +53,12 @@ public class PackagesImpl extends ComponentResource implements Packages { private final WebTarget packages; - private final AsyncHttpClient httpClient; + private final AsyncHttpRequestExecutor asyncHttpRequestExecutor; - public PackagesImpl(WebTarget webTarget, Authentication auth, AsyncHttpClient client, long requestTimeoutMs) { + public PackagesImpl(WebTarget webTarget, Authentication auth, AsyncHttpRequestExecutor asyncHttpRequestExecutor, + long requestTimeoutMs) { super(auth, requestTimeoutMs); - this.httpClient = client; + this.asyncHttpRequestExecutor = asyncHttpRequestExecutor; this.packages = webTarget.path("/admin/v3/packages"); } @@ -98,7 +97,7 @@ public CompletableFuture uploadAsync(PackageMetadata metadata, String pack .post(packages.path(PackageName.get(packageName).toRestPath()).getUri().toASCIIString()) .addBodyPart(new FilePart("file", new File(path), MediaType.APPLICATION_OCTET_STREAM)) .addBodyPart(new StringPart("metadata", new Gson().toJson(metadata), MediaType.APPLICATION_JSON)); - httpClient.executeRequest(addAuthHeaders(packages, builder).build()) + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(packages, builder).build()) .toCompletableFuture() .thenAccept(response -> { if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { @@ -138,55 +137,30 @@ public CompletableFuture downloadAsync(String packageName, String path) { FileChannel os = new FileOutputStream(destinyPath.toFile()).getChannel(); RequestBuilder builder = get(webTarget.getUri().toASCIIString()); - CompletableFuture statusFuture = - httpClient.executeRequest(addAuthHeaders(webTarget, builder).build(), - new AsyncHandler() { - private HttpResponseStatus status; + CompletableFuture responseFuture = + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(webTarget, builder).build(), + () -> new AsyncCompletionHandlerBase() { - @Override - public State onStatusReceived(HttpResponseStatus httpResponseStatus) throws Exception { - status = httpResponseStatus; - if (status.getStatusCode() != Response.Status.OK.getStatusCode()) { - return State.ABORT; + @Override + public State onBodyPartReceived(HttpResponseBodyPart bodyPart) throws Exception { + os.write(bodyPart.getBodyByteBuffer()); + return State.CONTINUE; } - return State.CONTINUE; - } - - @Override - public State onHeadersReceived(HttpHeaders httpHeaders) throws Exception { - return State.CONTINUE; - } - - @Override - public State onBodyPartReceived(HttpResponseBodyPart httpResponseBodyPart) throws Exception { - os.write(httpResponseBodyPart.getBodyByteBuffer()); - return State.CONTINUE; - } - - @Override - public void onThrowable(Throwable throwable) { - // we don't need to handle that throwable and use the returned future to handle it. - } - - @Override - public HttpResponseStatus onCompleted() throws Exception { - return status; - } - }).toCompletableFuture(); - statusFuture - .whenComplete((status, throwable) -> { + }); + responseFuture + .whenComplete((response, throwable) -> { try { os.close(); } catch (IOException e) { future.completeExceptionally(getApiException(throwable)); } }) - .thenAccept(status -> { - if (status.getStatusCode() < 200 || status.getStatusCode() >= 300) { + .thenAccept(response -> { + if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { future.completeExceptionally( getApiException(Response - .status(status.getStatusCode()) - .entity(status.getStatusText()) + .status(response.getStatusCode()) + .entity(response.getStatusText()) .build())); } else { future.complete(null); diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java index 009fa67fbaa29..5f4b6a6cf3492 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImpl.java @@ -46,6 +46,7 @@ public PulsarAdmin build() throws PulsarClientException { public PulsarAdminBuilderImpl() { this.conf = new ClientConfigurationData(); + this.conf.setConnectionsPerBroker(16); } private PulsarAdminBuilderImpl(ClientConfigurationData conf) { @@ -61,6 +62,15 @@ public PulsarAdminBuilder clone() { public PulsarAdminBuilder loadConf(Map config) { conf = ConfigurationDataUtils.loadData(config, conf, ClientConfigurationData.class); setAuthenticationFromPropsIfAvailable(conf); + // in ClientConfigurationData, the maxConnectionsPerHost maps to connectionsPerBroker + if (config.containsKey("maxConnectionsPerHost")) { + Object maxConnectionsPerHostObj = config.get("maxConnectionsPerHost"); + if (maxConnectionsPerHostObj instanceof Integer) { + maxConnectionsPerHost((Integer) maxConnectionsPerHostObj); + } else { + maxConnectionsPerHost(Integer.parseInt(maxConnectionsPerHostObj.toString())); + } + } return this; } @@ -227,4 +237,18 @@ public PulsarAdminBuilder setContextClassLoader(ClassLoader clientBuilderClassLo this.clientBuilderClassLoader = clientBuilderClassLoader; return this; } + + @Override + public PulsarAdminBuilder maxConnectionsPerHost(int maxConnectionsPerHost) { + // reuse the same configuration as the client, however for the admin client, the connection + // is usually established to a cluster address and not to a broker address + this.conf.setConnectionsPerBroker(maxConnectionsPerHost); + return this; + } + + @Override + public PulsarAdminBuilder connectionMaxIdleSeconds(int connectionMaxIdleSeconds) { + this.conf.setConnectionMaxIdleSeconds(connectionMaxIdleSeconds); + return this; + } } diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java index 7a68b0ade70e2..1d40f4b6b25f8 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/PulsarAdminImpl.java @@ -168,13 +168,13 @@ public PulsarAdminImpl(String serviceUrl, ClientConfigurationData clientConfigDa this.nonPersistentTopics = new NonPersistentTopicsImpl(root, auth, requestTimeoutMs); this.resourceQuotas = new ResourceQuotasImpl(root, auth, requestTimeoutMs); this.lookups = new LookupImpl(root, auth, useTls, requestTimeoutMs, topics); - this.functions = new FunctionsImpl(root, auth, asyncHttpConnector.getHttpClient(), requestTimeoutMs); - this.sources = new SourcesImpl(root, auth, asyncHttpConnector.getHttpClient(), requestTimeoutMs); - this.sinks = new SinksImpl(root, auth, asyncHttpConnector.getHttpClient(), requestTimeoutMs); + this.functions = new FunctionsImpl(root, auth, asyncHttpConnector, requestTimeoutMs); + this.sources = new SourcesImpl(root, auth, asyncHttpConnector, requestTimeoutMs); + this.sinks = new SinksImpl(root, auth, asyncHttpConnector, requestTimeoutMs); this.worker = new WorkerImpl(root, auth, requestTimeoutMs); this.schemas = new SchemasImpl(root, auth, requestTimeoutMs); this.bookies = new BookiesImpl(root, auth, requestTimeoutMs); - this.packages = new PackagesImpl(root, auth, asyncHttpConnector.getHttpClient(), requestTimeoutMs); + this.packages = new PackagesImpl(root, auth, asyncHttpConnector, requestTimeoutMs); this.transactions = new TransactionsImpl(root, auth, requestTimeoutMs); if (originalCtxLoader != null) { diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SinksImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SinksImpl.java index a30f51264cc2e..bba0289d81254 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SinksImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SinksImpl.java @@ -34,13 +34,13 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.Sink; import org.apache.pulsar.client.admin.Sinks; +import org.apache.pulsar.client.admin.internal.http.AsyncHttpRequestExecutor; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.common.functions.UpdateOptions; import org.apache.pulsar.common.functions.UpdateOptionsImpl; import org.apache.pulsar.common.io.ConnectorDefinition; import org.apache.pulsar.common.io.SinkConfig; import org.apache.pulsar.common.policies.data.SinkStatus; -import org.asynchttpclient.AsyncHttpClient; import org.asynchttpclient.RequestBuilder; import org.asynchttpclient.request.body.multipart.FilePart; import org.asynchttpclient.request.body.multipart.StringPart; @@ -51,12 +51,13 @@ public class SinksImpl extends ComponentResource implements Sinks, Sink { private final WebTarget sink; - private final AsyncHttpClient asyncHttpClient; + private final AsyncHttpRequestExecutor asyncHttpRequestExecutor; - public SinksImpl(WebTarget web, Authentication auth, AsyncHttpClient asyncHttpClient, long requestTimeoutMs) { + public SinksImpl(WebTarget web, Authentication auth, AsyncHttpRequestExecutor asyncHttpRequestExecutor, + long requestTimeoutMs) { super(auth, requestTimeoutMs); this.sink = web.path("/admin/v3/sink"); - this.asyncHttpClient = asyncHttpClient; + this.asyncHttpRequestExecutor = asyncHttpRequestExecutor; } @Override @@ -145,7 +146,7 @@ public CompletableFuture createSinkAsync(SinkConfig sinkConfig, String fil // If the function code is built in, we don't need to submit here builder.addBodyPart(new FilePart("data", new File(fileName), MediaType.APPLICATION_OCTET_STREAM)); } - asyncHttpClient.executeRequest(addAuthHeaders(sink, builder).build()) + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(sink, builder).build()) .toCompletableFuture() .thenAccept(response -> { if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { @@ -233,7 +234,7 @@ public CompletableFuture updateSinkAsync( // If the function code is built in, we don't need to submit here builder.addBodyPart(new FilePart("data", new File(fileName), MediaType.APPLICATION_OCTET_STREAM)); } - asyncHttpClient.executeRequest(addAuthHeaders(sink, builder).build()) + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(sink, builder).build()) .toCompletableFuture() .thenAccept(response -> { if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SourcesImpl.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SourcesImpl.java index 8821ed61ce5b8..56cf7db229b78 100644 --- a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SourcesImpl.java +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/SourcesImpl.java @@ -33,13 +33,13 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.Source; import org.apache.pulsar.client.admin.Sources; +import org.apache.pulsar.client.admin.internal.http.AsyncHttpRequestExecutor; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.common.functions.UpdateOptions; import org.apache.pulsar.common.functions.UpdateOptionsImpl; import org.apache.pulsar.common.io.ConnectorDefinition; import org.apache.pulsar.common.io.SourceConfig; import org.apache.pulsar.common.policies.data.SourceStatus; -import org.asynchttpclient.AsyncHttpClient; import org.asynchttpclient.RequestBuilder; import org.asynchttpclient.request.body.multipart.FilePart; import org.asynchttpclient.request.body.multipart.StringPart; @@ -50,12 +50,13 @@ public class SourcesImpl extends ComponentResource implements Sources, Source { private final WebTarget source; - private final AsyncHttpClient asyncHttpClient; + private final AsyncHttpRequestExecutor asyncHttpRequestExecutor; - public SourcesImpl(WebTarget web, Authentication auth, AsyncHttpClient asyncHttpClient, long requestTimeoutMs) { + public SourcesImpl(WebTarget web, Authentication auth, AsyncHttpRequestExecutor asyncHttpRequestExecutor, + long requestTimeoutMs) { super(auth, requestTimeoutMs); this.source = web.path("/admin/v3/source"); - this.asyncHttpClient = asyncHttpClient; + this.asyncHttpRequestExecutor = asyncHttpRequestExecutor; } @Override @@ -124,7 +125,7 @@ public CompletableFuture createSourceAsync(SourceConfig sourceConfig, Stri // If the function code is built in, we don't need to submit here builder.addBodyPart(new FilePart("data", new File(fileName), MediaType.APPLICATION_OCTET_STREAM)); } - asyncHttpClient.executeRequest(addAuthHeaders(source, builder).build()) + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(source, builder).build()) .toCompletableFuture() .thenAccept(response -> { if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { @@ -202,7 +203,7 @@ public CompletableFuture updateSourceAsync( // If the function code is built in, we don't need to submit here builder.addBodyPart(new FilePart("data", new File(fileName), MediaType.APPLICATION_OCTET_STREAM)); } - asyncHttpClient.executeRequest(addAuthHeaders(source, builder).build()) + asyncHttpRequestExecutor.executeRequest(addAuthHeaders(source, builder).build()) .toCompletableFuture() .thenAccept(response -> { if (response.getStatusCode() < 200 || response.getStatusCode() >= 300) { 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 0bb8606cfd0cc..bc266962027e9 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 @@ -18,6 +18,17 @@ */ package org.apache.pulsar.client.admin.internal.http; +import static org.asynchttpclient.util.HttpConstants.Methods.GET; +import static org.asynchttpclient.util.HttpConstants.Methods.HEAD; +import static org.asynchttpclient.util.HttpConstants.Methods.OPTIONS; +import static org.asynchttpclient.util.HttpConstants.ResponseStatusCodes.FOUND_302; +import static org.asynchttpclient.util.HttpConstants.ResponseStatusCodes.MOVED_PERMANENTLY_301; +import static org.asynchttpclient.util.HttpConstants.ResponseStatusCodes.PERMANENT_REDIRECT_308; +import static org.asynchttpclient.util.HttpConstants.ResponseStatusCodes.SEE_OTHER_303; +import static org.asynchttpclient.util.HttpConstants.ResponseStatusCodes.TEMPORARY_REDIRECT_307; +import static org.asynchttpclient.util.MiscUtils.isNonEmpty; +import com.spotify.futures.ConcurrencyReducer; +import io.netty.handler.codec.http.DefaultHttpHeaders; import io.netty.handler.codec.http.HttpRequest; import io.netty.handler.codec.http.HttpResponse; import io.netty.handler.ssl.SslContext; @@ -27,9 +38,12 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.net.URI; +import java.security.GeneralSecurityException; import java.time.Duration; +import java.util.Map; import java.util.concurrent.CancellationException; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.Future; @@ -37,32 +51,39 @@ import java.util.concurrent.TimeoutException; import java.util.function.Supplier; import javax.net.ssl.SSLContext; +import javax.ws.rs.ProcessingException; import javax.ws.rs.client.Client; import javax.ws.rs.core.HttpHeaders; import javax.ws.rs.core.Response.Status; import lombok.Getter; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; -import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.Validate; import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.client.admin.internal.PulsarAdminImpl; import org.apache.pulsar.client.api.AuthenticationDataProvider; import org.apache.pulsar.client.api.KeyStoreParams; import org.apache.pulsar.client.impl.PulsarServiceNameResolver; +import org.apache.pulsar.client.impl.ServiceNameResolver; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.apache.pulsar.client.util.WithSNISslEngineFactory; import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.SecurityUtility; import org.apache.pulsar.common.util.keystoretls.KeyStoreSSLContext; +import org.asynchttpclient.AsyncCompletionHandlerBase; +import org.asynchttpclient.AsyncHandler; import org.asynchttpclient.AsyncHttpClient; +import org.asynchttpclient.AsyncHttpClientConfig; import org.asynchttpclient.BoundRequestBuilder; import org.asynchttpclient.DefaultAsyncHttpClient; import org.asynchttpclient.DefaultAsyncHttpClientConfig; import org.asynchttpclient.ListenableFuture; import org.asynchttpclient.Request; import org.asynchttpclient.Response; +import org.asynchttpclient.SslEngineFactory; import org.asynchttpclient.channel.DefaultKeepAliveStrategy; import org.asynchttpclient.netty.ssl.JsseSslEngineFactory; +import org.asynchttpclient.uri.Uri; import org.glassfish.jersey.client.ClientProperties; import org.glassfish.jersey.client.ClientRequest; import org.glassfish.jersey.client.ClientResponse; @@ -73,16 +94,18 @@ * Customized Jersey client connector with multi-host support. */ @Slf4j -public class AsyncHttpConnector implements Connector { +public class AsyncHttpConnector implements Connector, AsyncHttpRequestExecutor { private static final TimeoutException REQUEST_TIMEOUT_EXCEPTION = FutureUtil.createTimeoutException("Request timeout", AsyncHttpConnector.class, "retryOrTimeout(...)"); + private static final int DEFAULT_MAX_QUEUE_SIZE_PER_HOST = 10000; @Getter private final AsyncHttpClient httpClient; private final Duration requestTimeout; private final int maxRetries; - private final PulsarServiceNameResolver serviceNameResolver; + private final ServiceNameResolver serviceNameResolver; private final ScheduledExecutorService delayer = Executors.newScheduledThreadPool(1, new DefaultThreadFactory("delayer")); + private final Map> concurrencyReducers = new ConcurrentHashMap<>(); public AsyncHttpConnector(Client client, ClientConfigurationData conf, int autoCertRefreshTimeSeconds) { this((int) client.getConfiguration().getProperty(ClientProperties.CONNECT_TIMEOUT), @@ -96,9 +119,46 @@ public AsyncHttpConnector(Client client, ClientConfigurationData conf, int autoC public AsyncHttpConnector(int connectTimeoutMs, int readTimeoutMs, int requestTimeoutMs, int autoCertRefreshTimeSeconds, ClientConfigurationData conf) { + Validate.notEmpty(conf.getServiceUrl(), "Service URL is not provided"); + serviceNameResolver = new PulsarServiceNameResolver(); + String serviceUrl = conf.getServiceUrl(); + serviceNameResolver.updateServiceUrl(serviceUrl); + AsyncHttpClientConfig asyncHttpClientConfig = + createAsyncHttpClientConfig(conf, connectTimeoutMs, readTimeoutMs, requestTimeoutMs, + autoCertRefreshTimeSeconds); + httpClient = createAsyncHttpClient(asyncHttpClientConfig); + this.requestTimeout = requestTimeoutMs > 0 ? Duration.ofMillis(requestTimeoutMs) : null; + this.maxRetries = httpClient.getConfig().getMaxRequestRetry(); + } + + private AsyncHttpClientConfig createAsyncHttpClientConfig(ClientConfigurationData conf, int connectTimeoutMs, + int readTimeoutMs, + int requestTimeoutMs, int autoCertRefreshTimeSeconds) + throws GeneralSecurityException, IOException { DefaultAsyncHttpClientConfig.Builder confBuilder = new DefaultAsyncHttpClientConfig.Builder(); + configureAsyncHttpClientConfig(conf, connectTimeoutMs, readTimeoutMs, requestTimeoutMs, confBuilder); + if (conf.getServiceUrl().startsWith("https://")) { + configureAsyncHttpClientSslEngineFactory(conf, autoCertRefreshTimeSeconds, confBuilder); + } + AsyncHttpClientConfig asyncHttpClientConfig = confBuilder.build(); + return asyncHttpClientConfig; + } + + private void configureAsyncHttpClientConfig(ClientConfigurationData conf, int connectTimeoutMs, int readTimeoutMs, + int requestTimeoutMs, + DefaultAsyncHttpClientConfig.Builder confBuilder) { + if (conf.getConnectionsPerBroker() > 0) { + confBuilder.setMaxConnectionsPerHost(conf.getConnectionsPerBroker()); + // Use the request timeout value for acquireFreeChannelTimeout so that we don't need to add + // yet another configuration property. When the ConcurrencyReducer is in use, it shouldn't be necessary to + // wait for a free channel since the ConcurrencyReducer will queue the requests. + confBuilder.setAcquireFreeChannelTimeout(conf.getRequestTimeoutMs()); + } + if (conf.getConnectionMaxIdleSeconds() > 0) { + confBuilder.setPooledConnectionIdleTimeout(conf.getConnectionMaxIdleSeconds() * 1000); + } confBuilder.setUseProxyProperties(true); - confBuilder.setFollowRedirect(true); + confBuilder.setFollowRedirect(false); confBuilder.setRequestTimeout(conf.getRequestTimeoutMs()); confBuilder.setConnectTimeout(connectTimeoutMs); confBuilder.setReadTimeout(readTimeoutMs); @@ -114,75 +174,75 @@ public boolean keepAlive(InetSocketAddress remoteAddress, Request ahcRequest, && super.keepAlive(remoteAddress, ahcRequest, request, response); } }); + confBuilder.setDisableHttpsEndpointIdentificationAlgorithm(!conf.isTlsHostnameVerificationEnable()); + } - serviceNameResolver = new PulsarServiceNameResolver(); - if (conf != null && StringUtils.isNotBlank(conf.getServiceUrl())) { - serviceNameResolver.updateServiceUrl(conf.getServiceUrl()); - if (conf.getServiceUrl().startsWith("https://")) { - // Set client key and certificate if available - AuthenticationDataProvider authData = conf.getAuthentication().getAuthData(); - - if (conf.isUseKeyStoreTls()) { - KeyStoreParams params = authData.hasDataForTls() ? authData.getTlsKeyStoreParams() : - new KeyStoreParams(conf.getTlsKeyStoreType(), conf.getTlsKeyStorePath(), - conf.getTlsKeyStorePassword()); - - final SSLContext sslCtx = KeyStoreSSLContext.createClientSslContext( - conf.getSslProvider(), - params.getKeyStoreType(), - params.getKeyStorePath(), - params.getKeyStorePassword(), - conf.isTlsAllowInsecureConnection(), - conf.getTlsTrustStoreType(), - conf.getTlsTrustStorePath(), - conf.getTlsTrustStorePassword(), - conf.getTlsCiphers(), - conf.getTlsProtocols()); - - JsseSslEngineFactory sslEngineFactory = new JsseSslEngineFactory(sslCtx); - confBuilder.setSslEngineFactory(sslEngineFactory); - } else { - SslProvider sslProvider = null; - if (conf.getSslProvider() != null) { - sslProvider = SslProvider.valueOf(conf.getSslProvider()); - } - SslContext sslCtx = null; - if (authData.hasDataForTls()) { - sslCtx = authData.getTlsTrustStoreStream() == null - ? SecurityUtility.createAutoRefreshSslContextForClient( - sslProvider, - conf.isTlsAllowInsecureConnection(), - conf.getTlsTrustCertsFilePath(), authData.getTlsCerificateFilePath(), - authData.getTlsPrivateKeyFilePath(), null, autoCertRefreshTimeSeconds, delayer) - : SecurityUtility.createNettySslContextForClient( - sslProvider, - conf.isTlsAllowInsecureConnection(), - authData.getTlsTrustStoreStream(), authData.getTlsCertificates(), - authData.getTlsPrivateKey(), - conf.getTlsCiphers(), - conf.getTlsProtocols()); - } else { - sslCtx = SecurityUtility.createNettySslContextForClient( - sslProvider, - conf.isTlsAllowInsecureConnection(), - conf.getTlsTrustCertsFilePath(), - conf.getTlsCertificateFilePath(), - conf.getTlsKeyFilePath(), - conf.getTlsCiphers(), - conf.getTlsProtocols()); - } - confBuilder.setSslContext(sslCtx); - if (!conf.isTlsHostnameVerificationEnable()) { - confBuilder.setSslEngineFactory(new WithSNISslEngineFactory(serviceNameResolver - .resolveHostUri().getHost())); - } - } + protected AsyncHttpClient createAsyncHttpClient(AsyncHttpClientConfig asyncHttpClientConfig) { + return new DefaultAsyncHttpClient(asyncHttpClientConfig); + } + + private void configureAsyncHttpClientSslEngineFactory(ClientConfigurationData conf, int autoCertRefreshTimeSeconds, + DefaultAsyncHttpClientConfig.Builder confBuilder) + throws GeneralSecurityException, IOException { + // Set client key and certificate if available + AuthenticationDataProvider authData = conf.getAuthentication().getAuthData(); + + SslEngineFactory sslEngineFactory = null; + if (conf.isUseKeyStoreTls()) { + KeyStoreParams params = authData.hasDataForTls() ? authData.getTlsKeyStoreParams() : + new KeyStoreParams(conf.getTlsKeyStoreType(), conf.getTlsKeyStorePath(), + conf.getTlsKeyStorePassword()); + + final SSLContext sslCtx = KeyStoreSSLContext.createClientSslContext( + conf.getSslProvider(), + params.getKeyStoreType(), + params.getKeyStorePath(), + params.getKeyStorePassword(), + conf.isTlsAllowInsecureConnection(), + conf.getTlsTrustStoreType(), + conf.getTlsTrustStorePath(), + conf.getTlsTrustStorePassword(), + conf.getTlsCiphers(), + conf.getTlsProtocols()); + + sslEngineFactory = new JsseSslEngineFactory(sslCtx); + confBuilder.setSslEngineFactory(sslEngineFactory); + } else { + SslProvider sslProvider = null; + if (conf.getSslProvider() != null) { + sslProvider = SslProvider.valueOf(conf.getSslProvider()); + } + SslContext sslCtx = null; + if (authData.hasDataForTls()) { + sslCtx = authData.getTlsTrustStoreStream() == null + ? SecurityUtility.createAutoRefreshSslContextForClient( + sslProvider, + conf.isTlsAllowInsecureConnection(), + conf.getTlsTrustCertsFilePath(), authData.getTlsCerificateFilePath(), + authData.getTlsPrivateKeyFilePath(), null, autoCertRefreshTimeSeconds, delayer) + : SecurityUtility.createNettySslContextForClient( + sslProvider, + conf.isTlsAllowInsecureConnection(), + authData.getTlsTrustStoreStream(), authData.getTlsCertificates(), + authData.getTlsPrivateKey(), + conf.getTlsCiphers(), + conf.getTlsProtocols()); + } else { + sslCtx = SecurityUtility.createNettySslContextForClient( + sslProvider, + conf.isTlsAllowInsecureConnection(), + conf.getTlsTrustCertsFilePath(), + conf.getTlsCertificateFilePath(), + conf.getTlsKeyFilePath(), + conf.getTlsCiphers(), + conf.getTlsProtocols()); + } + confBuilder.setSslContext(sslCtx); + if (!conf.isTlsHostnameVerificationEnable()) { + confBuilder.setSslEngineFactory(new WithSNISslEngineFactory(serviceNameResolver + .resolveHostUri().getHost())); } - confBuilder.setDisableHttpsEndpointIdentificationAlgorithm(!conf.isTlsHostnameVerificationEnable()); } - httpClient = new DefaultAsyncHttpClient(confBuilder.build()); - this.requestTimeout = requestTimeoutMs > 0 ? Duration.ofMillis(requestTimeoutMs) : null; - this.maxRetries = httpClient.getConfig().getMaxRequestRetry(); } @Override @@ -202,9 +262,8 @@ public void failure(Throwable failure) { try { return future.get(); } catch (InterruptedException | ExecutionException e) { - log.error(e.getMessage()); + throw new ProcessingException(e.getCause()); } - return null; } private URI replaceWithNew(InetSocketAddress address, URI uri) { @@ -266,6 +325,8 @@ private CompletableFuture retryOrTimeOut(ClientRequest request) { return resultFuture; } + // TODO: There are problems with this solution since AsyncHttpClient already contains logic to retry requests. + // This solution doesn't contain backoff handling. private void retryOperation( final CompletableFuture resultFuture, final Supplier> operation, @@ -277,9 +338,13 @@ private void retryOperation( operationFuture.whenComplete( (t, throwable) -> { if (throwable != null) { + throwable = FutureUtil.unwrapCompletionException(throwable); if (throwable instanceof CancellationException) { resultFuture.completeExceptionally( new RetryException("Operation future was cancelled.", throwable)); + } else if (throwable instanceof MaxRedirectException) { + // don't retry on max redirect + resultFuture.completeExceptionally(throwable); } else { if (retries > 0) { if (log.isDebugEnabled()) { @@ -319,7 +384,129 @@ public RetryException(String message, Throwable cause) { } } + public static class MaxRedirectException extends Exception { + public MaxRedirectException(String msg) { + super(msg, null, true, false); + } + } + protected CompletableFuture oneShot(InetSocketAddress host, ClientRequest request) { + Request preparedRequest; + try { + preparedRequest = prepareRequest(host, request); + } catch (IOException e) { + return FutureUtil.failedFuture(e); + } + return executeRequest(preparedRequest); + } + + public CompletableFuture executeRequest(Request request) { + return executeRequest(request, () -> new AsyncCompletionHandlerBase()); + } + + public CompletableFuture executeRequest(Request request, + Supplier> handlerSupplier) { + return executeRequest(request, handlerSupplier, 0); + } + + private CompletableFuture executeRequest(Request request, + Supplier> handlerSupplier, + int redirectCount) { + int maxRedirects = httpClient.getConfig().getMaxRedirects(); + if (redirectCount > maxRedirects) { + return FutureUtil.failedFuture( + new MaxRedirectException("Maximum redirect reached: " + maxRedirects + " uri:" + request.getUri())); + } + CompletableFuture responseFuture; + if (httpClient.getConfig().getMaxConnectionsPerHost() > 0) { + String hostAndPort = request.getUri().getHost() + ":" + request.getUri().getPort(); + ConcurrencyReducer responseConcurrencyReducer = concurrencyReducers.computeIfAbsent(hostAndPort, + h -> ConcurrencyReducer.create(httpClient.getConfig().getMaxConnectionsPerHost(), + DEFAULT_MAX_QUEUE_SIZE_PER_HOST)); + responseFuture = responseConcurrencyReducer.add(() -> doExecuteRequest(request, handlerSupplier)); + } else { + responseFuture = doExecuteRequest(request, handlerSupplier); + } + CompletableFuture futureWithRedirect = responseFuture.thenCompose(response -> { + if (isRedirectStatusCode(response.getStatusCode())) { + return executeRedirect(request, response, handlerSupplier, redirectCount); + } + return CompletableFuture.completedFuture(response); + }); + futureWithRedirect.whenComplete((response, throwable) -> { + // propagate cancellation or timeout to the original response future + responseFuture.cancel(false); + }); + return futureWithRedirect; + } + + private CompletableFuture executeRedirect(Request request, Response response, + Supplier> handlerSupplier, + int redirectCount) { + String originalMethod = request.getMethod(); + int statusCode = response.getStatusCode(); + boolean switchToGet = !originalMethod.equals(GET) + && !originalMethod.equals(OPTIONS) && !originalMethod.equals(HEAD) && ( + statusCode == MOVED_PERMANENTLY_301 || statusCode == SEE_OTHER_303 || statusCode == FOUND_302); + boolean keepBody = statusCode == TEMPORARY_REDIRECT_307 || statusCode == PERMANENT_REDIRECT_308; + String location = response.getHeader(HttpHeaders.LOCATION); + Uri newUri = Uri.create(request.getUri(), location); + BoundRequestBuilder builder = httpClient.prepareRequest(request); + if (switchToGet) { + builder.setMethod(GET); + } + builder.setUri(newUri); + if (keepBody) { + builder.setCharset(request.getCharset()); + if (isNonEmpty(request.getFormParams())) { + builder.setFormParams(request.getFormParams()); + } else if (request.getStringData() != null) { + builder.setBody(request.getStringData()); + } else if (request.getByteData() != null){ + builder.setBody(request.getByteData()); + } else if (request.getByteBufferData() != null) { + builder.setBody(request.getByteBufferData()); + } else if (request.getBodyGenerator() != null) { + builder.setBody(request.getBodyGenerator()); + } else if (isNonEmpty(request.getBodyParts())) { + builder.setBodyParts(request.getBodyParts()); + } + } else { + builder.resetFormParams(); + builder.resetNonMultipartData(); + builder.resetMultipartData(); + io.netty.handler.codec.http.HttpHeaders headers = new DefaultHttpHeaders(); + headers.add(request.getHeaders()); + headers.remove(HttpHeaders.CONTENT_LENGTH); + headers.remove(HttpHeaders.CONTENT_TYPE); + headers.remove(HttpHeaders.CONTENT_ENCODING); + builder.setHeaders(headers); + } + return executeRequest(builder.build(), handlerSupplier, redirectCount + 1); + } + + private static boolean isRedirectStatusCode(int statusCode) { + return statusCode == MOVED_PERMANENTLY_301 || statusCode == FOUND_302 || statusCode == SEE_OTHER_303 + || statusCode == TEMPORARY_REDIRECT_307 || statusCode == PERMANENT_REDIRECT_308; + } + + private CompletableFuture doExecuteRequest(Request request, + Supplier> handlerSupplier) { + ListenableFuture responseFuture = + httpClient.executeRequest(request, handlerSupplier.get()); + CompletableFuture completableFuture = responseFuture.toCompletableFuture(); + completableFuture.whenComplete((response, throwable) -> { + throwable = FutureUtil.unwrapCompletionException(throwable); + if (throwable != null && (throwable instanceof CancellationException + || throwable instanceof TimeoutException)) { + // abort the request if the future is cancelled or timed out + responseFuture.abort(throwable); + } + }); + return completableFuture; + } + + private Request prepareRequest(InetSocketAddress host, ClientRequest request) throws IOException { ClientRequest currentRequest = new ClientRequest(request); URI newUri = replaceWithNew(host, currentRequest.getUri()); currentRequest.setUri(newUri); @@ -330,14 +517,7 @@ protected CompletableFuture oneShot(InetSocketAddress host, ClientRequ if (currentRequest.hasEntity()) { ByteArrayOutputStream outStream = new ByteArrayOutputStream(); currentRequest.setStreamProvider(contentLength -> outStream); - try { - currentRequest.writeEntity(); - } catch (IOException e) { - CompletableFuture r = new CompletableFuture<>(); - r.completeExceptionally(e); - return r; - } - + currentRequest.writeEntity(); builder.setBody(outStream.toByteArray()); } @@ -347,16 +527,7 @@ protected CompletableFuture oneShot(InetSocketAddress host, ClientRequ } }); - ListenableFuture responseFuture = builder.execute(); - CompletableFuture completableFuture = responseFuture.toCompletableFuture(); - completableFuture.whenComplete((response, throwable) -> { - if (throwable != null && (throwable instanceof CancellationException - || throwable instanceof TimeoutException)) { - // abort the request if the future is cancelled or timed out - responseFuture.abort(throwable); - } - }); - return completableFuture; + return builder.build(); } @Override diff --git a/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpRequestExecutor.java b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpRequestExecutor.java new file mode 100644 index 0000000000000..d3c7a653b36b4 --- /dev/null +++ b/pulsar-client-admin/src/main/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpRequestExecutor.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.client.admin.internal.http; + +import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; +import org.asynchttpclient.AsyncHandler; +import org.asynchttpclient.Request; +import org.asynchttpclient.Response; + +/** + * Interface for executing HTTP requests asynchronously. + * This is used internally in the Pulsar Admin client for executing HTTP requests that by-pass the Jersey client + * and use the AsyncHttpClient API directly. + */ +public interface AsyncHttpRequestExecutor { + /** + * Execute the given HTTP request asynchronously. + * + * @param request the HTTP request to execute + * @return a future that will be completed with the HTTP response + */ + CompletableFuture executeRequest(Request request); + /** + * Execute the given HTTP request asynchronously. + * + * @param request the HTTP request to execute + * @param handlerSupplier a supplier for the async handler to use for the request + * @return a future that will be completed with the HTTP response + */ + CompletableFuture executeRequest(Request request, Supplier> handlerSupplier); +} diff --git a/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImplTest.java b/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImplTest.java index d3621e729973b..615576d488f94 100644 --- a/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImplTest.java +++ b/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/PulsarAdminBuilderImplTest.java @@ -65,6 +65,7 @@ public void testGetPropertiesFromConf() throws Exception { config.put("autoCertRefreshSeconds", 20); config.put("connectionTimeoutMs", 30); config.put("readTimeoutMs", 40); + config.put("maxConnectionsPerHost", 50); PulsarAdminBuilder adminBuilder = PulsarAdmin.builder().loadConf(config); PulsarAdminImpl admin = (PulsarAdminImpl) adminBuilder.build(); ClientConfigurationData clientConfigData = admin.getClientConfigData(); @@ -72,6 +73,7 @@ public void testGetPropertiesFromConf() throws Exception { Assert.assertEquals(clientConfigData.getAutoCertRefreshSeconds(), 20); Assert.assertEquals(clientConfigData.getConnectionTimeoutMs(), 30); Assert.assertEquals(clientConfigData.getReadTimeoutMs(), 40); + Assert.assertEquals(clientConfigData.getConnectionsPerBroker(), 50); } @Test diff --git a/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorTest.java b/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorTest.java index 5523e02d6e57a..44e97386779fb 100644 --- a/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorTest.java +++ b/pulsar-client-admin/src/test/java/org/apache/pulsar/client/admin/internal/http/AsyncHttpConnectorTest.java @@ -20,23 +20,34 @@ import static com.github.tomakehurst.wiremock.client.WireMock.aResponse; import static com.github.tomakehurst.wiremock.client.WireMock.get; +import static com.github.tomakehurst.wiremock.client.WireMock.post; import static com.github.tomakehurst.wiremock.client.WireMock.urlEqualTo; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; import com.github.tomakehurst.wiremock.WireMockServer; +import com.github.tomakehurst.wiremock.common.FileSource; import com.github.tomakehurst.wiremock.core.WireMockConfiguration; +import com.github.tomakehurst.wiremock.extension.Parameters; +import com.github.tomakehurst.wiremock.extension.ResponseTransformer; import com.github.tomakehurst.wiremock.stubbing.Scenario; import java.io.IOException; import java.net.InetSocketAddress; import java.net.URI; +import java.util.ArrayList; +import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import lombok.Cleanup; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; +import org.apache.pulsar.common.util.FutureUtil; +import org.asynchttpclient.Request; +import org.asynchttpclient.RequestBuilder; import org.asynchttpclient.Response; import org.glassfish.jersey.client.ClientConfig; import org.glassfish.jersey.client.ClientRequest; @@ -52,10 +63,74 @@ public class AsyncHttpConnectorTest { WireMockServer server; + ConcurrencyTestTransformer concurrencyTestTransformer = new ConcurrencyTestTransformer(); + + private static class CopyRequestBodyToResponseBodyTransformer extends ResponseTransformer { + @Override + public com.github.tomakehurst.wiremock.http.Response transform( + com.github.tomakehurst.wiremock.http.Request request, + com.github.tomakehurst.wiremock.http.Response response, FileSource fileSource, Parameters parameters) { + return com.github.tomakehurst.wiremock.http.Response.Builder.like(response) + .body(request.getBodyAsString()) + .build(); + } + + @Override + public String getName() { + return "copy-body"; + } + + @Override + public boolean applyGlobally() { + return false; + } + } + + private static class ConcurrencyTestTransformer extends ResponseTransformer { + private static final long DELAY_MS = 100; + private final AtomicInteger concurrencyCounter = new AtomicInteger(0); + private final AtomicInteger maxConcurrency = new AtomicInteger(0); + + @Override + public com.github.tomakehurst.wiremock.http.Response transform( + com.github.tomakehurst.wiremock.http.Request request, + com.github.tomakehurst.wiremock.http.Response response, FileSource fileSource, Parameters parameters) { + int currentCounter = concurrencyCounter.incrementAndGet(); + maxConcurrency.updateAndGet(v -> Math.max(v, currentCounter)); + try { + try { + Thread.sleep(DELAY_MS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + return com.github.tomakehurst.wiremock.http.Response.Builder.like(response) + .body(String.valueOf(currentCounter)) + .build(); + } finally { + concurrencyCounter.decrementAndGet(); + } + } + + public int getMaxConcurrency() { + return maxConcurrency.get(); + } + + @Override + public String getName() { + return "concurrency-test"; + } + + @Override + public boolean applyGlobally() { + return false; + } + } @BeforeClass(alwaysRun = true) void beforeClass() throws IOException { server = new WireMockServer(WireMockConfiguration.wireMockConfig() + .extensions(new CopyRequestBodyToResponseBodyTransformer(), concurrencyTestTransformer) + .containerThreads(100) .port(0)); server.start(); } @@ -137,4 +212,129 @@ public void failure(Throwable failure) { assertEquals(scenarioState, "next"); assertTrue(future.isCompletedExceptionally()); } + + @Test + void testMaxRedirects() { + // Redirect to itself to test max redirects + server.stubFor(get(urlEqualTo("/admin/v2/clusters")) + .willReturn(aResponse() + .withStatus(301) + .withHeader("Location", "http://localhost:" + server.port() + "/admin/v2/clusters"))); + + ClientConfigurationData conf = new ClientConfigurationData(); + conf.setServiceUrl("http://localhost:" + server.port()); + + @Cleanup + AsyncHttpConnector connector = new AsyncHttpConnector(5000, 5000, + 5000, 0, conf); + + Request request = new RequestBuilder("GET") + .setUrl("http://localhost:" + server.port() + "/admin/v2/clusters") + .build(); + + try { + connector.executeRequest(request).get(); + fail(); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof AsyncHttpConnector.MaxRedirectException); + } catch (InterruptedException e) { + fail(); + } + } + + @Test + void testRelativeRedirect() throws ExecutionException, InterruptedException { + doTestRedirect("path2"); + } + + @Test + void testAbsoluteRedirect() throws ExecutionException, InterruptedException { + doTestRedirect("/path2"); + } + + @Test + void testUrlRedirect() throws ExecutionException, InterruptedException { + doTestRedirect("http://localhost:" + server.port() + "/path2"); + } + + private void doTestRedirect(String location) throws InterruptedException, ExecutionException { + server.stubFor(get(urlEqualTo("/path1")) + .willReturn(aResponse() + .withStatus(301) + .withHeader("Location", location))); + + server.stubFor(get(urlEqualTo("/path2")) + .willReturn(aResponse() + .withBody("OK"))); + + ClientConfigurationData conf = new ClientConfigurationData(); + conf.setServiceUrl("http://localhost:" + server.port()); + + @Cleanup + AsyncHttpConnector connector = new AsyncHttpConnector(5000, 5000, + 5000, 0, conf); + + Request request = new RequestBuilder("GET") + .setUrl("http://localhost:" + server.port() + "/path1") + .build(); + + Response response = connector.executeRequest(request).get(); + assertEquals(response.getResponseBody(), "OK"); + } + + @Test + void testRedirectWithBody() throws ExecutionException, InterruptedException { + server.stubFor(post(urlEqualTo("/path1")) + .willReturn(aResponse() + .withStatus(307) + .withHeader("Location", "/path2"))); + + server.stubFor(post(urlEqualTo("/path2")) + .willReturn(aResponse() + .withTransformers("copy-body"))); + + ClientConfigurationData conf = new ClientConfigurationData(); + conf.setServiceUrl("http://localhost:" + server.port()); + + @Cleanup + AsyncHttpConnector connector = new AsyncHttpConnector(5000, 5000, + 5000, 0, conf); + + Request request = new RequestBuilder("POST") + .setUrl("http://localhost:" + server.port() + "/path1") + .setBody("Hello world!") + .build(); + + Response response = connector.executeRequest(request).get(); + assertEquals(response.getResponseBody(), "Hello world!"); + } + + @Test + void testMaxConnections() throws ExecutionException, InterruptedException { + server.stubFor(post(urlEqualTo("/concurrency-test")) + .willReturn(aResponse() + .withTransformers("concurrency-test"))); + + ClientConfigurationData conf = new ClientConfigurationData(); + int maxConnections = 10; + conf.setConnectionsPerBroker(maxConnections); + conf.setServiceUrl("http://localhost:" + server.port()); + + @Cleanup + AsyncHttpConnector connector = new AsyncHttpConnector(5000, 5000, + 5000, 0, conf); + + Request request = new RequestBuilder("POST") + .setUrl("http://localhost:" + server.port() + "/concurrency-test") + .build(); + + List> futures = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + futures.add(connector.executeRequest(request)); + } + FutureUtil.waitForAll(futures).get(); + int maxConcurrency = concurrencyTestTransformer.getMaxConcurrency(); + assertTrue(maxConcurrency > maxConnections / 2 && maxConcurrency <= maxConnections, + "concurrency didn't get limited as expected (max: " + maxConcurrency + ")"); + } } \ No newline at end of file diff --git a/pulsar-client-all/pom.xml b/pulsar-client-all/pom.xml index 284e6e324e64d..579a00b215561 100644 --- a/pulsar-client-all/pom.xml +++ b/pulsar-client-all/pom.xml @@ -167,6 +167,7 @@ com.google.errorprone:* com.google.j2objc:* com.google.code.gson:gson + com.spotify:completable-futures com.fasterxml.jackson.*:* io.netty:netty io.netty:netty-all @@ -243,6 +244,10 @@ com.google.protobuf.* + + com.spotify.futures + org.apache.pulsar.shade.com.spotify.futures + com.fasterxml.jackson org.apache.pulsar.shade.com.fasterxml.jackson 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 8b959690a0363..7bf1a03d3f455 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 @@ -128,6 +128,8 @@ public interface ClientBuilder extends Serializable, Cloneable { /** * Release the connection if it is not used for more than {@param connectionMaxIdleSeconds} seconds. + * Defaults to 25 seconds. + * * @return the client builder instance */ ClientBuilder connectionMaxIdleSeconds(int connectionMaxIdleSeconds); diff --git a/pulsar-client-shaded/pom.xml b/pulsar-client-shaded/pom.xml index 5352d4dddca4a..d1f9754628738 100644 --- a/pulsar-client-shaded/pom.xml +++ b/pulsar-client-shaded/pom.xml @@ -145,6 +145,7 @@ com.google.errorprone:* com.google.j2objc:* com.google.code.gson:gson + com.spotify:completable-futures com.fasterxml.jackson.*:* io.netty:* io.netty.incubator:* @@ -204,6 +205,10 @@ com.google.protobuf.* + + com.spotify.futures + org.apache.pulsar.shade.com.spotify.futures + com.fasterxml.jackson org.apache.pulsar.shade.com.fasterxml.jackson 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 ef3a9249c820a..9c73684b356a5 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 @@ -61,7 +61,7 @@ public class ConnectionPool implements AutoCloseable { - public static final int IDLE_DETECTION_INTERVAL_SECONDS_MIN = 60; + public static final int IDLE_DETECTION_INTERVAL_SECONDS_MIN = 15; protected final ConcurrentHashMap>> pool; 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 7d94675ccba7d..a5157618ff2a5 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 @@ -19,6 +19,7 @@ package org.apache.pulsar.client.impl.conf; import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import io.swagger.annotations.ApiModelProperty; import java.io.Serializable; import java.net.InetSocketAddress; @@ -45,6 +46,7 @@ @Data @NoArgsConstructor @AllArgsConstructor +@JsonIgnoreProperties(ignoreUnknown = true) public class ClientConfigurationData implements Serializable, Cloneable { private static final long serialVersionUID = 1L; @@ -130,7 +132,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 = 180; + private int connectionMaxIdleSeconds = 25; @ApiModelProperty( name = "useTcpNoDelay", diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientBuilderImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientBuilderImplTest.java index 9a39c906b8ff6..86837a3ebc402 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientBuilderImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ClientBuilderImplTest.java @@ -109,7 +109,7 @@ public void testConnectionMaxIdleSeconds() throws Exception { PulsarClient.builder().connectionMaxIdleSeconds(60); // test config not correct. try { - PulsarClient.builder().connectionMaxIdleSeconds(30); + PulsarClient.builder().connectionMaxIdleSeconds(14); fail(); } catch (IllegalArgumentException e){ } diff --git a/pulsar-common/pom.xml b/pulsar-common/pom.xml index f9108b833c7fd..af1d4240a0a3c 100644 --- a/pulsar-common/pom.xml +++ b/pulsar-common/pom.xml @@ -206,6 +206,11 @@ protobuf-java + + com.spotify + completable-futures + + org.bouncycastle diff --git a/pulsar-sql/presto-distribution/LICENSE b/pulsar-sql/presto-distribution/LICENSE index 4178f50b3680f..dcf97e5541306 100644 --- a/pulsar-sql/presto-distribution/LICENSE +++ b/pulsar-sql/presto-distribution/LICENSE @@ -479,6 +479,8 @@ The Apache Software License, Version 2.0 - hppc-0.9.1.jar * RoaringBitmap - RoaringBitmap-1.2.0.jar + * Spotify completable-futures + - completable-futures-0.3.6.jar Protocol Buffers License * Protocol Buffers @@ -535,15 +537,15 @@ CDDL-1.1 -- licenses/LICENSE-CDDL-1.1.txt - aopalliance-repackaged-2.6.1.jar * Jersey - jaxrs-213.jar - - jersey-client-2.41.jar - - jersey-common-2.41.jar - - jersey-container-servlet-2.41.jar - - jersey-container-servlet-core-2.41.jar - - jersey-entity-filtering-2.41.jar - - jersey-hk2-2.41.jar - - jersey-media-json-jackson-2.41.jar - - jersey-media-multipart-2.41.jar - - jersey-server-2.41.jar + - jersey-client-2.42.jar + - jersey-common-2.42.jar + - jersey-container-servlet-2.42.jar + - jersey-container-servlet-core-2.42.jar + - jersey-entity-filtering-2.42.jar + - jersey-hk2-2.42.jar + - jersey-media-json-jackson-2.42.jar + - jersey-media-multipart-2.42.jar + - jersey-server-2.42.jar * JAXB - jaxb-api-2.3.1.jar - jaxb-runtime-2.3.4.jar diff --git a/pulsar-sql/presto-distribution/pom.xml b/pulsar-sql/presto-distribution/pom.xml index f8f3420e49f6f..a86dee5446dad 100644 --- a/pulsar-sql/presto-distribution/pom.xml +++ b/pulsar-sql/presto-distribution/pom.xml @@ -33,7 +33,7 @@ false - 2.41 + 2.42 2.6 0.0.12 3.0.5 From 6cfacf2348ecfd67c7e59effdd421ab40c12383f Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Sat, 10 Aug 2024 15:55:55 +0800 Subject: [PATCH 661/699] [fix] [broker] Fix compatibility issues for PIP-344 (#23136) Co-authored-by: Lari Hotari (cherry picked from commit 702c0b3cfa251636099871f65f0ba1ac1a52069c) (cherry picked from commit 617c110ce8a1af367ea09e3b34477b6633e8b5e0) --- .../broker/namespace/NamespaceService.java | 17 +++- .../GetPartitionMetadataMultiBrokerTest.java | 88 +++++++++++++++++++ .../admin/GetPartitionMetadataTest.java | 74 ++++++++++++++-- .../broker/admin/TopicAutoCreationTest.java | 4 +- .../buffer/TransactionLowWaterMarkTest.java | 6 +- .../pulsar/client/impl/ClientCnxTest.java | 2 +- .../client/api/PulsarClientException.java | 25 ++++++ .../client/impl/BinaryProtoLookupService.java | 29 ++++-- .../client/impl/ConsumerBuilderImpl.java | 2 +- .../pulsar/client/impl/HttpLookupService.java | 6 +- .../pulsar/client/impl/LookupService.java | 19 +++- .../client/impl/MultiTopicsConsumerImpl.java | 2 +- .../pulsar/client/impl/PulsarClientImpl.java | 30 ++++--- .../TransactionCoordinatorClientImpl.java | 2 +- .../impl/MultiTopicsConsumerImplTest.java | 8 +- .../client/impl/PulsarClientImplTest.java | 2 +- pulsar-common/src/main/proto/PulsarApi.proto | 1 + 17 files changed, 273 insertions(+), 44 deletions(-) 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 00168b50e3e0d..d51bea94d92fc 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 @@ -23,6 +23,7 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.commons.lang3.StringUtils.isNotBlank; +import static org.apache.pulsar.client.api.PulsarClientException.FailedFeatureCheck.SupportsGetPartitionedMetadataWithoutAutoCreation; import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; import com.google.common.hash.Hashing; import io.prometheus.client.Counter; @@ -1441,8 +1442,22 @@ public CompletableFuture checkNonPersistentNonPartitionedTopicExists(St || actEx instanceof PulsarClientException.TopicDoesNotExistException || actEx instanceof PulsarAdminException.NotFoundException) { return CompletableFuture.completedFuture(false); + } else if (actEx instanceof PulsarClientException.FeatureNotSupportedException fe){ + if (fe.getFailedFeatureCheck() == SupportsGetPartitionedMetadataWithoutAutoCreation) { + // Since the feature PIP-344 isn't supported, restore the behavior to previous + // behavior before https://github.com/apache/pulsar/pull/22838 changes. + log.info("{} Checking the existence of a non-persistent non-partitioned topic " + + "was performed using the behavior prior to PIP-344 changes, " + + "because the broker does not support the PIP-344 feature " + + "'supports_get_partitioned_metadata_without_auto_creation'.", + topic); + return CompletableFuture.completedFuture(false); + } else { + log.error("{} Failed to get partition metadata", topic, ex); + return CompletableFuture.failedFuture(ex); + } } else { - log.error("{} Failed to get partition metadata due to redirecting fails", topic, ex); + log.error("{} Failed to get partition metadata", topic, ex); return CompletableFuture.failedFuture(ex); } }); 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 28cf91ee165e2..60691203e777d 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 @@ -18,20 +18,32 @@ */ package org.apache.pulsar.broker.admin; +import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; +import java.lang.reflect.Field; import java.net.URL; +import java.util.Arrays; import java.util.List; +import java.util.concurrent.CompletableFuture; import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.impl.ClientCnx; +import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.naming.TopicDomain; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.policies.data.TopicType; +import org.apache.pulsar.common.util.FutureUtil; +import org.awaitility.Awaitility; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Test(groups = "broker-admin") @@ -219,4 +231,80 @@ public void testGetMetadataIfNotAllowedCreateOfNonPersistentTopic(boolean config super.testGetMetadataIfNotAllowedCreateOfNonPersistentTopic(configAllowAutoTopicCreation, paramMetadataAutoCreationEnabled, isUsingHttpLookup); } + + @DataProvider(name = "autoCreationParamsAllForNonPersistentTopic") + public Object[][] autoCreationParamsAllForNonPersistentTopic(){ + return new Object[][]{ + // configAllowAutoTopicCreation, paramCreateIfAutoCreationEnabled, isUsingHttpLookup. + {true, true, true}, + {true, true, false}, + {true, false, true}, + {true, false, false}, + {false, true, true}, + {false, true, false}, + {false, false, true}, + {false, false, false} + }; + } + + @Test(dataProvider = "autoCreationParamsAllForNonPersistentTopic", priority = Integer.MAX_VALUE) + public void testCompatibilityDifferentBrokersForNonPersistentTopic(boolean configAllowAutoTopicCreation, + boolean paramMetadataAutoCreationEnabled, + boolean isUsingHttpLookup) throws Exception { + modifyTopicAutoCreation(configAllowAutoTopicCreation, TopicType.PARTITIONED, 3); + + // 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")); + + // Inject a not support flag into the connections initialized. + Field field = ClientCnx.class.getDeclaredField("supportsGetPartitionedMetadataWithoutAutoCreation"); + field.setAccessible(true); + for (PulsarClientImpl client : Arrays.asList(client1, client2)) { + ConnectionPool pool = client.getCnxPool(); + for (CompletableFuture connectionFuture : pool.getConnections()) { + ClientCnx clientCnx = connectionFuture.join(); + clientCnx.isSupportsGetPartitionedMetadataWithoutAutoCreation(); + 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); + for (PulsarClientImpl client : clientArray) { + final String topicNameStr = BrokerTestUtil.newUniqueName("non-persistent://" + DEFAULT_NS + "/tp"); + try { + PartitionedTopicMetadata topicMetadata = client + .getPartitionedTopicMetadata(topicNameStr, paramMetadataAutoCreationEnabled, false) + .join(); + log.info("Get topic metadata: {}", topicMetadata.partitions); + } catch (Exception ex) { + Throwable unwrapEx = FutureUtil.unwrapCompletionException(ex); + assertTrue(unwrapEx instanceof PulsarClientException.TopicDoesNotExistException + || unwrapEx instanceof PulsarClientException.NotFoundException); + assertFalse(ex.getMessage().contains("getting partitions without auto-creation is not supported from" + + " the broker")); + } + } + + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + assertEquals(getLookupRequestPermits(), lookupPermitsBefore); + }); + + // reset clients. + for (PulsarClientImpl client : Arrays.asList(client1, client2)) { + ConnectionPool pool = client.getCnxPool(); + for (CompletableFuture connectionFuture : pool.getConnections()) { + ClientCnx clientCnx = connectionFuture.join(); + clientCnx.isSupportsGetPartitionedMetadataWithoutAutoCreation(); + field.set(clientCnx, true); + } + } + } } 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 bf99b172829a7..87bc4267b48a3 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 @@ -23,10 +23,12 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; import com.google.common.collect.Sets; +import java.lang.reflect.Field; import java.net.URL; 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.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; @@ -34,6 +36,8 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.impl.ClientCnx; +import org.apache.pulsar.client.impl.ConnectionPool; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicDomain; @@ -225,6 +229,60 @@ public void testAutoCreatingMetadataWhenCallingOldAPI(TopicDomain topicDomain) t } } + @Test(dataProvider = "topicDomains", priority = Integer.MAX_VALUE) + public void testCompatibilityForNewClientAndOldBroker(TopicDomain topicDomain) throws Exception { + modifyTopicAutoCreation(true, TopicType.PARTITIONED, 3); + // Initialize connections. + String pulsarUrl = pulsar1.getBrokerServiceUrl(); + PulsarClientImpl[] clients = getClientsToTest(false); + for (PulsarClientImpl client : clients) { + client.getLookup(pulsarUrl).getBroker(TopicName.get(DEFAULT_NS + "/tp1")); + } + // Inject a not support flag into the connections initialized. + Field field = ClientCnx.class.getDeclaredField("supportsGetPartitionedMetadataWithoutAutoCreation"); + field.setAccessible(true); + for (PulsarClientImpl client : clients) { + ConnectionPool pool = client.getCnxPool(); + for (CompletableFuture connectionFuture : pool.getConnections()) { + ClientCnx clientCnx = connectionFuture.join(); + clientCnx.isSupportsGetPartitionedMetadataWithoutAutoCreation(); + field.set(clientCnx, false); + } + } + + // Verify: the method "getPartitionsForTopic(topic, false, true)" will fallback to + // "getPartitionsForTopic(topic)" behavior. + int lookupPermitsBefore = getLookupRequestPermits(); + for (PulsarClientImpl client : clients) { + // Verify: the behavior of topic creation. + final String tp = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); + client.getPartitionedTopicMetadata(tp, false, true).join(); + Optional metadata1 = pulsar1.getPulsarResources().getNamespaceResources() + .getPartitionedTopicResources() + .getPartitionedTopicMetadataAsync(TopicName.get(tp), true).join(); + assertTrue(metadata1.isPresent()); + assertEquals(metadata1.get().partitions, 3); + + // Verify: lookup semaphore has been releases. + Awaitility.await().untilAsserted(() -> { + assertEquals(getLookupRequestPermits(), lookupPermitsBefore); + }); + + // Cleanup. + admin1.topics().deletePartitionedTopic(tp, false); + } + + // reset clients. + for (PulsarClientImpl client : clients) { + ConnectionPool pool = client.getCnxPool(); + for (CompletableFuture connectionFuture : pool.getConnections()) { + ClientCnx clientCnx = connectionFuture.join(); + clientCnx.isSupportsGetPartitionedMetadataWithoutAutoCreation(); + field.set(clientCnx, true); + } + } + } + @DataProvider(name = "autoCreationParamsAll") public Object[][] autoCreationParamsAll(){ return new Object[][]{ @@ -265,7 +323,7 @@ public void testGetMetadataIfNonPartitionedTopicExists(boolean configAllowAutoTo for (PulsarClientImpl client : clientArray) { // Verify: the result of get partitioned topic metadata. PartitionedTopicMetadata response = - client.getPartitionedTopicMetadata(topicNameStr, paramMetadataAutoCreationEnabled).join(); + client.getPartitionedTopicMetadata(topicNameStr, paramMetadataAutoCreationEnabled, false).join(); assertEquals(response.partitions, 0); List partitionedTopics = admin1.topics().getPartitionedTopicList("public/default"); assertFalse(partitionedTopics.contains(topicNameStr)); @@ -298,7 +356,7 @@ public void testGetMetadataIfPartitionedTopicExists(boolean configAllowAutoTopic for (PulsarClientImpl client : clientArray) { // Verify: the result of get partitioned topic metadata. PartitionedTopicMetadata response = - client.getPartitionedTopicMetadata(topicNameStr, paramMetadataAutoCreationEnabled).join(); + client.getPartitionedTopicMetadata(topicNameStr, paramMetadataAutoCreationEnabled, false).join(); assertEquals(response.partitions, 3); verifyNonPartitionedTopicNeverCreated(topicNameStr); @@ -332,7 +390,7 @@ public void testAutoCreatePartitionedTopic(boolean isUsingHttpLookup, TopicDomai // Case-1: normal topic. final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); // Verify: the result of get partitioned topic metadata. - PartitionedTopicMetadata response = client.getPartitionedTopicMetadata(topicNameStr, true).join(); + PartitionedTopicMetadata response = client.getPartitionedTopicMetadata(topicNameStr, true, false).join(); assertEquals(response.partitions, 3); // Verify: the behavior of topic creation. List partitionedTopics = admin1.topics().getPartitionedTopicList("public/default"); @@ -347,7 +405,7 @@ public void testAutoCreatePartitionedTopic(boolean isUsingHttpLookup, TopicDomai topicDomain.value() + "://" + DEFAULT_NS + "/tp") + "-partition-1"; // Verify: the result of get partitioned topic metadata. PartitionedTopicMetadata response2 = - client.getPartitionedTopicMetadata(topicNameStrWithSuffix, true).join(); + client.getPartitionedTopicMetadata(topicNameStrWithSuffix, true, false).join(); assertEquals(response2.partitions, 0); // Verify: the behavior of topic creation. List partitionedTopics2 = @@ -380,7 +438,7 @@ public void testAutoCreateNonPartitionedTopic(boolean isUsingHttpLookup, TopicDo // Case 1: normal topic. final String topicNameStr = BrokerTestUtil.newUniqueName(topicDomain.value() + "://" + DEFAULT_NS + "/tp"); // Verify: the result of get partitioned topic metadata. - PartitionedTopicMetadata response = client.getPartitionedTopicMetadata(topicNameStr, true).join(); + PartitionedTopicMetadata response = client.getPartitionedTopicMetadata(topicNameStr, true, false).join(); assertEquals(response.partitions, 0); // Verify: the behavior of topic creation. List partitionedTopics = admin1.topics().getPartitionedTopicList("public/default"); @@ -392,7 +450,7 @@ public void testAutoCreateNonPartitionedTopic(boolean isUsingHttpLookup, TopicDo topicDomain.value() + "://" + DEFAULT_NS + "/tp") + "-partition-1"; // Verify: the result of get partitioned topic metadata. PartitionedTopicMetadata response2 = - client.getPartitionedTopicMetadata(topicNameStrWithSuffix, true).join(); + client.getPartitionedTopicMetadata(topicNameStrWithSuffix, true, false).join(); assertEquals(response2.partitions, 0); // Verify: the behavior of topic creation. List partitionedTopics2 = @@ -443,7 +501,7 @@ public void testGetMetadataIfNotAllowedCreate(boolean configAllowAutoTopicCreati final TopicName topicName = TopicName.get(topicNameStr); // Verify: the result of get partitioned topic metadata. try { - client.getPartitionedTopicMetadata(topicNameStr, paramMetadataAutoCreationEnabled) + client.getPartitionedTopicMetadata(topicNameStr, paramMetadataAutoCreationEnabled, false) .join(); fail("Expect a not found exception"); } catch (Exception e) { @@ -496,7 +554,7 @@ public void testGetMetadataIfNotAllowedCreateOfNonPersistentTopic(boolean config // Verify: the result of get partitioned topic metadata. try { PartitionedTopicMetadata topicMetadata = client - .getPartitionedTopicMetadata(topicNameStr, paramMetadataAutoCreationEnabled) + .getPartitionedTopicMetadata(topicNameStr, paramMetadataAutoCreationEnabled, false) .join(); log.info("Get topic metadata: {}", topicMetadata.partitions); fail("Expected a not found ex"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java index 4712682e71b57..45c7dbea2aba7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/admin/TopicAutoCreationTest.java @@ -136,7 +136,9 @@ public void testPartitionedTopicAutoCreationForbiddenDuringNamespaceDeletion() ((PulsarClientImpl) pulsarClient).setLookup(mockLookup); when(mockLookup.getPartitionedTopicMetadata(any(), anyBoolean())).thenAnswer( i -> CompletableFuture.completedFuture(new PartitionedTopicMetadata(0))); - when(mockLookup.getBroker(any())).thenAnswer(i -> { + when(mockLookup.getPartitionedTopicMetadata(any(), anyBoolean(), anyBoolean())).thenAnswer( + i -> CompletableFuture.completedFuture(new PartitionedTopicMetadata(0))); + when(mockLookup.getBroker(any())).thenAnswer(ignored -> { InetSocketAddress brokerAddress = new InetSocketAddress(pulsar.getAdvertisedAddress(), pulsar.getBrokerListenPort().get()); return CompletableFuture.completedFuture(Pair.of(brokerAddress, brokerAddress)); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionLowWaterMarkTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionLowWaterMarkTest.java index 3f268c4b7c973..bd2121948dcb7 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionLowWaterMarkTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/buffer/TransactionLowWaterMarkTest.java @@ -146,7 +146,8 @@ public void testTransactionBufferLowWaterMark() throws Exception { PartitionedTopicMetadata partitionedTopicMetadata = ((PulsarClientImpl) pulsarClient).getLookup() - .getPartitionedTopicMetadata(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, false).get(); + .getPartitionedTopicMetadata(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, false) + .get(); Transaction lowWaterMarkTxn = null; for (int i = 0; i < partitionedTopicMetadata.partitions; i++) { lowWaterMarkTxn = pulsarClient.newTransaction() @@ -251,7 +252,8 @@ public void testPendingAckLowWaterMark() throws Exception { PartitionedTopicMetadata partitionedTopicMetadata = ((PulsarClientImpl) pulsarClient).getLookup() - .getPartitionedTopicMetadata(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, false).get(); + .getPartitionedTopicMetadata(SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN, false) + .get(); Transaction lowWaterMarkTxn = null; for (int i = 0; i < partitionedTopicMetadata.partitions; i++) { lowWaterMarkTxn = pulsarClient.newTransaction() 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 1a9b4bbcb0d21..bdff97dbb3bfa 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 @@ -159,7 +159,7 @@ public void testSupportsGetPartitionedMetadataWithoutAutoCreation() throws Excep clientWitBinaryLookup.getPartitionsForTopic(topic, false).join(); Assert.fail("Expected an error that the broker version is too old."); } catch (Exception ex) { - Assert.assertTrue(ex.getMessage().contains("without auto-creation is not supported from the broker")); + Assert.assertTrue(ex.getMessage().contains("without auto-creation is not supported by the broker")); } // cleanup. 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 22a97571e532e..0e12b4f8e3211 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 @@ -22,6 +22,7 @@ import java.util.Collection; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; +import lombok.Getter; import org.apache.pulsar.common.classification.InterfaceAudience; import org.apache.pulsar.common.classification.InterfaceStability; @@ -737,6 +738,30 @@ public NotSupportedException(String msg) { } } + /** + * Not supported exception thrown by Pulsar client. + */ + public static class FeatureNotSupportedException extends NotSupportedException { + + @Getter + private final FailedFeatureCheck failedFeatureCheck; + + public FeatureNotSupportedException(String msg, FailedFeatureCheck failedFeatureCheck) { + super(msg); + this.failedFeatureCheck = failedFeatureCheck; + } + } + + /** + * "supports_auth_refresh" was introduced at "2.6" and is no longer supported, so skip this enum. + * "supports_broker_entry_metadata" was introduced at "2.8" and is no longer supported, so skip this enum. + * "supports_partial_producer" was introduced at "2.10" and is no longer supported, so skip this enum. + * "supports_topic_watchers" was introduced at "2.11" and is no longer supported, so skip this enum. + */ + public enum FailedFeatureCheck { + SupportsGetPartitionedMetadataWithoutAutoCreation; + } + /** * Not allowed exception thrown by Pulsar client. */ diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java index 980e8a0c78627..fb6b84b1096a1 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/BinaryProtoLookupService.java @@ -19,6 +19,7 @@ package org.apache.pulsar.client.impl; import static java.lang.String.format; +import static org.apache.pulsar.client.api.PulsarClientException.FailedFeatureCheck.SupportsGetPartitionedMetadataWithoutAutoCreation; import io.netty.buffer.ByteBuf; import java.net.InetSocketAddress; import java.net.URI; @@ -123,12 +124,13 @@ public CompletableFuture> getBroker(T */ @Override public CompletableFuture getPartitionedTopicMetadata( - TopicName topicName, boolean metadataAutoCreationEnabled) { + TopicName topicName, boolean metadataAutoCreationEnabled, boolean useFallbackForNonPIP344Brokers) { final MutableObject newFutureCreated = new MutableObject<>(); try { return partitionedMetadataInProgress.computeIfAbsent(topicName, tpName -> { CompletableFuture newFuture = getPartitionedTopicMetadata( - serviceNameResolver.resolveHost(), topicName, metadataAutoCreationEnabled); + serviceNameResolver.resolveHost(), topicName, metadataAutoCreationEnabled, + useFallbackForNonPIP344Brokers); newFutureCreated.setValue(newFuture); return newFuture; }); @@ -224,20 +226,31 @@ private CompletableFuture> findBroker } private CompletableFuture getPartitionedTopicMetadata(InetSocketAddress socketAddress, - TopicName topicName, boolean metadataAutoCreationEnabled) { + TopicName topicName, boolean metadataAutoCreationEnabled, boolean useFallbackForNonPIP344Brokers) { CompletableFuture partitionFuture = new CompletableFuture<>(); client.getCnxPool().getConnection(socketAddress).thenAccept(clientCnx -> { + boolean finalAutoCreationEnabled = metadataAutoCreationEnabled; if (!metadataAutoCreationEnabled && !clientCnx.isSupportsGetPartitionedMetadataWithoutAutoCreation()) { - partitionFuture.completeExceptionally(new PulsarClientException.NotSupportedException("The feature of" - + " getting partitions without auto-creation is not supported from the broker," - + " please upgrade the broker to the latest version.")); - return; + if (useFallbackForNonPIP344Brokers) { + log.info("[{}] Using original behavior of getPartitionedTopicMetadata(topic) in " + + "getPartitionedTopicMetadata(topic, false) " + + "since the target broker does not support PIP-344 and fallback is enabled.", topicName); + finalAutoCreationEnabled = true; + } else { + partitionFuture.completeExceptionally( + new PulsarClientException.FeatureNotSupportedException("The feature of " + + "getting partitions without auto-creation is not supported by the broker. " + + "Please upgrade the broker to version that supports PIP-344 to resolve this " + + "issue.", + SupportsGetPartitionedMetadataWithoutAutoCreation)); + return; + } } long requestId = client.newRequestId(); ByteBuf request = Commands.newPartitionMetadataRequest(topicName.toString(), requestId, - metadataAutoCreationEnabled); + finalAutoCreationEnabled); clientCnx.newLookup(request, requestId).whenComplete((r, t) -> { if (t != null) { log.warn("[{}] failed to get Partitioned metadata : {}", topicName, 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 4d6cf96a01068..2d31dc427d0c6 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 @@ -106,7 +106,7 @@ public Consumer subscribe() throws PulsarClientException { private CompletableFuture checkDlqAlreadyExists(String topic) { CompletableFuture existsFuture = new CompletableFuture<>(); - client.getPartitionedTopicMetadata(topic, false).thenAccept(metadata -> { + client.getPartitionedTopicMetadata(topic, false, true).thenAccept(metadata -> { TopicName topicName = TopicName.get(topic); if (topicName.isPersistent()) { // Either partitioned or non-partitioned, it exists. diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java index ba04aaa3b3117..38fdeff24a98a 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/HttpLookupService.java @@ -107,9 +107,13 @@ public CompletableFuture> getBroker(T }); } + /** + * {@inheritDoc} + * @param useFallbackForNonPIP344Brokers HttpLookupService ignores this parameter + */ @Override public CompletableFuture getPartitionedTopicMetadata( - TopicName topicName, boolean metadataAutoCreationEnabled) { + TopicName topicName, boolean metadataAutoCreationEnabled, boolean useFallbackForNonPIP344Brokers) { String format = topicName.isV2() ? "admin/v2/%s/partitions" : "admin/%s/partitions"; return httpClient.get(String.format(format, topicName.getLookupName()) + "?checkAllowAutoCreation=" + metadataAutoCreationEnabled, PartitionedTopicMetadata.class); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java index 675781b122f0a..9668cf2373b08 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java @@ -61,11 +61,19 @@ public interface LookupService extends AutoCloseable { /** * Returns {@link PartitionedTopicMetadata} for a given topic. * Note: this method will try to create the topic partitioned metadata if it does not exist. - * @deprecated Please call {{@link #getPartitionedTopicMetadata(TopicName, boolean)}}. + * @deprecated Please call {{@link #getPartitionedTopicMetadata(TopicName, boolean, boolean)}}. */ @Deprecated default CompletableFuture getPartitionedTopicMetadata(TopicName topicName) { - return getPartitionedTopicMetadata(topicName, true); + return getPartitionedTopicMetadata(topicName, true, true); + } + + /** + * See the doc {@link #getPartitionedTopicMetadata(TopicName, boolean, boolean)}. + */ + default CompletableFuture getPartitionedTopicMetadata(TopicName topicName, + boolean metadataAutoCreationEnabled) { + return getPartitionedTopicMetadata(topicName, metadataAutoCreationEnabled, false); } /** @@ -80,10 +88,15 @@ default CompletableFuture getPartitionedTopicMetadata( * 3.When {@param metadataAutoCreationEnabled} is "true," it will trigger an auto-creation for this topic(using * the default topic auto-creation strategy you set for the broker), and the corresponding result is returned. * For the result, see case 1. + * @param useFallbackForNonPIP344Brokers

    If true, fallback to the prior behavior of the method + * {@link #getPartitionedTopicMetadata(TopicName)} if the broker does not support the PIP-344 feature + * 'supports_get_partitioned_metadata_without_auto_creation'. This parameter only affects the behavior when + * {@param metadataAutoCreationEnabled} is false.

    * @version 3.3.0. */ CompletableFuture getPartitionedTopicMetadata(TopicName topicName, - boolean metadataAutoCreationEnabled); + boolean metadataAutoCreationEnabled, + boolean useFallbackForNonPIP344Brokers); /** * Returns current SchemaInfo {@link SchemaInfo} for a given topic. 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 526689934f8f1..303857eb187c2 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 @@ -957,7 +957,7 @@ public CompletableFuture subscribeAsync(String topicName, boolean createTo CompletableFuture subscribeResult = new CompletableFuture<>(); - client.getPartitionedTopicMetadata(topicName, true) + client.getPartitionedTopicMetadata(topicName, true, false) .thenAccept(metadata -> subscribeTopicPartitions(subscribeResult, fullTopicName, metadata.partitions, createTopicIfDoesNotExist)) .exceptionally(ex1 -> { 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 3ac9c5727caad..f28b81e8e5544 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 @@ -379,7 +379,7 @@ public CompletableFuture> createProducerAsync(ProducerConfigurat private CompletableFuture checkPartitions(String topic, boolean forceNoPartitioned, @Nullable String producerNameForLog) { CompletableFuture checkPartitions = new CompletableFuture<>(); - getPartitionedTopicMetadata(topic, !forceNoPartitioned).thenAccept(metadata -> { + getPartitionedTopicMetadata(topic, !forceNoPartitioned, true).thenAccept(metadata -> { if (forceNoPartitioned && metadata.partitions > 0) { String errorMsg = String.format("Can not create the producer[%s] for the topic[%s] that contains %s" + " partitions, but the producer does not support for a partitioned topic.", @@ -552,7 +552,7 @@ private CompletableFuture> doSingleTopicSubscribeAsync(ConsumerC String topic = conf.getSingleTopic(); - getPartitionedTopicMetadata(topic, true).thenAccept(metadata -> { + getPartitionedTopicMetadata(topic, true, false).thenAccept(metadata -> { if (log.isDebugEnabled()) { log.debug("[{}] Received topic metadata. partitions: {}", topic, metadata.partitions); } @@ -701,7 +701,7 @@ protected CompletableFuture> createSingleTopicReaderAsync( CompletableFuture> readerFuture = new CompletableFuture<>(); - getPartitionedTopicMetadata(topic, true).thenAccept(metadata -> { + getPartitionedTopicMetadata(topic, true, false).thenAccept(metadata -> { if (log.isDebugEnabled()) { log.debug("[{}] Received topic metadata. partitions: {}", topic, metadata.partitions); } @@ -1112,8 +1112,15 @@ public LookupService createLookup(String url) throws PulsarClientException { } } + /** + * @param useFallbackForNonPIP344Brokers

    If true, fallback to the prior behavior of the method + * getPartitionedTopicMetadata if the broker does not support the PIP-344 + * feature 'supports_get_partitioned_metadata_without_auto_creation'. This + * parameter only affects the behavior when + * {@param metadataAutoCreationEnabled} is false.

    + */ public CompletableFuture getPartitionedTopicMetadata( - String topic, boolean metadataAutoCreationEnabled) { + String topic, boolean metadataAutoCreationEnabled, boolean useFallbackForNonPIP344Brokers) { CompletableFuture metadataFuture = new CompletableFuture<>(); @@ -1125,8 +1132,8 @@ public CompletableFuture getPartitionedTopicMetadata( .setMandatoryStop(opTimeoutMs.get() * 2, TimeUnit.MILLISECONDS) .setMax(conf.getMaxBackoffIntervalNanos(), TimeUnit.NANOSECONDS) .create(); - getPartitionedTopicMetadata(topicName, backoff, opTimeoutMs, - metadataFuture, new ArrayList<>(), metadataAutoCreationEnabled); + getPartitionedTopicMetadata(topicName, backoff, opTimeoutMs, metadataFuture, new ArrayList<>(), + metadataAutoCreationEnabled, useFallbackForNonPIP344Brokers); } catch (IllegalArgumentException e) { return FutureUtil.failedFuture(new PulsarClientException.InvalidConfigurationException(e.getMessage())); } @@ -1138,10 +1145,11 @@ private void getPartitionedTopicMetadata(TopicName topicName, AtomicLong remainingTime, CompletableFuture future, List previousExceptions, - boolean metadataAutoCreationEnabled) { + boolean metadataAutoCreationEnabled, + boolean useFallbackForNonPIP344Brokers) { long startTime = System.nanoTime(); - CompletableFuture queryFuture = - lookup.getPartitionedTopicMetadata(topicName, metadataAutoCreationEnabled); + CompletableFuture queryFuture = lookup.getPartitionedTopicMetadata(topicName, + metadataAutoCreationEnabled, useFallbackForNonPIP344Brokers); queryFuture.thenAccept(future::complete).exceptionally(e -> { remainingTime.addAndGet(-1 * TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); long nextDelay = Math.min(backoff.next(), remainingTime.get()); @@ -1162,7 +1170,7 @@ private void getPartitionedTopicMetadata(TopicName topicName, + "Will try again in {} ms", topicName, nextDelay); remainingTime.addAndGet(-nextDelay); getPartitionedTopicMetadata(topicName, backoff, remainingTime, future, previousExceptions, - metadataAutoCreationEnabled); + metadataAutoCreationEnabled, useFallbackForNonPIP344Brokers); }, nextDelay, TimeUnit.MILLISECONDS); return null; }); @@ -1170,7 +1178,7 @@ private void getPartitionedTopicMetadata(TopicName topicName, @Override public CompletableFuture> getPartitionsForTopic(String topic, boolean metadataAutoCreationEnabled) { - return getPartitionedTopicMetadata(topic, metadataAutoCreationEnabled).thenApply(metadata -> { + return getPartitionedTopicMetadata(topic, metadataAutoCreationEnabled, false).thenApply(metadata -> { if (metadata.partitions > 0) { TopicName topicName = TopicName.get(topic); List partitions = new ArrayList<>(metadata.partitions); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/transaction/TransactionCoordinatorClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/transaction/TransactionCoordinatorClientImpl.java index 45a3ad4f978b1..ce19cbf873eea 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/transaction/TransactionCoordinatorClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/transaction/TransactionCoordinatorClientImpl.java @@ -80,7 +80,7 @@ public void start() throws TransactionCoordinatorClientException { public CompletableFuture startAsync() { if (STATE_UPDATER.compareAndSet(this, State.NONE, State.STARTING)) { return pulsarClient.getPartitionedTopicMetadata( - SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN.getPartitionedTopicName(), true) + SystemTopicNames.TRANSACTION_COORDINATOR_ASSIGN.getPartitionedTopicName(), true, false) .thenCompose(partitionMeta -> { List> connectFutureList = new ArrayList<>(); if (LOG.isDebugEnabled()) { diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImplTest.java index 191124bb7b002..02a4d2ebba8c1 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImplTest.java @@ -154,7 +154,7 @@ private MultiTopicsConsumerImpl createMultiTopicsConsumer( int completionDelayMillis = 100; Schema schema = Schema.BYTES; PulsarClientImpl clientMock = createPulsarClientMockWithMockedClientCnx(executorProvider, internalExecutor); - when(clientMock.getPartitionedTopicMetadata(any(), anyBoolean())) + when(clientMock.getPartitionedTopicMetadata(any(), anyBoolean(), anyBoolean())) .thenAnswer(invocation -> createDelayedCompletedFuture( new PartitionedTopicMetadata(), completionDelayMillis)); MultiTopicsConsumerImpl impl = new MultiTopicsConsumerImpl( @@ -203,7 +203,7 @@ public void testConsumerCleanupOnSubscribeFailure() { int completionDelayMillis = 10; Schema schema = Schema.BYTES; PulsarClientImpl clientMock = createPulsarClientMockWithMockedClientCnx(executorProvider, internalExecutor); - when(clientMock.getPartitionedTopicMetadata(any(), anyBoolean())) + when(clientMock.getPartitionedTopicMetadata(any(), anyBoolean(), anyBoolean())) .thenAnswer(invocation -> createExceptionFuture( new PulsarClientException.InvalidConfigurationException("a mock exception"), completionDelayMillis)); CompletableFuture> completeFuture = new CompletableFuture<>(); @@ -240,7 +240,7 @@ public void testDontCheckForPartitionsUpdatesOnNonPartitionedTopics() throws Exc // Simulate non partitioned topics PartitionedTopicMetadata metadata = new PartitionedTopicMetadata(0); - when(clientMock.getPartitionedTopicMetadata(any(), anyBoolean())) + when(clientMock.getPartitionedTopicMetadata(any(), anyBoolean(), anyBoolean())) .thenReturn(CompletableFuture.completedFuture(metadata)); CompletableFuture> completeFuture = new CompletableFuture<>(); @@ -252,7 +252,7 @@ public void testDontCheckForPartitionsUpdatesOnNonPartitionedTopics() throws Exc // getPartitionedTopicMetadata should have been called only the first time, for each of the 3 topics, // but not anymore since the topics are not partitioned. - verify(clientMock, times(3)).getPartitionedTopicMetadata(any(), anyBoolean()); + verify(clientMock, times(3)).getPartitionedTopicMetadata(any(), anyBoolean(), anyBoolean()); } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java index e13c060a052ec..8bf0c037e4683 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PulsarClientImplTest.java @@ -107,7 +107,7 @@ public void testConsumerIsClosed() throws Exception { nullable(String.class))) .thenReturn(CompletableFuture.completedFuture( new GetTopicsResult(Collections.emptyList(), null, false, true))); - when(lookup.getPartitionedTopicMetadata(any(TopicName.class), anyBoolean())) + when(lookup.getPartitionedTopicMetadata(any(TopicName.class), anyBoolean(), anyBoolean())) .thenReturn(CompletableFuture.completedFuture(new PartitionedTopicMetadata())); when(lookup.getBroker(any())) .thenReturn(CompletableFuture.completedFuture( diff --git a/pulsar-common/src/main/proto/PulsarApi.proto b/pulsar-common/src/main/proto/PulsarApi.proto index f56df6ae9d103..fa2181b484f41 100644 --- a/pulsar-common/src/main/proto/PulsarApi.proto +++ b/pulsar-common/src/main/proto/PulsarApi.proto @@ -295,6 +295,7 @@ message CommandConnect { optional string proxy_version = 11; // Version of the proxy. Should only be forwarded by a proxy. } +// Please also add a new enum for the class "PulsarClientException.FailedFeatureCheck" when adding a new feature flag. message FeatureFlags { optional bool supports_auth_refresh = 1 [default = false]; optional bool supports_broker_entry_metadata = 2 [default = false]; From e16a0e26e1d7e59fe6ba9c123ae4e2c0e4428f9c Mon Sep 17 00:00:00 2001 From: Heesung Sohn <103456639+heesung-sn@users.noreply.github.com> Date: Sat, 10 Aug 2024 02:24:26 -0700 Subject: [PATCH 662/699] [improve][broker] Explicitly close LB internal topics when playing a follower (ExtensibleLoadManagerImpl only) (#23144) (cherry picked from commit 1b43b9de72ccf57b56375870b43d551def59d8a3) (cherry picked from commit 1e1860a3d915eb9825646449fc075d2bdb31a4a9) --- .../extensions/ExtensibleLoadManagerImpl.java | 20 +- .../ExtensibleLoadManagerImplTest.java | 235 +++++++++++------- 2 files changed, 162 insertions(+), 93 deletions(-) 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 e0ed8c9198d57..9dc4967cd741a 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 @@ -122,7 +122,7 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager { private static final String ELECTION_ROOT = "/loadbalance/extension/leader"; - private static final Set INTERNAL_TOPICS = + public static final Set INTERNAL_TOPICS = Set.of(BROKER_LOAD_DATA_STORE_TOPIC, TOP_BUNDLES_LOAD_DATA_STORE_TOPIC, TOPIC); @VisibleForTesting @@ -144,7 +144,10 @@ public class ExtensibleLoadManagerImpl implements ExtensibleLoadManager { @Getter private IsolationPoliciesHelper isolationPoliciesHelper; + @Getter private LoadDataStore brokerLoadDataStore; + + @Getter private LoadDataStore topBundlesLoadDataStore; private LoadManagerScheduler unloadScheduler; @@ -246,6 +249,7 @@ public enum Role { Follower } + @Getter private volatile Role role; /** @@ -849,6 +853,7 @@ synchronized void playFollower() { } unloadScheduler.close(); serviceUnitStateChannel.cancelOwnershipMonitor(); + closeInternalTopics(); brokerLoadDataStore.init(); topBundlesLoadDataStore.close(); topBundlesLoadDataStore.startProducer(); @@ -945,12 +950,13 @@ public void disableBroker() throws Exception { private void closeInternalTopics() { List> futures = new ArrayList<>(); for (String name : INTERNAL_TOPICS) { - futures.add(pulsar.getBrokerService().getTopicIfExists(name) - .thenAccept(topicOptional -> topicOptional.ifPresent(topic -> topic.close(true))) - .exceptionally(__ -> { - log.warn("Failed to close internal topic:{}", name); - return null; - })); + pulsar.getBrokerService() + .getTopicReference(name) + .ifPresent(topic -> futures.add(topic.close(true) + .exceptionally(__ -> { + log.warn("Failed to close internal topic:{}", name); + return null; + }))); } try { FutureUtil.waitForAll(futures) 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 c219fafc284d1..b941293c35f04 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 @@ -94,7 +94,6 @@ import org.apache.pulsar.broker.loadbalance.extensions.models.UnloadCounter; import org.apache.pulsar.broker.loadbalance.extensions.reporter.BrokerLoadDataReporter; import org.apache.pulsar.broker.loadbalance.extensions.scheduler.TransferShedder; -import org.apache.pulsar.broker.loadbalance.extensions.store.LoadDataStore; import org.apache.pulsar.broker.loadbalance.impl.ModularLoadManagerImpl; import org.apache.pulsar.broker.lookup.LookupResult; import org.apache.pulsar.broker.namespace.LookupOptions; @@ -107,7 +106,6 @@ import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionInitialPosition; -import org.apache.pulsar.client.impl.TableViewImpl; import org.apache.pulsar.common.naming.NamespaceBundle; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.ServiceUnitId; @@ -118,6 +116,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.policies.data.loadbalancer.NamespaceBundleStats; import org.apache.pulsar.policies.data.loadbalancer.ResourceUsage; import org.apache.pulsar.policies.data.loadbalancer.SystemResourceUsage; import org.awaitility.Awaitility; @@ -792,55 +791,49 @@ private void assertLookupSLANamespaceOwner(PulsarService pulsar, assertEquals(result, expectedBrokerServiceUrl); } - @Test(priority = 10) - public void testTopBundlesLoadDataStoreTableViewFromChannelOwner() throws Exception { - var topBundlesLoadDataStorePrimary = - (LoadDataStore) FieldUtils.readDeclaredField(primaryLoadManager, "topBundlesLoadDataStore", true); - var serviceUnitStateChannelPrimary = - (ServiceUnitStateChannelImpl) FieldUtils.readDeclaredField(primaryLoadManager, - "serviceUnitStateChannel", true); - var tvPrimary = - (TableViewImpl) FieldUtils.readDeclaredField(topBundlesLoadDataStorePrimary, "tableView", true); - - var topBundlesLoadDataStoreSecondary = - (LoadDataStore) FieldUtils.readDeclaredField(secondaryLoadManager, "topBundlesLoadDataStore", true); - var tvSecondary = - (TableViewImpl) FieldUtils.readDeclaredField(topBundlesLoadDataStoreSecondary, "tableView", true); - - if (serviceUnitStateChannelPrimary.isChannelOwnerAsync().get(5, TimeUnit.SECONDS)) { - assertNotNull(tvPrimary); - assertNull(tvSecondary); - } else { - assertNull(tvPrimary); - assertNotNull(tvSecondary); + + private void makePrimaryAsLeader() throws Exception { + log.info("makePrimaryAsLeader"); + if (channel2.isChannelOwner()) { + pulsar2.getLeaderElectionService().close(); + Awaitility.await().atMost(30, TimeUnit.SECONDS).untilAsserted(() -> { + assertTrue(channel1.isChannelOwner()); + }); + pulsar2.getLeaderElectionService().start(); } - restartBroker(); - pulsar1 = pulsar; - setPrimaryLoadManager(); - admin.namespaces().setNamespaceReplicationClusters(defaultTestNamespace, - Sets.newHashSet(this.conf.getClusterName())); - - var serviceUnitStateChannelPrimaryNew = - (ServiceUnitStateChannelImpl) FieldUtils.readDeclaredField(primaryLoadManager, - "serviceUnitStateChannel", true); - var topBundlesLoadDataStorePrimaryNew = - (LoadDataStore) FieldUtils.readDeclaredField(primaryLoadManager, "topBundlesLoadDataStore" - , true); - Awaitility.await().atMost(5, TimeUnit.SECONDS).untilAsserted(() -> { - assertFalse(serviceUnitStateChannelPrimaryNew.isChannelOwnerAsync().get(5, TimeUnit.SECONDS)); - assertNotNull(FieldUtils.readDeclaredField(topBundlesLoadDataStoreSecondary, "tableView" - , true)); - assertNull(FieldUtils.readDeclaredField(topBundlesLoadDataStorePrimaryNew, "tableView" - , true)); - } - ); + Awaitility.await().atMost(30, TimeUnit.SECONDS).untilAsserted(() -> { + assertTrue(channel1.isChannelOwner()); + }); + Awaitility.await().atMost(30, TimeUnit.SECONDS).untilAsserted(() -> { + assertFalse(channel2.isChannelOwner()); + }); } - @Test - public void testRoleChange() throws Exception { - var topBundlesLoadDataStorePrimary = (LoadDataStore) - FieldUtils.readDeclaredField(primaryLoadManager, "topBundlesLoadDataStore", true); + private void makeSecondaryAsLeader() throws Exception { + log.info("makeSecondaryAsLeader"); + if (channel1.isChannelOwner()) { + pulsar1.getLeaderElectionService().close(); + Awaitility.await().atMost(30, TimeUnit.SECONDS).untilAsserted(() -> { + assertTrue(channel2.isChannelOwner()); + }); + pulsar1.getLeaderElectionService().start(); + } + + Awaitility.await().atMost(30, TimeUnit.SECONDS).untilAsserted(() -> { + assertTrue(channel2.isChannelOwner()); + }); + Awaitility.await().atMost(30, TimeUnit.SECONDS).untilAsserted(() -> { + assertFalse(channel1.isChannelOwner()); + }); + } + + @Test(timeOut = 30 * 1000) + public void testRoleChangeIdempotency() throws Exception { + + makePrimaryAsLeader(); + + var topBundlesLoadDataStorePrimary = primaryLoadManager.getTopBundlesLoadDataStore(); var topBundlesLoadDataStorePrimarySpy = spy(topBundlesLoadDataStorePrimary); AtomicInteger countPri = new AtomicInteger(3); AtomicInteger countPri2 = new AtomicInteger(3); @@ -861,8 +854,7 @@ public void testRoleChange() throws Exception { return null; }).when(topBundlesLoadDataStorePrimarySpy).closeTableView(); - var topBundlesLoadDataStoreSecondary = (LoadDataStore) - FieldUtils.readDeclaredField(secondaryLoadManager, "topBundlesLoadDataStore", true); + var topBundlesLoadDataStoreSecondary = secondaryLoadManager.getTopBundlesLoadDataStore(); var topBundlesLoadDataStoreSecondarySpy = spy(topBundlesLoadDataStoreSecondary); AtomicInteger countSec = new AtomicInteger(3); AtomicInteger countSec2 = new AtomicInteger(3); @@ -890,51 +882,30 @@ public void testRoleChange() throws Exception { topBundlesLoadDataStoreSecondarySpy, true); - if (channel1.isChannelOwnerAsync().get(5, TimeUnit.SECONDS)) { - primaryLoadManager.playLeader(); - secondaryLoadManager.playFollower(); - verify(topBundlesLoadDataStorePrimarySpy, times(3)).startTableView(); - verify(topBundlesLoadDataStorePrimarySpy, times(5)).closeTableView(); - verify(topBundlesLoadDataStoreSecondarySpy, times(0)).startTableView(); - verify(topBundlesLoadDataStoreSecondarySpy, times(3)).closeTableView(); - } else { - primaryLoadManager.playFollower(); - secondaryLoadManager.playLeader(); - verify(topBundlesLoadDataStoreSecondarySpy, times(3)).startTableView(); - verify(topBundlesLoadDataStoreSecondarySpy, times(5)).closeTableView(); - verify(topBundlesLoadDataStorePrimarySpy, times(0)).startTableView(); - verify(topBundlesLoadDataStorePrimarySpy, times(3)).closeTableView(); - } + + primaryLoadManager.playLeader(); + secondaryLoadManager.playFollower(); + verify(topBundlesLoadDataStorePrimarySpy, times(3)).startTableView(); + verify(topBundlesLoadDataStorePrimarySpy, times(5)).closeTableView(); + verify(topBundlesLoadDataStoreSecondarySpy, times(0)).startTableView(); + verify(topBundlesLoadDataStoreSecondarySpy, times(3)).closeTableView(); + primaryLoadManager.playFollower(); secondaryLoadManager.playFollower(); + assertEquals(ExtensibleLoadManagerImpl.Role.Leader, + primaryLoadManager.getRole()); + assertEquals(ExtensibleLoadManagerImpl.Role.Follower, + secondaryLoadManager.getRole()); - if (channel1.isChannelOwnerAsync().get(5, TimeUnit.SECONDS)) { - assertEquals(ExtensibleLoadManagerImpl.Role.Leader, - FieldUtils.readDeclaredField(primaryLoadManager, "role", true)); - assertEquals(ExtensibleLoadManagerImpl.Role.Follower, - FieldUtils.readDeclaredField(secondaryLoadManager, "role", true)); - } else { - assertEquals(ExtensibleLoadManagerImpl.Role.Follower, - FieldUtils.readDeclaredField(primaryLoadManager, "role", true)); - assertEquals(ExtensibleLoadManagerImpl.Role.Leader, - FieldUtils.readDeclaredField(secondaryLoadManager, "role", true)); - } primaryLoadManager.playLeader(); secondaryLoadManager.playLeader(); + assertEquals(ExtensibleLoadManagerImpl.Role.Leader, + primaryLoadManager.getRole()); + assertEquals(ExtensibleLoadManagerImpl.Role.Follower, + secondaryLoadManager.getRole()); - if (channel1.isChannelOwnerAsync().get(5, TimeUnit.SECONDS)) { - assertEquals(ExtensibleLoadManagerImpl.Role.Leader, - FieldUtils.readDeclaredField(primaryLoadManager, "role", true)); - assertEquals(ExtensibleLoadManagerImpl.Role.Follower, - FieldUtils.readDeclaredField(secondaryLoadManager, "role", true)); - } else { - assertEquals(ExtensibleLoadManagerImpl.Role.Follower, - FieldUtils.readDeclaredField(primaryLoadManager, "role", true)); - assertEquals(ExtensibleLoadManagerImpl.Role.Leader, - FieldUtils.readDeclaredField(secondaryLoadManager, "role", true)); - } } finally { FieldUtils.writeDeclaredField(primaryLoadManager, "topBundlesLoadDataStore", topBundlesLoadDataStorePrimary, true); @@ -942,6 +913,98 @@ public void testRoleChange() throws Exception { topBundlesLoadDataStoreSecondary, true); } } + @Test(timeOut = 30 * 1000) + public void testRoleChange() throws Exception { + makePrimaryAsLeader(); + + var leader = primaryLoadManager; + var follower = secondaryLoadManager; + + BrokerLoadData brokerLoadExpected = new BrokerLoadData(); + SystemResourceUsage usage = new SystemResourceUsage(); + var cpu = new ResourceUsage(1.0, 100.0); + String key = "b1"; + usage.setCpu(cpu); + brokerLoadExpected.update(usage, 0, 0, 0, 0, 0, 0, conf); + String bundle = "public/default/0x00000000_0xffffffff"; + TopBundlesLoadData topBundlesExpected = new TopBundlesLoadData(); + topBundlesExpected.getTopBundlesLoadData().clear(); + topBundlesExpected.getTopBundlesLoadData().add(new TopBundlesLoadData.BundleLoadData(bundle, new NamespaceBundleStats())); + + follower.getBrokerLoadDataStore().pushAsync(key, brokerLoadExpected); + follower.getTopBundlesLoadDataStore().pushAsync(bundle, topBundlesExpected); + + Awaitility.await().atMost(30, TimeUnit.SECONDS).untilAsserted(() -> { + + assertNotNull(FieldUtils.readDeclaredField(leader.getTopBundlesLoadDataStore(), "tableView", true)); + assertNull(FieldUtils.readDeclaredField(follower.getTopBundlesLoadDataStore(), "tableView", true)); + + for (String internalTopic : ExtensibleLoadManagerImpl.INTERNAL_TOPICS) { + assertTrue(leader.pulsar.getBrokerService().getTopicReference(internalTopic) + .isPresent()); + assertTrue(follower.pulsar.getBrokerService().getTopicReference(internalTopic) + .isEmpty()); + + assertTrue(leader.pulsar.getNamespaceService() + .isServiceUnitOwnedAsync(TopicName.get(internalTopic)).get()); + assertFalse(follower.pulsar.getNamespaceService() + .isServiceUnitOwnedAsync(TopicName.get(internalTopic)).get()); + } + + var actualBrokerLoadLeader = leader.getBrokerLoadDataStore().get(key); + if (actualBrokerLoadLeader.isPresent()) { + assertEquals(actualBrokerLoadLeader.get(), brokerLoadExpected); + } + + var actualTopBundlesLeader = leader.getTopBundlesLoadDataStore().get(bundle); + if (actualTopBundlesLeader.isPresent()) { + assertEquals(actualTopBundlesLeader.get(), topBundlesExpected); + } + + var actualBrokerLoadFollower = follower.getBrokerLoadDataStore().get(key); + if (actualBrokerLoadFollower.isPresent()) { + assertEquals(actualBrokerLoadFollower.get(), brokerLoadExpected); + } + }); + + makeSecondaryAsLeader(); + + var leader2 = secondaryLoadManager; + var follower2 = primaryLoadManager; + + brokerLoadExpected.update(usage, 1, 0, 0, 0, 0, 0, conf); + topBundlesExpected.getTopBundlesLoadData().get(0).stats().msgRateIn = 1; + + follower.getBrokerLoadDataStore().pushAsync(key, brokerLoadExpected); + follower.getTopBundlesLoadDataStore().pushAsync(bundle, topBundlesExpected); + + Awaitility.await().atMost(30, TimeUnit.SECONDS).ignoreExceptions().untilAsserted(() -> { + assertNotNull(FieldUtils.readDeclaredField(leader2.getTopBundlesLoadDataStore(), "tableView", true)); + assertNull(FieldUtils.readDeclaredField(follower2.getTopBundlesLoadDataStore(), "tableView", true)); + + for (String internalTopic : ExtensibleLoadManagerImpl.INTERNAL_TOPICS) { + assertTrue(leader2.pulsar.getBrokerService().getTopicReference(internalTopic) + .isPresent()); + assertTrue(follower2.pulsar.getBrokerService().getTopicReference(internalTopic) + .isEmpty()); + + assertTrue(leader2.pulsar.getNamespaceService() + .isServiceUnitOwnedAsync(TopicName.get(internalTopic)).get()); + assertFalse(follower2.pulsar.getNamespaceService() + .isServiceUnitOwnedAsync(TopicName.get(internalTopic)).get()); + } + + + var actualBrokerLoadLeader = leader2.getBrokerLoadDataStore().get(key); + assertEquals(actualBrokerLoadLeader.get(), brokerLoadExpected); + + var actualTopBundlesLeader = leader2.getTopBundlesLoadDataStore().get(bundle); + assertEquals(actualTopBundlesLeader.get(), topBundlesExpected); + + var actualBrokerLoadFollower = follower2.getBrokerLoadDataStore().get(key); + assertEquals(actualBrokerLoadFollower.get(), brokerLoadExpected); + }); + } @Test public void testGetMetrics() throws Exception { From b3d47d201a0664cc3ae4e0d2057956f934f2db83 Mon Sep 17 00:00:00 2001 From: hanmz Date: Tue, 13 Aug 2024 11:06:01 +0800 Subject: [PATCH 663/699] [fix][broker] Fix 'Disabled replicated subscriptions controller' logic and logging (#23142) (cherry picked from commit 9bf714ff756b0729094e9a8611137d3dcfaed5b0) (cherry picked from commit 4235a53241c359c10826e98327814f04335c35cc) --- .../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 272aaf0a5084a..033f8e1350427 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 @@ -3771,8 +3771,8 @@ private synchronized void checkReplicatedSubscriptionControllerState(boolean sho log.info("[{}] Enabling replicated subscriptions controller", topic); replicatedSubscriptionsController = Optional.of(new ReplicatedSubscriptionsController(this, brokerService.pulsar().getConfiguration().getClusterName())); - } else if (isCurrentlyEnabled && !shouldBeEnabled || !isEnableReplicatedSubscriptions - || !replicationEnabled) { + } else if (isCurrentlyEnabled && (!shouldBeEnabled || !isEnableReplicatedSubscriptions + || !replicationEnabled)) { log.info("[{}] Disabled replicated subscriptions controller", topic); replicatedSubscriptionsController.ifPresent(ReplicatedSubscriptionsController::close); replicatedSubscriptionsController = Optional.empty(); From 68db449642bfbc82f782be6b2922c0e367564a7a Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 12 Aug 2024 23:10:19 +0800 Subject: [PATCH 664/699] [fix] [broker] Let Pending ack handler can retry to init when encounters a metadata store error (#23153) (cherry picked from commit 2dde4032127e228c34ac2c3729191b76220aec29) (cherry picked from commit d40619360337b4355d5ff0173bb1f39cefefe128) --- .../pendingack/impl/PendingAckHandleImpl.java | 6 +++- .../pendingack/PendingAckPersistentTest.java | 35 ++++++++++++++----- 2 files changed, 32 insertions(+), 9 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java index 9d07af4d26c44..3481c6e39d23a 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/transaction/pendingack/impl/PendingAckHandleImpl.java @@ -39,6 +39,7 @@ import java.util.concurrent.TimeUnit; import lombok.Getter; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.api.BKException; import org.apache.bookkeeper.mledger.ManagedLedger; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.Position; @@ -67,6 +68,7 @@ import org.apache.pulsar.common.util.FutureUtil; import org.apache.pulsar.common.util.RecoverTimeRecord; import org.apache.pulsar.common.util.collections.BitSetRecyclable; +import org.apache.pulsar.metadata.api.MetadataStoreException; import org.apache.pulsar.transaction.common.exception.TransactionConflictException; /** @@ -990,7 +992,9 @@ private static boolean isRetryableException(Throwable ex) { && !(realCause instanceof ManagedLedgerException.NonRecoverableLedgerException)) || realCause instanceof PulsarClientException.BrokerPersistenceException || realCause instanceof PulsarClientException.LookupException - || realCause instanceof PulsarClientException.ConnectException; + || realCause instanceof PulsarClientException.ConnectException + || realCause instanceof MetadataStoreException + || realCause instanceof BKException; } @Override 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 93a2f274517d5..6d2bb0c77b8a9 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 @@ -45,11 +45,13 @@ import java.util.concurrent.TimeUnit; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.client.api.BKException; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.map.LinkedMap; import org.apache.pulsar.PrometheusMetricsTestUtil; +import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.PulsarService; import org.apache.pulsar.broker.service.AbstractTopic; import org.apache.pulsar.broker.service.BrokerService; @@ -78,10 +80,12 @@ import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TopicStats; import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.metadata.api.MetadataStoreException; 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.Test; /** @@ -105,16 +109,26 @@ protected void cleanup() { super.internalCleanup(); } + + @DataProvider(name = "retryableErrors") + public Object[][] retryableErrors() { + return new Object[][] { + {new ManagedLedgerException("mock retryable error")}, + {new MetadataStoreException("mock retryable error")}, + {new BKException(-1)}, + }; + } + /** * Test consumer can be built successfully with retryable exception * and get correct error with no-retryable exception. * @throws Exception */ - @Test(timeOut = 60000) - public void testBuildConsumerEncounterPendingAckInitFailure() throws Exception { + @Test(timeOut = 60000, dataProvider = "retryableErrors") + public void testBuildConsumerEncounterPendingAckInitFailure(Exception retryableError) throws Exception { // 1. Prepare and make sure the consumer can be built successfully. - String topic = NAMESPACE1 + "/testUnloadSubscriptionWhenFailedInitPendingAck"; - @Cleanup + String topic = BrokerTestUtil.newUniqueName(NAMESPACE1 + "/tp"); + admin.topics().createNonPartitionedTopic(topic); Consumer consumer1 = pulsarClient.newConsumer() .subscriptionName("subName1") .topic(topic) @@ -132,11 +146,10 @@ public void testBuildConsumerEncounterPendingAckInitFailure() throws Exception { // The consumer will be built successfully after one time retry. when(mockProvider.checkInitializedBefore(any())) // First, the method checkInitializedBefore will fail with a retryable exception. - .thenReturn(FutureUtil.failedFuture(new ManagedLedgerException("mock fail initialize"))) + .thenReturn(FutureUtil.failedFuture(retryableError)) // Then, the method will be executed successfully. .thenReturn(CompletableFuture.completedFuture(false)); transactionPendingAckStoreProviderField.set(pulsarServiceList.get(0), mockProvider); - @Cleanup Consumer consumer2 = pulsarClient.newConsumer() .subscriptionName("subName2") .topic(topic) @@ -153,7 +166,6 @@ public void testBuildConsumerEncounterPendingAckInitFailure() throws Exception { // Then, the method will be executed successfully. .thenCallRealMethod(); transactionPendingAckStoreProviderField.set(pulsarServiceList.get(0), mockProvider); - @Cleanup Consumer consumer3 = pulsarClient.newConsumer() .subscriptionName("subName3") .topic(topic) @@ -166,7 +178,7 @@ public void testBuildConsumerEncounterPendingAckInitFailure() throws Exception { .thenReturn(FutureUtil.failedFuture(new ManagedLedgerException .NonRecoverableLedgerException("mock fail"))) .thenReturn(CompletableFuture.completedFuture(false)); - @Cleanup PulsarClient pulsarClient = PulsarClient.builder() + PulsarClient pulsarClient = PulsarClient.builder() .serviceUrl(pulsarServiceList.get(0).getBrokerServiceUrl()) .operationTimeout(3, TimeUnit.SECONDS) .build(); @@ -180,6 +192,13 @@ public void testBuildConsumerEncounterPendingAckInitFailure() throws Exception { } catch (Exception exception) { assertTrue(exception.getMessage().contains("Failed to init transaction pending ack.")); } + + // cleanup. + consumer1.close(); + consumer2.close(); + consumer3.close(); + pulsarClient.close(); + admin.topics().delete(topic, false); } @Test From b46125ed6a2ad903b48e55a532984e7c2cbfb0c0 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 13 Aug 2024 00:47:04 +0800 Subject: [PATCH 665/699] [improve] [client]Add new ServiceUrlProvider implementation: SameAuthParamsAutoClusterFailover (#23129) - excluding changes to pulsar-broker/src/test/java/org/apache/pulsar/client/api/NonDurableSubscriptionTest.java PR #23168 (cherry picked from commit 06a2f5cc63c126a0262bec4602b81c1c716e4e36) (cherry picked from commit 286a5dc1272276f71d0dff577ead1df2e3be5c30) --- ...thParamsLookupAutoClusterFailoverTest.java | 176 +++++++++ .../auth/MockedPulsarServiceBaseTest.java | 2 +- .../broker/service/NetworkErrorTestBase.java | 2 +- .../service/OneWayReplicatorTestBase.java | 21 +- .../pulsar/client/api/ServiceUrlProvider.java | 2 +- ...meAuthParamsLookupAutoClusterFailover.java | 343 ++++++++++++++++++ .../client/impl/AutoClusterFailoverTest.java | 12 +- .../impl/ControlledClusterFailoverTest.java | 5 +- 8 files changed, 548 insertions(+), 15 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/SameAuthParamsLookupAutoClusterFailoverTest.java create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/SameAuthParamsLookupAutoClusterFailover.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/SameAuthParamsLookupAutoClusterFailoverTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/SameAuthParamsLookupAutoClusterFailoverTest.java new file mode 100644 index 0000000000000..b39f8135e0e0c --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/SameAuthParamsLookupAutoClusterFailoverTest.java @@ -0,0 +1,176 @@ +/* + * 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.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.CA_CERT_FILE_PATH; +import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.getTlsFileForClient; +import static org.apache.pulsar.client.impl.SameAuthParamsLookupAutoClusterFailover.PulsarServiceState; +import io.netty.channel.EventLoopGroup; +import java.net.ServerSocket; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import org.apache.pulsar.broker.service.NetworkErrorTestBase; +import org.apache.pulsar.broker.service.OneWayReplicatorTestBase; +import org.apache.pulsar.client.api.ClientBuilder; +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.impl.SameAuthParamsLookupAutoClusterFailover; +import org.apache.pulsar.client.impl.auth.AuthenticationTls; +import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +public class SameAuthParamsLookupAutoClusterFailoverTest extends OneWayReplicatorTestBase { + + public void setup() throws Exception { + super.setup(); + } + + @Override + @AfterMethod(alwaysRun = true, timeOut = 300000) + public void cleanup() throws Exception { + super.cleanup(); + } + + @DataProvider(name = "enabledTls") + public Object[][] enabledTls () { + return new Object[][] { + {true}, + {false} + }; + } + + @Test(dataProvider = "enabledTls", timeOut = 240 * 1000) + public void testAutoClusterFailover(boolean enabledTls) throws Exception { + // Start clusters. + setup(); + ServerSocket dummyServer = new ServerSocket(NetworkErrorTestBase.getOneFreePort()); + + // Initialize client. + String urlProxy = enabledTls ? "pulsar+tls://127.0.0.1:" + dummyServer.getLocalPort() + : "pulsar://127.0.0.1:" + dummyServer.getLocalPort(); + String url1 = enabledTls ? pulsar1.getBrokerServiceUrlTls() : pulsar1.getBrokerServiceUrl(); + String url2 = enabledTls ? pulsar2.getBrokerServiceUrlTls() : pulsar2.getBrokerServiceUrl(); + final String[] urlArray = new String[]{url1, urlProxy, url2}; + final SameAuthParamsLookupAutoClusterFailover failover = SameAuthParamsLookupAutoClusterFailover.builder() + .pulsarServiceUrlArray(urlArray) + .failoverThreshold(5) + .recoverThreshold(5) + .checkHealthyIntervalMs(300) + .testTopic("a/b/c") + .markTopicNotFoundAsAvailable(true) + .build(); + ClientBuilder clientBuilder = PulsarClient.builder().serviceUrlProvider(failover); + if (enabledTls) { + Map authParams = new HashMap<>(); + authParams.put("tlsCertFile", getTlsFileForClient("admin.cert")); + authParams.put("tlsKeyFile", getTlsFileForClient("admin.key-pk8")); + clientBuilder.authentication(AuthenticationTls.class.getName(), authParams) + .enableTls(true) + .allowTlsInsecureConnection(false) + .tlsTrustCertsFilePath(CA_CERT_FILE_PATH); + } + final PulsarClient client = clientBuilder.build(); + failover.initialize(client); + final EventLoopGroup executor = WhiteboxImpl.getInternalState(failover, "executor"); + final PulsarServiceState[] stateArray = + WhiteboxImpl.getInternalState(failover, "pulsarServiceStateArray"); + + // Test all things is fine. + final String tp = BrokerTestUtil.newUniqueName(nonReplicatedNamespace + "/tp"); + final Producer producer = client.newProducer(Schema.STRING).topic(tp).create(); + producer.send("0"); + Assert.assertEquals(failover.getCurrentPulsarServiceIndex(), 0); + + CompletableFuture checkStatesFuture1 = new CompletableFuture<>(); + executor.submit(() -> { + boolean res = stateArray[0] == PulsarServiceState.Healthy; + res = res & stateArray[1] == PulsarServiceState.Healthy; + res = res & stateArray[2] == PulsarServiceState.Healthy; + checkStatesFuture1.complete(res); + }); + Assert.assertTrue(checkStatesFuture1.join()); + + // Test failover 0 --> 3. + pulsar1.close(); + Awaitility.await().atMost(60, TimeUnit.SECONDS).untilAsserted(() -> { + CompletableFuture checkStatesFuture2 = new CompletableFuture<>(); + executor.submit(() -> { + boolean res = stateArray[0] == PulsarServiceState.Failed; + res = res & stateArray[1] == PulsarServiceState.Failed; + res = res & stateArray[2] == PulsarServiceState.Healthy; + checkStatesFuture2.complete(res); + }); + Assert.assertTrue(checkStatesFuture2.join()); + producer.send("0->2"); + Assert.assertEquals(failover.getCurrentPulsarServiceIndex(), 2); + }); + + // Test recover 2 --> 1. + executor.execute(() -> { + urlArray[1] = url2; + }); + Awaitility.await().atMost(60, TimeUnit.SECONDS).untilAsserted(() -> { + CompletableFuture checkStatesFuture3 = new CompletableFuture<>(); + executor.submit(() -> { + boolean res = stateArray[0] == PulsarServiceState.Failed; + res = res & stateArray[1] == PulsarServiceState.Healthy; + res = res & stateArray[2] == PulsarServiceState.Healthy; + checkStatesFuture3.complete(res); + }); + Assert.assertTrue(checkStatesFuture3.join()); + producer.send("2->1"); + Assert.assertEquals(failover.getCurrentPulsarServiceIndex(), 1); + }); + + // Test recover 1 --> 0. + executor.execute(() -> { + urlArray[0] = url2; + }); + Awaitility.await().atMost(60, TimeUnit.SECONDS).untilAsserted(() -> { + CompletableFuture checkStatesFuture4 = new CompletableFuture<>(); + executor.submit(() -> { + boolean res = stateArray[0] == PulsarServiceState.Healthy; + res = res & stateArray[1] == PulsarServiceState.Healthy; + res = res & stateArray[2] == PulsarServiceState.Healthy; + checkStatesFuture4.complete(res); + }); + Assert.assertTrue(checkStatesFuture4.join()); + producer.send("1->0"); + Assert.assertEquals(failover.getCurrentPulsarServiceIndex(), 0); + }); + + // cleanup. + producer.close(); + client.close(); + dummyServer.close(); + } + + @Override + protected void cleanupPulsarResources() { + // Nothing to do. + } + +} 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 7595e0c8bf53f..68356b1140d99 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 @@ -80,7 +80,7 @@ public abstract class MockedPulsarServiceBaseTest extends TestRetrySupport { // All certificate-authority files are copied from the tests/certificate-authority directory and all share the same // root CA. - protected static String getTlsFileForClient(String name) { + public static String getTlsFileForClient(String name) { return ResourceUtils.getAbsolutePath(String.format("certificate-authority/client-keys/%s.pem", name)); } public final static String CA_CERT_FILE_PATH = diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/NetworkErrorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/NetworkErrorTestBase.java index 36f8cb4761248..742194d9b12a1 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/NetworkErrorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/NetworkErrorTestBase.java @@ -102,7 +102,7 @@ protected void startBrokers() throws Exception { log.info("broker-1: {}, broker-2: {}", broker1.getListenPort(), broker2.getListenPort()); } - protected int getOneFreePort() throws IOException { + public static int getOneFreePort() throws IOException { ServerSocket serverSocket = new ServerSocket(0); int port = serverSocket.getLocalPort(); serverSocket.close(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java index d66e666e3a055..f3076ebdec6c9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OneWayReplicatorTestBase.java @@ -18,6 +18,9 @@ */ package org.apache.pulsar.broker.service; +import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.BROKER_CERT_FILE_PATH; +import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.BROKER_KEY_FILE_PATH; +import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.CA_CERT_FILE_PATH; import static org.apache.pulsar.compaction.Compactor.COMPACTION_SUBSCRIPTION; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; @@ -267,10 +270,18 @@ protected void setConfigDefaults(ServiceConfiguration config, String clusterName config.setReplicatedSubscriptionsSnapshotFrequencyMillis(1000); config.setLoadBalancerSheddingEnabled(false); config.setForceDeleteNamespaceAllowed(true); + config.setTlsCertificateFilePath(BROKER_CERT_FILE_PATH); + config.setTlsKeyFilePath(BROKER_KEY_FILE_PATH); + config.setTlsTrustCertsFilePath(CA_CERT_FILE_PATH); + config.setClusterName(clusterName); + config.setTlsRequireTrustedClientCertOnConnect(false); + Set tlsProtocols = Sets.newConcurrentHashSet(); + tlsProtocols.add("TLSv1.3"); + tlsProtocols.add("TLSv1.2"); + config.setTlsProtocols(tlsProtocols); } - @Override - protected void cleanup() throws Exception { + protected void cleanupPulsarResources() throws Exception { // delete namespaces. waitChangeEventsInit(replicatedNamespace); admin1.namespaces().setNamespaceReplicationClusters(replicatedNamespace, Sets.newHashSet(cluster1)); @@ -283,6 +294,12 @@ protected void cleanup() throws Exception { admin2.namespaces().deleteNamespace(replicatedNamespace, true); admin2.namespaces().deleteNamespace(nonReplicatedNamespace, true); } + } + + @Override + protected void cleanup() throws Exception { + // cleanup pulsar resources. + cleanupPulsarResources(); // shutdown. markCurrentSetupNumberCleaned(); diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ServiceUrlProvider.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ServiceUrlProvider.java index 5cb22276553ab..e8b513b103f65 100644 --- a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ServiceUrlProvider.java +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ServiceUrlProvider.java @@ -56,7 +56,7 @@ public interface ServiceUrlProvider extends AutoCloseable { * */ @Override - default void close() { + default void close() throws Exception { // do nothing } } 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 new file mode 100644 index 0000000000000..1b7013f3018b8 --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/SameAuthParamsLookupAutoClusterFailover.java @@ -0,0 +1,343 @@ +/* + * 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 edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import io.netty.channel.EventLoopGroup; +import io.netty.util.concurrent.ScheduledFuture; +import java.net.InetSocketAddress; +import java.util.Arrays; +import java.util.HashSet; +import java.util.concurrent.TimeUnit; +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.mutable.MutableInt; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.ServiceUrlProvider; +import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.util.FutureUtil; +import org.apache.pulsar.common.util.netty.EventLoopUtil; + +@Slf4j +@SuppressFBWarnings(value = {"EI_EXPOSE_REP2"}) +public class SameAuthParamsLookupAutoClusterFailover implements ServiceUrlProvider { + + private PulsarClientImpl pulsarClient; + private EventLoopGroup executor; + private volatile boolean closed; + private ScheduledFuture scheduledCheckTask; + @Getter + private int failoverThreshold = 5; + @Getter + private int recoverThreshold = 5; + @Getter + private long checkHealthyIntervalMs = 1000; + @Getter + private boolean markTopicNotFoundAsAvailable = true; + @Getter + private String testTopic = "public/default/tp_test"; + + private String[] pulsarServiceUrlArray; + private PulsarServiceState[] pulsarServiceStateArray; + private MutableInt[] checkCounterArray; + @Getter + private volatile int currentPulsarServiceIndex; + + private SameAuthParamsLookupAutoClusterFailover() {} + + @Override + public void initialize(PulsarClient client) { + this.currentPulsarServiceIndex = 0; + this.pulsarClient = (PulsarClientImpl) client; + this.executor = EventLoopUtil.newEventLoopGroup(1, false, + new ExecutorProvider.ExtendedThreadFactory("broker-service-url-check")); + scheduledCheckTask = executor.scheduleAtFixedRate(() -> { + if (closed) { + return; + } + checkPulsarServices(); + int firstHealthyPulsarService = firstHealthyPulsarService(); + if (firstHealthyPulsarService == currentPulsarServiceIndex) { + return; + } + if (firstHealthyPulsarService < 0) { + int failoverTo = findFailoverTo(); + if (failoverTo < 0) { + // No healthy pulsar service to connect. + log.error("Failed to choose a pulsar service to connect, no one pulsar service is healthy. Current" + + " pulsar service: [{}] {}. States: {}, Counters: {}", currentPulsarServiceIndex, + pulsarServiceUrlArray[currentPulsarServiceIndex], Arrays.toString(pulsarServiceStateArray), + Arrays.toString(checkCounterArray)); + } else { + // Failover to low priority pulsar service. + updateServiceUrl(failoverTo); + } + } else { + // Back to high priority pulsar service. + updateServiceUrl(firstHealthyPulsarService); + } + }, checkHealthyIntervalMs, checkHealthyIntervalMs, TimeUnit.MILLISECONDS); + } + + @Override + public String getServiceUrl() { + return pulsarServiceUrlArray[currentPulsarServiceIndex]; + } + + @Override + public void close() throws Exception { + log.info("Closing service url provider. Current pulsar service: [{}] {}", currentPulsarServiceIndex, + pulsarServiceUrlArray[currentPulsarServiceIndex]); + closed = true; + scheduledCheckTask.cancel(false); + executor.shutdownNow(); + } + + private int firstHealthyPulsarService() { + for (int i = 0; i <= currentPulsarServiceIndex; i++) { + if (pulsarServiceStateArray[i] == PulsarServiceState.Healthy + || pulsarServiceStateArray[i] == PulsarServiceState.PreFail) { + return i; + } + } + return -1; + } + + private int findFailoverTo() { + for (int i = currentPulsarServiceIndex + 1; i <= pulsarServiceUrlArray.length; i++) { + if (probeAvailable(i)) { + return i; + } + } + return -1; + } + + private void checkPulsarServices() { + for (int i = 0; i <= currentPulsarServiceIndex; i++) { + if (probeAvailable(i)) { + switch (pulsarServiceStateArray[i]) { + case Healthy: { + break; + } + case PreFail: { + pulsarServiceStateArray[i] = PulsarServiceState.Healthy; + checkCounterArray[i].setValue(0); + break; + } + case Failed: { + pulsarServiceStateArray[i] = PulsarServiceState.PreRecover; + checkCounterArray[i].setValue(1); + break; + } + case PreRecover: { + checkCounterArray[i].setValue(checkCounterArray[i].getValue() + 1); + if (checkCounterArray[i].getValue() >= recoverThreshold) { + pulsarServiceStateArray[i] = PulsarServiceState.Healthy; + checkCounterArray[i].setValue(0); + } + break; + } + } + } else { + switch (pulsarServiceStateArray[i]) { + case Healthy: { + pulsarServiceStateArray[i] = PulsarServiceState.PreFail; + checkCounterArray[i].setValue(1); + break; + } + case PreFail: { + checkCounterArray[i].setValue(checkCounterArray[i].getValue() + 1); + if (checkCounterArray[i].getValue() >= failoverThreshold) { + pulsarServiceStateArray[i] = PulsarServiceState.Failed; + checkCounterArray[i].setValue(0); + } + break; + } + case Failed: { + break; + } + case PreRecover: { + pulsarServiceStateArray[i] = PulsarServiceState.Failed; + checkCounterArray[i].setValue(0); + break; + } + } + } + } + } + + private boolean probeAvailable(int brokerServiceIndex) { + String url = pulsarServiceUrlArray[brokerServiceIndex]; + try { + Pair res = + pulsarClient.getLookup(url).getBroker(TopicName.get(testTopic)).get(3, TimeUnit.SECONDS); + if (log.isDebugEnabled()) { + log.debug("Success to probe available(lookup res: {}), [{}] {}}. States: {}, Counters: {}", + res.toString(), brokerServiceIndex, url, Arrays.toString(pulsarServiceStateArray), + Arrays.toString(checkCounterArray)); + } + return true; + } catch (Exception e) { + Throwable actEx = FutureUtil.unwrapCompletionException(e); + if (actEx instanceof PulsarAdminException.NotFoundException + || actEx instanceof PulsarClientException.NotFoundException + || actEx instanceof PulsarClientException.TopicDoesNotExistException + || actEx instanceof PulsarClientException.LookupException) { + if (markTopicNotFoundAsAvailable) { + if (log.isDebugEnabled()) { + log.debug("Success to probe available(case tenant/namespace/topic not found), [{}] {}." + + " States: {}, Counters: {}", brokerServiceIndex, url, + Arrays.toString(pulsarServiceStateArray), Arrays.toString(checkCounterArray)); + } + return true; + } else { + log.warn("Failed to probe available(error tenant/namespace/topic not found), [{}] {}. States: {}," + + " Counters: {}", brokerServiceIndex, url, Arrays.toString(pulsarServiceStateArray), + Arrays.toString(checkCounterArray)); + return false; + } + } + log.warn("Failed to probe available, [{}] {}. States: {}, Counters: {}", brokerServiceIndex, url, + Arrays.toString(pulsarServiceStateArray), Arrays.toString(checkCounterArray)); + return false; + } + } + + private void updateServiceUrl(int targetIndex) { + String currentUrl = pulsarServiceUrlArray[currentPulsarServiceIndex]; + String targetUrl = pulsarServiceUrlArray[targetIndex]; + String logMsg; + if (targetIndex < currentPulsarServiceIndex) { + logMsg = String.format("Recover to high priority pulsar service [%s] %s --> [%s] %s. States: %s," + + " Counters: %s", currentPulsarServiceIndex, currentUrl, targetIndex, targetUrl, + Arrays.toString(pulsarServiceStateArray), Arrays.toString(checkCounterArray)); + } else { + logMsg = String.format("Failover to low priority pulsar service [%s] %s --> [%s] %s. States: %s," + + " Counters: %s", currentPulsarServiceIndex, currentUrl, targetIndex, targetUrl, + Arrays.toString(pulsarServiceStateArray), Arrays.toString(checkCounterArray)); + } + log.info(logMsg); + try { + pulsarClient.updateServiceUrl(targetUrl); + pulsarClient.reloadLookUp(); + currentPulsarServiceIndex = targetIndex; + } catch (Exception e) { + log.error("Failed to {}", logMsg, e); + } + } + + public enum PulsarServiceState { + Healthy, + PreFail, + Failed, + PreRecover; + } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + + private SameAuthParamsLookupAutoClusterFailover + sameAuthParamsLookupAutoClusterFailover = new SameAuthParamsLookupAutoClusterFailover(); + + public Builder failoverThreshold(int failoverThreshold) { + if (failoverThreshold < 1) { + throw new IllegalArgumentException("failoverThreshold must be larger than 0"); + } + sameAuthParamsLookupAutoClusterFailover.failoverThreshold = failoverThreshold; + return this; + } + + public Builder recoverThreshold(int recoverThreshold) { + if (recoverThreshold < 1) { + throw new IllegalArgumentException("recoverThreshold must be larger than 0"); + } + sameAuthParamsLookupAutoClusterFailover.recoverThreshold = recoverThreshold; + return this; + } + + public Builder checkHealthyIntervalMs(int checkHealthyIntervalMs) { + if (checkHealthyIntervalMs < 1) { + throw new IllegalArgumentException("checkHealthyIntervalMs must be larger than 0"); + } + sameAuthParamsLookupAutoClusterFailover.checkHealthyIntervalMs = checkHealthyIntervalMs; + return this; + } + + public Builder testTopic(String testTopic) { + if (StringUtils.isBlank(testTopic) && TopicName.get(testTopic) != null) { + throw new IllegalArgumentException("testTopic can not be blank"); + } + sameAuthParamsLookupAutoClusterFailover.testTopic = testTopic; + return this; + } + + public Builder markTopicNotFoundAsAvailable(boolean markTopicNotFoundAsAvailable) { + sameAuthParamsLookupAutoClusterFailover.markTopicNotFoundAsAvailable = markTopicNotFoundAsAvailable; + return this; + } + + public Builder pulsarServiceUrlArray(String[] pulsarServiceUrlArray) { + if (pulsarServiceUrlArray == null || pulsarServiceUrlArray.length == 0) { + throw new IllegalArgumentException("pulsarServiceUrlArray can not be empty"); + } + sameAuthParamsLookupAutoClusterFailover.pulsarServiceUrlArray = pulsarServiceUrlArray; + int pulsarServiceLen = pulsarServiceUrlArray.length; + HashSet uniqueChecker = new HashSet<>(); + for (int i = 0; i < pulsarServiceLen; i++) { + String pulsarService = pulsarServiceUrlArray[i]; + if (StringUtils.isBlank(pulsarService)) { + throw new IllegalArgumentException("pulsarServiceUrlArray contains a blank value at index " + i); + } + if (pulsarService.startsWith("http") || pulsarService.startsWith("HTTP")) { + throw new IllegalArgumentException("SameAuthParamsLookupAutoClusterFailover does not support HTTP" + + " protocol pulsar service url so far."); + } + if (!uniqueChecker.add(pulsarService)) { + throw new IllegalArgumentException("pulsarServiceUrlArray contains duplicated value " + + pulsarServiceUrlArray[i]); + } + } + return this; + } + + public SameAuthParamsLookupAutoClusterFailover build() { + String[] pulsarServiceUrlArray = sameAuthParamsLookupAutoClusterFailover.pulsarServiceUrlArray; + if (pulsarServiceUrlArray == null) { + throw new IllegalArgumentException("pulsarServiceUrlArray can not be empty"); + } + int pulsarServiceLen = pulsarServiceUrlArray.length; + sameAuthParamsLookupAutoClusterFailover.pulsarServiceStateArray = new PulsarServiceState[pulsarServiceLen]; + sameAuthParamsLookupAutoClusterFailover.checkCounterArray = new MutableInt[pulsarServiceLen]; + for (int i = 0; i < pulsarServiceLen; i++) { + sameAuthParamsLookupAutoClusterFailover.pulsarServiceStateArray[i] = PulsarServiceState.Healthy; + sameAuthParamsLookupAutoClusterFailover.checkCounterArray[i] = new MutableInt(0); + } + return sameAuthParamsLookupAutoClusterFailover; + } + } +} + diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java index 63fbb239439bd..5dd1e6f27c61e 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/AutoClusterFailoverTest.java @@ -23,7 +23,6 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; -import java.io.IOException; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -31,7 +30,6 @@ import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationFactory; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.ServiceUrlProvider; import org.apache.pulsar.client.impl.conf.ClientConfigurationData; import org.awaitility.Awaitility; @@ -42,7 +40,7 @@ @Slf4j public class AutoClusterFailoverTest { @Test - public void testBuildAutoClusterFailoverInstance() throws PulsarClientException { + public void testBuildAutoClusterFailoverInstance() throws Exception { String primary = "pulsar://localhost:6650"; String secondary = "pulsar://localhost:6651"; long failoverDelay = 30; @@ -103,7 +101,7 @@ public void testBuildAutoClusterFailoverInstance() throws PulsarClientException } @Test - public void testInitialize() { + public void testInitialize() throws Exception { String primary = "pulsar://localhost:6650"; String secondary = "pulsar://localhost:6651"; long failoverDelay = 10; @@ -147,7 +145,7 @@ public void testInitialize() { } @Test - public void testAutoClusterFailoverSwitchWithoutAuthentication() { + public void testAutoClusterFailoverSwitchWithoutAuthentication() throws Exception { String primary = "pulsar://localhost:6650"; String secondary = "pulsar://localhost:6651"; long failoverDelay = 1; @@ -182,7 +180,7 @@ public void testAutoClusterFailoverSwitchWithoutAuthentication() { } @Test - public void testAutoClusterFailoverSwitchWithAuthentication() throws IOException { + public void testAutoClusterFailoverSwitchWithAuthentication() throws Exception { String primary = "pulsar+ssl://localhost:6651"; String secondary = "pulsar+ssl://localhost:6661"; long failoverDelay = 1; @@ -245,7 +243,7 @@ public void testAutoClusterFailoverSwitchWithAuthentication() throws IOException } @Test - public void testAutoClusterFailoverSwitchTlsTrustStore() throws IOException { + public void testAutoClusterFailoverSwitchTlsTrustStore() throws Exception { String primary = "pulsar+ssl://localhost:6651"; String secondary = "pulsar+ssl://localhost:6661"; long failoverDelay = 1; diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java index 227e0db10b724..c24eefef3970f 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/ControlledClusterFailoverTest.java @@ -18,7 +18,6 @@ */ package org.apache.pulsar.client.impl; -import java.io.IOException; import java.util.HashMap; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -36,7 +35,7 @@ @Test(groups = "broker-impl") public class ControlledClusterFailoverTest { @Test - public void testBuildControlledClusterFailoverInstance() throws IOException { + public void testBuildControlledClusterFailoverInstance() throws Exception { String defaultServiceUrl = "pulsar://localhost:6650"; String urlProvider = "http://localhost:8080/test"; String keyA = "key-a"; @@ -65,7 +64,7 @@ public void testBuildControlledClusterFailoverInstance() throws IOException { } @Test - public void testControlledClusterFailoverSwitch() throws IOException { + public void testControlledClusterFailoverSwitch() throws Exception { String defaultServiceUrl = "pulsar+ssl://localhost:6651"; String backupServiceUrl = "pulsar+ssl://localhost:6661"; String urlProvider = "http://localhost:8080"; From b04b9050666206a09d5cf117e4d3dbac09ba33f4 Mon Sep 17 00:00:00 2001 From: Yuri Mizushima Date: Wed, 14 Aug 2024 11:09:50 +0900 Subject: [PATCH 666/699] [improve][proxy] Reuse authentication instance in pulsar-proxy (#23113) (cherry picked from commit 3e461c004ea229ef9b526a51fd0ed91e8157e873) (cherry picked from commit b805a4ad45427a143d54013ad92035be06d8c641) --- .../ProxySaslAuthenticationTest.java | 6 +- .../proxy/server/AdminProxyHandler.java | 23 +---- .../proxy/server/DirectProxyHandler.java | 4 +- .../pulsar/proxy/server/ProxyService.java | 12 +-- .../proxy/server/ProxyServiceStarter.java | 46 ++++++++-- .../SimpleProxyExtensionTestBase.java | 12 ++- .../AdminProxyHandlerKeystoreTLSTest.java | 25 +++-- .../proxy/server/AdminProxyHandlerTest.java | 3 +- .../server/AuthedAdminProxyHandlerTest.java | 15 ++- .../server/FunctionWorkerRoutingTest.java | 10 +- ...nvalidProxyConfigForAuthorizationTest.java | 3 +- .../server/ProxyAdditionalServletTest.java | 55 ++++++----- ...roxyAuthenticatedProducerConsumerTest.java | 11 ++- .../proxy/server/ProxyAuthenticationTest.java | 7 +- .../server/ProxyConnectionThrottlingTest.java | 11 ++- .../server/ProxyDisableZeroCopyTest.java | 25 +---- .../ProxyEnableHAProxyProtocolTest.java | 12 ++- .../server/ProxyForwardAuthDataTest.java | 10 +- .../proxy/server/ProxyIsAHttpProxyTest.java | 60 +++++++++--- .../server/ProxyKeyStoreTlsTestWithAuth.java | 12 ++- .../ProxyKeyStoreTlsTestWithoutAuth.java | 12 ++- .../server/ProxyKeyStoreTlsTransportTest.java | 12 ++- .../server/ProxyLookupThrottlingTest.java | 14 ++- .../proxy/server/ProxyMutualTlsTest.java | 12 ++- .../pulsar/proxy/server/ProxyParserTest.java | 11 ++- .../server/ProxyPrometheusMetricsTest.java | 15 ++- .../proxy/server/ProxyRefreshAuthTest.java | 12 ++- .../server/ProxyRolesEnforcementTest.java | 9 +- .../proxy/server/ProxyServiceStarterTest.java | 91 +++++++++++++++++++ .../pulsar/proxy/server/ProxyStatsTest.java | 14 ++- .../server/ProxyStuckConnectionTest.java | 12 ++- .../apache/pulsar/proxy/server/ProxyTest.java | 28 ++++-- .../pulsar/proxy/server/ProxyTlsTest.java | 12 ++- .../proxy/server/ProxyTlsTestWithAuth.java | 12 ++- .../server/ProxyWithAuthorizationNegTest.java | 10 +- .../server/ProxyWithAuthorizationTest.java | 21 ++++- .../server/ProxyWithJwtAuthorizationTest.java | 28 ++++-- .../ProxyWithoutServiceDiscoveryTest.java | 11 ++- .../SuperUserAuthedAdminProxyHandlerTest.java | 15 ++- .../URLRegexLookupProxyHandlerTest.java | 14 ++- .../server/UnauthedAdminProxyHandlerTest.java | 16 +++- 41 files changed, 581 insertions(+), 162 deletions(-) diff --git a/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/ProxySaslAuthenticationTest.java b/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/ProxySaslAuthenticationTest.java index f0e45aa734afb..c4ccc31142c93 100644 --- a/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/ProxySaslAuthenticationTest.java +++ b/pulsar-broker-auth-sasl/src/test/java/org/apache/pulsar/broker/authentication/ProxySaslAuthenticationTest.java @@ -258,7 +258,11 @@ void testAuthentication() throws Exception { proxyConfig.setForwardAuthorizationCredentials(true); AuthenticationService authenticationService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); - ProxyService proxyService = new ProxyService(proxyConfig, authenticationService); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + ProxyService proxyService = new ProxyService(proxyConfig, authenticationService, proxyClientAuthentication); proxyService.start(); final String proxyServiceUrl = "pulsar://localhost:" + proxyService.getListenPort().get(); 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 caaa99c5d40cc..0108b770249a0 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 @@ -29,7 +29,6 @@ import java.util.Collections; import java.util.HashSet; import java.util.Iterator; -import java.util.Objects; import java.util.Set; import java.util.concurrent.Executor; import javax.net.ssl.SSLContext; @@ -40,7 +39,6 @@ import org.apache.pulsar.broker.web.AuthenticationFilter; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationDataProvider; -import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.KeyStoreParams; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.common.util.SecurityUtility; @@ -87,12 +85,15 @@ class AdminProxyHandler extends ProxyServlet { private final ProxyConfiguration config; private final BrokerDiscoveryProvider discoveryProvider; + private final Authentication proxyClientAuthentication; private final String brokerWebServiceUrl; private final String functionWorkerWebServiceUrl; - AdminProxyHandler(ProxyConfiguration config, BrokerDiscoveryProvider discoveryProvider) { + AdminProxyHandler(ProxyConfiguration config, BrokerDiscoveryProvider discoveryProvider, + Authentication proxyClientAuthentication) { this.config = config; this.discoveryProvider = discoveryProvider; + this.proxyClientAuthentication = proxyClientAuthentication; this.brokerWebServiceUrl = config.isTlsEnabledWithBroker() ? config.getBrokerWebServiceURLTLS() : config.getBrokerWebServiceURL(); this.functionWorkerWebServiceUrl = config.isTlsEnabledWithBroker() ? config.getFunctionWorkerWebServiceURLTLS() @@ -256,22 +257,13 @@ protected ContentProvider proxyRequestContent(HttpServletRequest request, @Override protected HttpClient newHttpClient() { try { - Authentication auth = AuthenticationFactory.create( - config.getBrokerClientAuthenticationPlugin(), - config.getBrokerClientAuthenticationParameters() - ); - - Objects.requireNonNull(auth, "No supported auth found for proxy"); - - auth.start(); - if (config.isTlsEnabledWithBroker()) { try { X509Certificate[] trustCertificates = SecurityUtility .loadCertificatesFromPemFile(config.getBrokerClientTrustCertsFilePath()); SSLContext sslCtx; - AuthenticationDataProvider authData = auth.getAuthData(); + AuthenticationDataProvider authData = proxyClientAuthentication.getAuthData(); if (config.isBrokerClientTlsEnabledWithKeyStore()) { KeyStoreParams params = authData.hasDataForTls() ? authData.getTlsKeyStoreParams() : null; sslCtx = KeyStoreSSLContext.createClientSslContext( @@ -311,11 +303,6 @@ protected HttpClient newHttpClient() { return new JettyHttpClient(contextFactory); } catch (Exception e) { LOG.error("new jetty http client exception ", e); - try { - auth.close(); - } catch (IOException ioe) { - LOG.error("Failed to close the authentication service", ioe); - } throw new PulsarClientException.InvalidConfigurationException(e.getMessage()); } } 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 d63b04b6734de..4678db82c6e55 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 @@ -52,7 +52,6 @@ import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationDataProvider; -import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.api.AuthData; @@ -114,8 +113,7 @@ public DirectProxyHandler(ProxyService service, ProxyConnection proxyConnection) if (!isEmpty(config.getBrokerClientAuthenticationPlugin())) { try { - authData = AuthenticationFactory.create(config.getBrokerClientAuthenticationPlugin(), - config.getBrokerClientAuthenticationParameters()).getAuthData(); + authData = authentication.getAuthData(); } catch (PulsarClientException e) { throw new RuntimeException(e); } 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 a3917f2bc6708..5446930a5a348 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 @@ -63,8 +63,6 @@ import org.apache.pulsar.broker.stats.prometheus.PrometheusRawMetricsProvider; import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServlets; import org.apache.pulsar.client.api.Authentication; -import org.apache.pulsar.client.api.AuthenticationFactory; -import org.apache.pulsar.client.impl.auth.AuthenticationDisabled; import org.apache.pulsar.common.allocator.PulsarByteBufAllocator; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.util.netty.DnsResolverUtil; @@ -152,7 +150,8 @@ public class ProxyService implements Closeable { private final ConnectionController connectionController; public ProxyService(ProxyConfiguration proxyConfig, - AuthenticationService authenticationService) throws Exception { + AuthenticationService authenticationService, + Authentication proxyClientAuthentication) throws Exception { requireNonNull(proxyConfig); this.proxyConfig = proxyConfig; this.clientCnxs = Sets.newConcurrentHashSet(); @@ -201,12 +200,7 @@ public ProxyService(ProxyConfiguration proxyConfig, }); }, 60, TimeUnit.SECONDS); this.proxyAdditionalServlets = AdditionalServlets.load(proxyConfig); - if (proxyConfig.getBrokerClientAuthenticationPlugin() != null) { - proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), - proxyConfig.getBrokerClientAuthenticationParameters()); - } else { - proxyClientAuthentication = AuthenticationDisabled.INSTANCE; - } + this.proxyClientAuthentication = proxyClientAuthentication; this.connectionController = new ConnectionController.DefaultConnectionController( proxyConfig.getMaxConcurrentInboundConnections(), proxyConfig.getMaxConcurrentInboundConnectionsPerIp()); diff --git a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java index e0c67c151bc55..508328878ebe1 100644 --- a/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java +++ b/pulsar-proxy/src/main/java/org/apache/pulsar/proxy/server/ProxyServiceStarter.java @@ -31,11 +31,13 @@ import io.prometheus.client.Gauge; import io.prometheus.client.Gauge.Child; import io.prometheus.client.hotspot.DefaultExports; +import java.io.IOException; import java.text.DateFormat; import java.text.SimpleDateFormat; import java.util.Collection; import java.util.Collections; import java.util.Date; +import java.util.Objects; import java.util.function.Consumer; import lombok.Getter; import org.apache.logging.log4j.LogManager; @@ -45,6 +47,10 @@ import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsServlet; import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServletWithClassLoader; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.impl.auth.AuthenticationDisabled; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.configuration.VipStatus; import org.apache.pulsar.common.policies.data.ClusterData; @@ -99,6 +105,9 @@ public class ProxyServiceStarter { private ProxyConfiguration config; + @Getter + private Authentication proxyClientAuthentication; + @Getter private ProxyService proxyService; @@ -239,8 +248,27 @@ public static void main(String[] args) throws Exception { public void start() throws Exception { AuthenticationService authenticationService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(config)); + + if (config.getBrokerClientAuthenticationPlugin() != null) { + proxyClientAuthentication = AuthenticationFactory.create(config.getBrokerClientAuthenticationPlugin(), + config.getBrokerClientAuthenticationParameters()); + Objects.requireNonNull(proxyClientAuthentication, "No supported auth found for proxy"); + try { + proxyClientAuthentication.start(); + } catch (Exception e) { + try { + proxyClientAuthentication.close(); + } catch (IOException ioe) { + log.error("Failed to close the authentication service", ioe); + } + throw new PulsarClientException.InvalidConfigurationException(e.getMessage()); + } + } else { + proxyClientAuthentication = AuthenticationDisabled.INSTANCE; + } + // create proxy service - proxyService = new ProxyService(config, authenticationService); + proxyService = new ProxyService(config, authenticationService, proxyClientAuthentication); // create a web-service server = new WebServer(config, authenticationService); @@ -287,7 +315,8 @@ public double get() { metricsInitialized = true; } - addWebServerHandlers(server, config, proxyService, proxyService.getDiscoveryProvider()); + addWebServerHandlers(server, config, proxyService, proxyService.getDiscoveryProvider(), + proxyClientAuthentication); // start web-service server.start(); @@ -301,6 +330,9 @@ public void close() { if (server != null) { server.stop(); } + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } catch (Exception e) { log.warn("server couldn't stop gracefully {}", e.getMessage(), e); } finally { @@ -311,9 +343,10 @@ public void close() { } public static void addWebServerHandlers(WebServer server, - ProxyConfiguration config, - ProxyService service, - BrokerDiscoveryProvider discoveryProvider) throws Exception { + ProxyConfiguration config, + ProxyService service, + BrokerDiscoveryProvider discoveryProvider, + Authentication proxyClientAuthentication) throws Exception { // We can make 'status.html' publicly accessible without authentication since // it does not contain any sensitive data. server.addRestResource("/", VipStatus.ATTRIBUTE_STATUS_FILE_PATH, config.getStatusFilePath(), @@ -330,7 +363,8 @@ public static void addWebServerHandlers(WebServer server, } } - AdminProxyHandler adminProxyHandler = new AdminProxyHandler(config, discoveryProvider); + AdminProxyHandler adminProxyHandler = new AdminProxyHandler(config, discoveryProvider, + proxyClientAuthentication); ServletHolder servletHolder = new ServletHolder(adminProxyHandler); server.addServlet("/admin", servletHolder); server.addServlet("/lookup", servletHolder); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/extensions/SimpleProxyExtensionTestBase.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/extensions/SimpleProxyExtensionTestBase.java index 79662097c3b2f..4f5436dccd61a 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/extensions/SimpleProxyExtensionTestBase.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/extensions/SimpleProxyExtensionTestBase.java @@ -26,6 +26,8 @@ import org.apache.commons.io.IOUtils; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; +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.PortManager; import org.apache.pulsar.metadata.impl.ZKMetadataStore; @@ -121,6 +123,7 @@ public void close() { private ProxyService proxyService; private boolean useSeparateThreadPoolForProxyExtensions; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; public SimpleProxyExtensionTestBase(boolean useSeparateThreadPoolForProxyExtensions) { this.useSeparateThreadPoolForProxyExtensions = useSeparateThreadPoolForProxyExtensions; @@ -141,8 +144,12 @@ protected void setup() throws Exception { proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(new ZKMetadataStore(mockZooKeeper)).when(proxyService).createLocalMetadataStore(); doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(proxyService).createConfigurationMetadataStore(); @@ -172,6 +179,9 @@ public void testBootstrapProtocolHandler() throws Exception { protected void cleanup() throws Exception { super.internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } if (tempDirectory != null) { FileUtils.deleteDirectory(tempDirectory); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java index d6796b7eaa6d2..b4be7bebb8329 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerKeystoreTLSTest.java @@ -18,11 +18,18 @@ */ package org.apache.pulsar.proxy.server; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; +import java.util.HashSet; +import java.util.Optional; +import java.util.Set; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationProviderTls; import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.impl.auth.AuthenticationKeyStoreTls; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.policies.data.ClusterData; @@ -34,18 +41,13 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; -import java.util.HashSet; -import java.util.Optional; -import java.util.Set; - -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.spy; - public class AdminProxyHandlerKeystoreTLSTest extends MockedPulsarServiceBaseTest { private final ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; + private WebServer webServer; private BrokerDiscoveryProvider discoveryProvider; @@ -103,12 +105,16 @@ protected void setup() throws Exception { resource = new PulsarResources(new ZKMetadataStore(mockZooKeeper), new ZKMetadataStore(mockZooKeeperGlobal)); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + webServer = new WebServer(proxyConfig, new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig))); discoveryProvider = spy(new BrokerDiscoveryProvider(proxyConfig, resource)); LoadManagerReport report = new LoadReport(brokerUrl.toString(), brokerUrlTls.toString(), null, null); doReturn(report).when(discoveryProvider).nextBroker(); - ServletHolder servletHolder = new ServletHolder(new AdminProxyHandler(proxyConfig, discoveryProvider)); + ServletHolder servletHolder = new ServletHolder(new AdminProxyHandler(proxyConfig, discoveryProvider, proxyClientAuthentication)); webServer.addServlet("/admin", servletHolder); webServer.addServlet("/lookup", servletHolder); webServer.start(); @@ -118,6 +124,9 @@ protected void setup() throws Exception { @Override protected void cleanup() throws Exception { webServer.stop(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } super.internalCleanup(); } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerTest.java index becebe0059e56..4f925618e8a79 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AdminProxyHandlerTest.java @@ -32,6 +32,7 @@ import javax.servlet.ServletException; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; +import org.apache.pulsar.client.api.Authentication; import org.eclipse.jetty.client.HttpClient; import org.eclipse.jetty.client.api.Request; import org.testng.Assert; @@ -46,7 +47,7 @@ public void setupMocks() throws ServletException { // given HttpClient httpClient = mock(HttpClient.class); adminProxyHandler = new AdminProxyHandler(mock(ProxyConfiguration.class), - mock(BrokerDiscoveryProvider.class)) { + mock(BrokerDiscoveryProvider.class), mock(Authentication.class)) { @Override protected HttpClient createHttpClient() throws ServletException { return httpClient; diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AuthedAdminProxyHandlerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AuthedAdminProxyHandlerTest.java index af70276aed95e..840bfe17fe4d8 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AuthedAdminProxyHandlerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/AuthedAdminProxyHandlerTest.java @@ -20,18 +20,17 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; - import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; - import java.util.Optional; - import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationProviderTls; import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.policies.data.ClusterData; @@ -51,6 +50,7 @@ public class AuthedAdminProxyHandlerTest extends MockedPulsarServiceBaseTest { private static final Logger LOG = LoggerFactory.getLogger(AuthedAdminProxyHandlerTest.class); private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; private WebServer webServer; private BrokerDiscoveryProvider discoveryProvider; private PulsarResources resource; @@ -100,13 +100,17 @@ protected void setup() throws Exception { resource = new PulsarResources(new ZKMetadataStore(mockZooKeeper), new ZKMetadataStore(mockZooKeeperGlobal)); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + webServer = new WebServer(proxyConfig, new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig))); discoveryProvider = spy(new BrokerDiscoveryProvider(proxyConfig, resource)); LoadManagerReport report = new LoadReport(brokerUrl.toString(), brokerUrlTls.toString(), null, null); doReturn(report).when(discoveryProvider).nextBroker(); - ServletHolder servletHolder = new ServletHolder(new AdminProxyHandler(proxyConfig, discoveryProvider)); + ServletHolder servletHolder = new ServletHolder(new AdminProxyHandler(proxyConfig, discoveryProvider, proxyClientAuthentication)); webServer.addServlet("/admin", servletHolder); webServer.addServlet("/lookup", servletHolder); @@ -118,6 +122,9 @@ protected void setup() throws Exception { @Override protected void cleanup() throws Exception { webServer.stop(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } super.internalCleanup(); } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/FunctionWorkerRoutingTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/FunctionWorkerRoutingTest.java index db5e9e12bd2db..a07a0f082d39a 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/FunctionWorkerRoutingTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/FunctionWorkerRoutingTest.java @@ -18,6 +18,9 @@ */ package org.apache.pulsar.proxy.server; +import lombok.Cleanup; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.testng.Assert; import org.testng.annotations.Test; @@ -37,8 +40,13 @@ public void testFunctionWorkerRedirect() throws Exception { proxyConfig.setBrokerWebServiceURL(brokerUrl); proxyConfig.setFunctionWorkerWebServiceURL(functionWorkerUrl); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + BrokerDiscoveryProvider discoveryProvider = mock(BrokerDiscoveryProvider.class); - AdminProxyHandler handler = new AdminProxyHandler(proxyConfig, discoveryProvider); + AdminProxyHandler handler = new AdminProxyHandler(proxyConfig, discoveryProvider, proxyClientAuthentication); String funcUrl = handler.rewriteTarget(buildRequest("/admin/v3/functions/test/test")); Assert.assertEquals(funcUrl, String.format("%s/admin/v3/functions/%s/%s", diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/InvalidProxyConfigForAuthorizationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/InvalidProxyConfigForAuthorizationTest.java index c29bfaa964812..b7ef0855e383c 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/InvalidProxyConfigForAuthorizationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/InvalidProxyConfigForAuthorizationTest.java @@ -22,6 +22,7 @@ import static org.testng.Assert.fail; import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.client.api.Authentication; import org.mockito.Mockito; import org.testng.annotations.Test; @@ -33,7 +34,7 @@ void startupShouldFailWhenAuthorizationIsEnabledWithoutAuthentication() throws E proxyConfiguration.setAuthorizationEnabled(true); proxyConfiguration.setAuthenticationEnabled(false); try (ProxyService proxyService = new ProxyService(proxyConfiguration, - Mockito.mock(AuthenticationService.class))) { + Mockito.mock(AuthenticationService.class), Mockito.mock(Authentication.class))) { proxyService.start(); fail("An exception should have been thrown"); } catch (Exception e) { 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 17cd3c33e799d..67cf5aca911c9 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 @@ -18,18 +18,36 @@ */ package org.apache.pulsar.proxy.server; +import static org.mockito.Mockito.doReturn; +import static org.testng.Assert.assertEquals; import com.google.common.collect.Sets; +import java.io.IOException; +import java.net.URL; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import javax.servlet.Servlet; +import javax.servlet.ServletConfig; +import javax.servlet.ServletException; +import javax.servlet.ServletOutputStream; +import javax.servlet.ServletRequest; +import javax.servlet.ServletResponse; import lombok.extern.slf4j.Slf4j; import okhttp3.OkHttpClient; import okhttp3.Response; import org.apache.commons.lang3.RandomUtils; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; -import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; -import org.apache.pulsar.metadata.impl.ZKMetadataStore; +import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServlet; import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServletWithClassLoader; import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServlets; -import org.apache.pulsar.broker.web.plugin.servlet.AdditionalServlet; +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.metadata.impl.ZKMetadataStore; import org.eclipse.jetty.server.Request; import org.eclipse.jetty.servlet.ServletHolder; import org.mockito.Mockito; @@ -38,24 +56,6 @@ import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import javax.servlet.Servlet; -import javax.servlet.ServletConfig; -import javax.servlet.ServletException; -import javax.servlet.ServletOutputStream; -import javax.servlet.ServletRequest; -import javax.servlet.ServletResponse; -import java.io.IOException; -import java.net.URL; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import java.util.Properties; - -import static org.mockito.Mockito.doReturn; -import static org.testng.Assert.assertEquals; - @Slf4j public class ProxyAdditionalServletTest extends MockedPulsarServiceBaseTest { @@ -65,6 +65,7 @@ public class ProxyAdditionalServletTest extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private WebServer proxyWebServer; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeClass @@ -82,8 +83,13 @@ protected void setup() throws Exception { // this is for nar package test // addServletNar(); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, - new AuthenticationService(PulsarConfigurationLoader.convertFrom(proxyConfig)))); + new AuthenticationService(PulsarConfigurationLoader.convertFrom(proxyConfig)), + proxyClientAuthentication)); doReturn(new ZKMetadataStore(mockZooKeeper)).when(proxyService).createLocalMetadataStore(); doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(proxyService).createConfigurationMetadataStore(); @@ -97,7 +103,7 @@ protected void setup() throws Exception { mockAdditionalServlet(); proxyWebServer = new WebServer(proxyConfig, authService); - ProxyServiceStarter.addWebServerHandlers(proxyWebServer, proxyConfig, proxyService, null); + ProxyServiceStarter.addWebServerHandlers(proxyWebServer, proxyConfig, proxyService, null, proxyClientAuthentication); proxyWebServer.start(); } @@ -177,6 +183,9 @@ protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticatedProducerConsumerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticatedProducerConsumerTest.java index bfe86f86976ee..ab97bf052013b 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticatedProducerConsumerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticatedProducerConsumerTest.java @@ -35,6 +35,7 @@ import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Producer; @@ -74,6 +75,7 @@ public class ProxyAuthenticatedProducerConsumerTest extends ProducerConsumerBase private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; private final String configClusterName = "test"; @BeforeMethod @@ -138,8 +140,12 @@ protected void setup() throws Exception { proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(new ZKMetadataStore(mockZooKeeper)).when(proxyService).createLocalMetadataStore(); doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(proxyService).createConfigurationMetadataStore(); proxyService.start(); @@ -150,6 +156,9 @@ protected void setup() throws Exception { protected void cleanup() throws Exception { super.internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } /** diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticationTest.java index 9c8e5197adf1a..3207c2c3d6a7e 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyAuthenticationTest.java @@ -43,6 +43,7 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationDataProvider; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; @@ -233,7 +234,11 @@ void testAuthentication() throws Exception { AuthenticationService authenticationService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); @Cleanup - ProxyService proxyService = new ProxyService(proxyConfig, authenticationService); + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + @Cleanup + ProxyService proxyService = new ProxyService(proxyConfig, authenticationService, proxyClientAuthentication); proxyService.start(); final String proxyServiceUrl = proxyService.getServiceUrl(); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyConnectionThrottlingTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyConnectionThrottlingTest.java index 336f11ae19da6..5d950d847eff0 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyConnectionThrottlingTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyConnectionThrottlingTest.java @@ -27,6 +27,8 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.limiter.ConnectionController; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; @@ -46,6 +48,7 @@ public class ProxyConnectionThrottlingTest extends MockedPulsarServiceBaseTest { private final int NUM_CONCURRENT_INBOUND_CONNECTION = 4; private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeClass @@ -59,8 +62,11 @@ protected void setup() throws Exception { proxyConfig.setMaxConcurrentLookupRequests(NUM_CONCURRENT_LOOKUP); proxyConfig.setMaxConcurrentInboundConnections(NUM_CONCURRENT_INBOUND_CONNECTION); proxyConfig.setMaxConcurrentInboundConnectionsPerIp(NUM_CONCURRENT_INBOUND_CONNECTION); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(new ZKMetadataStore(mockZooKeeper)).when(proxyService).createLocalMetadataStore(); doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(proxyService).createConfigurationMetadataStore(); @@ -72,6 +78,9 @@ protected void setup() throws Exception { protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyDisableZeroCopyTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyDisableZeroCopyTest.java index 3aa71413d540b..6a3992c550fd3 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyDisableZeroCopyTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyDisableZeroCopyTest.java @@ -18,32 +18,11 @@ */ package org.apache.pulsar.proxy.server; -import static org.mockito.Mockito.doReturn; -import java.util.Optional; -import org.apache.pulsar.broker.authentication.AuthenticationService; -import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; -import org.apache.pulsar.metadata.impl.ZKMetadataStore; -import org.mockito.Mockito; -import org.testng.annotations.BeforeClass; - public class ProxyDisableZeroCopyTest extends ProxyTest { @Override - @BeforeClass - protected void setup() throws Exception { - internalSetup(); - - proxyConfig.setServicePort(Optional.ofNullable(0)); - proxyConfig.setBrokerProxyAllowedTargetPorts("*"); - proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); - proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + protected void initializeProxyConfig() throws Exception { + super.initializeProxyConfig(); proxyConfig.setProxyZeroCopyModeEnabled(false); - - proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); - doReturn(new ZKMetadataStore(mockZooKeeper)).when(proxyService).createLocalMetadataStore(); - doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(proxyService).createConfigurationMetadataStore(); - - proxyService.start(); } } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyEnableHAProxyProtocolTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyEnableHAProxyProtocolTest.java index 8b3092c6f5170..77212c9535b1b 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyEnableHAProxyProtocolTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyEnableHAProxyProtocolTest.java @@ -22,6 +22,8 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.impl.ConsumerImpl; @@ -48,6 +50,7 @@ public class ProxyEnableHAProxyProtocolTest extends MockedPulsarServiceBaseTest private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeClass @@ -61,8 +64,12 @@ protected void setup() throws Exception { proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); proxyConfig.setHaProxyProtocolEnabled(true); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(new ZKMetadataStore(mockZooKeeper)).when(proxyService).createLocalMetadataStore(); doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(proxyService).createConfigurationMetadataStore(); @@ -75,6 +82,9 @@ protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java index b7cfb87474707..8e3a2fdf264ef 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyForwardAuthDataTest.java @@ -30,6 +30,8 @@ import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; @@ -116,7 +118,11 @@ public void testForwardAuthData() throws Exception { AuthenticationService authenticationService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); - try (ProxyService proxyService = new ProxyService(proxyConfig, authenticationService)) { + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + try (ProxyService proxyService = new ProxyService(proxyConfig, authenticationService, proxyClientAuthentication)) { proxyService.start(); try (PulsarClient proxyClient = createPulsarClient(proxyService.getServiceUrl(), clientAuthParams)) { proxyClient.newConsumer().topic(topicName).subscriptionName(subscriptionName).subscribe(); @@ -132,7 +138,7 @@ public void testForwardAuthData() throws Exception { PulsarConfigurationLoader.convertFrom(proxyConfig)); @Cleanup - ProxyService proxyService = new ProxyService(proxyConfig, authenticationService); + ProxyService proxyService = new ProxyService(proxyConfig, authenticationService, proxyClientAuthentication); proxyService.start(); @Cleanup diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyIsAHttpProxyTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyIsAHttpProxyTest.java index 246dd9f85e319..be51bd18323de 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyIsAHttpProxyTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyIsAHttpProxyTest.java @@ -36,10 +36,12 @@ import javax.ws.rs.client.Client; import javax.ws.rs.client.ClientBuilder; import javax.ws.rs.core.Response; - +import lombok.Cleanup; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.resources.PulsarResources; +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.metadata.impl.ZKMetadataStore; import org.eclipse.jetty.client.HttpClient; @@ -201,10 +203,14 @@ public void testSingleRedirect() throws Exception { ProxyConfiguration proxyConfig = PulsarConfigurationLoader.create(props, ProxyConfiguration.class); AuthenticationService authService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); WebServer webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, null, - new BrokerDiscoveryProvider(proxyConfig, resource)); + new BrokerDiscoveryProvider(proxyConfig, resource), proxyClientAuthentication); webServer.start(); try { Response r = client.target(webServer.getServiceUri()).path("/ui/foobar").request().get(); @@ -230,10 +236,14 @@ public void testMultipleRedirect() throws Exception { ProxyConfiguration proxyConfig = PulsarConfigurationLoader.create(props, ProxyConfiguration.class); AuthenticationService authService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); WebServer webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, null, - new BrokerDiscoveryProvider(proxyConfig, resource)); + new BrokerDiscoveryProvider(proxyConfig, resource), proxyClientAuthentication); webServer.start(); try { Response r1 = client.target(webServer.getServiceUri()).path("/server1/foobar").request().get(); @@ -261,10 +271,14 @@ public void testTryingToUseExistingPath() throws Exception { ProxyConfiguration proxyConfig = PulsarConfigurationLoader.create(props, ProxyConfiguration.class); AuthenticationService authService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); WebServer webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, null, - new BrokerDiscoveryProvider(proxyConfig, resource)); + new BrokerDiscoveryProvider(proxyConfig, resource), proxyClientAuthentication); } @@ -280,10 +294,14 @@ public void testLongPathInProxyTo() throws Exception { ProxyConfiguration proxyConfig = PulsarConfigurationLoader.create(props, ProxyConfiguration.class); AuthenticationService authService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); WebServer webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, null, - new BrokerDiscoveryProvider(proxyConfig, resource)); + new BrokerDiscoveryProvider(proxyConfig, resource), proxyClientAuthentication); webServer.start(); try { Response r = client.target(webServer.getServiceUri()).path("/ui/foobar").request().get(); @@ -307,10 +325,14 @@ public void testProxyToEndsInSlash() throws Exception { ProxyConfiguration proxyConfig = PulsarConfigurationLoader.create(props, ProxyConfiguration.class); AuthenticationService authService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); WebServer webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, null, - new BrokerDiscoveryProvider(proxyConfig, resource)); + new BrokerDiscoveryProvider(proxyConfig, resource), proxyClientAuthentication); webServer.start(); try { Response r = client.target(webServer.getServiceUri()).path("/ui/foobar").request().get(); @@ -333,10 +355,14 @@ public void testLongPath() throws Exception { ProxyConfiguration proxyConfig = PulsarConfigurationLoader.create(props, ProxyConfiguration.class); AuthenticationService authService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); WebServer webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, null, - new BrokerDiscoveryProvider(proxyConfig, resource)); + new BrokerDiscoveryProvider(proxyConfig, resource), proxyClientAuthentication); webServer.start(); try { Response r = client.target(webServer.getServiceUri()).path("/foo/bar/blah/foobar").request().get(); @@ -358,6 +384,10 @@ public void testLongUri() throws Exception { ProxyConfiguration proxyConfig = PulsarConfigurationLoader.create(props, ProxyConfiguration.class); AuthenticationService authService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); StringBuilder longUri = new StringBuilder("/service3/tp"); for (int i = 10 * 1024; i > 0; i = i - 11){ @@ -366,7 +396,7 @@ public void testLongUri() throws Exception { WebServer webServerMaxUriLen8k = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServerMaxUriLen8k, proxyConfig, null, - new BrokerDiscoveryProvider(proxyConfig, resource)); + new BrokerDiscoveryProvider(proxyConfig, resource), proxyClientAuthentication); webServerMaxUriLen8k.start(); try { Response r = client.target(webServerMaxUriLen8k.getServiceUri()).path(longUri.toString()).request().get(); @@ -378,7 +408,7 @@ public void testLongUri() throws Exception { proxyConfig.setHttpMaxRequestHeaderSize(12 * 1024); WebServer webServerMaxUriLen12k = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServerMaxUriLen12k, proxyConfig, null, - new BrokerDiscoveryProvider(proxyConfig, resource)); + new BrokerDiscoveryProvider(proxyConfig, resource), proxyClientAuthentication); webServerMaxUriLen12k.start(); try { Response r = client.target(webServerMaxUriLen12k.getServiceUri()).path(longUri.toString()).request().get(); @@ -399,10 +429,14 @@ public void testPathEndsInSlash() throws Exception { ProxyConfiguration proxyConfig = PulsarConfigurationLoader.create(props, ProxyConfiguration.class); AuthenticationService authService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); WebServer webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, null, - new BrokerDiscoveryProvider(proxyConfig, resource)); + new BrokerDiscoveryProvider(proxyConfig, resource), proxyClientAuthentication); webServer.start(); try { Response r = client.target(webServer.getServiceUri()).path("/ui/foobar").request().get(); @@ -431,10 +465,14 @@ public void testStreaming() throws Exception { ProxyConfiguration proxyConfig = PulsarConfigurationLoader.create(props, ProxyConfiguration.class); AuthenticationService authService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); WebServer webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, null, - new BrokerDiscoveryProvider(proxyConfig, resource)); + new BrokerDiscoveryProvider(proxyConfig, resource), proxyClientAuthentication); webServer.start(); HttpClient httpClient = new HttpClient(); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithAuth.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithAuth.java index 88e7b269d6eeb..d22c64f977fc1 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithAuth.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithAuth.java @@ -33,6 +33,8 @@ import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationProviderTls; import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -54,6 +56,7 @@ public class ProxyKeyStoreTlsTestWithAuth extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeMethod @@ -87,9 +90,13 @@ protected void setup() throws Exception { providers.add(AuthenticationProviderTls.class.getName()); proxyConfig.setAuthenticationProviders(providers); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(new ZKMetadataStore(mockZooKeeper)).when(proxyService).createLocalMetadataStore(); doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(proxyService).createConfigurationMetadataStore(); @@ -102,6 +109,9 @@ protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } protected PulsarClient internalSetUpForClient(boolean addCertificates, String lookupUrl) throws Exception { diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithoutAuth.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithoutAuth.java index 5feef74e3b94b..050b230ba4d6e 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithoutAuth.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTestWithoutAuth.java @@ -29,6 +29,8 @@ import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -50,6 +52,7 @@ public class ProxyKeyStoreTlsTestWithoutAuth extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeMethod @@ -75,8 +78,12 @@ protected void setup() throws Exception { proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(new ZKMetadataStore(mockZooKeeper)).when(proxyService).createLocalMetadataStore(); doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(proxyService).createConfigurationMetadataStore(); @@ -107,6 +114,9 @@ protected PulsarClient internalSetUpForClient(boolean addCertificates, String lo protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTransportTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTransportTest.java index 5c4e40ed65a70..2a9bc720b79d8 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTransportTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyKeyStoreTlsTransportTest.java @@ -24,6 +24,8 @@ import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; @@ -40,6 +42,7 @@ public class ProxyKeyStoreTlsTransportTest extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeMethod @@ -86,9 +89,13 @@ protected void setup() throws Exception { proxyConfig.setBrokerClientTlsTrustStore(BROKER_TRUSTSTORE_FILE_PATH); proxyConfig.setBrokerClientTlsTrustStorePassword(BROKER_TRUSTSTORE_PW); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(new ZKMetadataStore(mockZooKeeper)).when(proxyService).createLocalMetadataStore(); doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(proxyService).createConfigurationMetadataStore(); @@ -101,6 +108,9 @@ protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } protected PulsarClient newClient() throws Exception { diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java index 1b63aa14dfe42..9348a4df4db32 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyLookupThrottlingTest.java @@ -21,16 +21,15 @@ import static org.mockito.Mockito.doReturn; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; - import java.util.Optional; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; - import lombok.Cleanup; - import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; @@ -53,6 +52,7 @@ public class ProxyLookupThrottlingTest extends MockedPulsarServiceBaseTest { private final int NUM_CONCURRENT_INBOUND_CONNECTION = 5; private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeMethod(alwaysRun = true) @@ -68,7 +68,10 @@ protected void setup() throws Exception { AuthenticationService authenticationService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); - proxyService = Mockito.spy(new ProxyService(proxyConfig, authenticationService)); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, authenticationService, proxyClientAuthentication)); doReturn(new ZKMetadataStore(mockZooKeeper)).when(proxyService).createLocalMetadataStore(); doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(proxyService).createConfigurationMetadataStore(); @@ -82,6 +85,9 @@ protected void cleanup() throws Exception { if (proxyService != null) { proxyService.close(); } + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test(groups = "quarantine") diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyMutualTlsTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyMutualTlsTest.java index ad237c2539700..a73b2d388f8ec 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyMutualTlsTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyMutualTlsTest.java @@ -26,6 +26,8 @@ import lombok.Cleanup; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; @@ -48,6 +50,7 @@ public class ProxyMutualTlsTest extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeClass @@ -67,8 +70,12 @@ protected void setup() throws Exception { proxyConfig.setTlsRequireTrustedClientCertOnConnect(true); proxyConfig.setTlsAllowInsecureConnection(false); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(new ZKMetadataStore(mockZooKeeper)).when(proxyService).createLocalMetadataStore(); doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(proxyService).createConfigurationMetadataStore(); @@ -81,6 +88,9 @@ protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java index 82cd702aa7f0a..74a4435b768bf 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyParserTest.java @@ -34,6 +34,8 @@ import lombok.Cleanup; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageRoutingMode; @@ -64,6 +66,7 @@ public class ProxyParserTest extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeClass @@ -76,9 +79,12 @@ protected void setup() throws Exception { proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); //enable full parsing feature proxyConfig.setProxyLogLevel(Optional.ofNullable(2)); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(new ZKMetadataStore(mockZooKeeper)).when(proxyService).createLocalMetadataStore(); doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(proxyService).createConfigurationMetadataStore(); @@ -93,6 +99,9 @@ protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPrometheusMetricsTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPrometheusMetricsTest.java index 6948996ad4636..69dcda2a4d71b 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPrometheusMetricsTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyPrometheusMetricsTest.java @@ -42,6 +42,8 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; +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.metadata.impl.ZKMetadataStore; import org.awaitility.Awaitility; @@ -59,6 +61,7 @@ public class ProxyPrometheusMetricsTest extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private WebServer proxyWebServer; private final ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeClass @@ -72,8 +75,13 @@ protected void setup() throws Exception { proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); proxyConfig.setClusterName(TEST_CLUSTER); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, - new AuthenticationService(PulsarConfigurationLoader.convertFrom(proxyConfig)))); + new AuthenticationService(PulsarConfigurationLoader.convertFrom(proxyConfig)), + proxyClientAuthentication)); doReturn(new ZKMetadataStore(mockZooKeeper)).when(proxyService).createLocalMetadataStore(); doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(proxyService).createConfigurationMetadataStore(); @@ -85,7 +93,7 @@ protected void setup() throws Exception { PulsarConfigurationLoader.convertFrom(proxyConfig)); proxyWebServer = new WebServer(proxyConfig, authService); - ProxyServiceStarter.addWebServerHandlers(proxyWebServer, proxyConfig, proxyService, null); + ProxyServiceStarter.addWebServerHandlers(proxyWebServer, proxyConfig, proxyService, null, proxyClientAuthentication); proxyWebServer.start(); } @@ -108,6 +116,9 @@ protected void cleanup() throws Exception { if (proxyService != null) { proxyService.close(); } + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } /** diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java index 2f36cc679f1f2..b058e4af8301e 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRefreshAuthTest.java @@ -35,6 +35,8 @@ import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.impl.ClientCnx; @@ -56,6 +58,7 @@ public class ProxyRefreshAuthTest extends ProducerConsumerBase { private ProxyService proxyService; private final ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override protected void doInitConf() throws Exception { @@ -125,9 +128,13 @@ protected void setup() throws Exception { properties.setProperty("tokenSecretKey", AuthTokenUtils.encodeKeyBase64(SECRET_KEY)); proxyConfig.setProperties(properties); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); } @AfterClass(alwaysRun = true) @@ -135,6 +142,9 @@ protected void setup() throws Exception { protected void cleanup() throws Exception { super.internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } private void startProxy(boolean forwardAuthData) throws Exception { diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRolesEnforcementTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRolesEnforcementTest.java index 3259cfd95c741..e32f79b0f0fcf 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRolesEnforcementTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyRolesEnforcementTest.java @@ -28,6 +28,7 @@ import java.util.Optional; import java.util.Set; import javax.naming.AuthenticationException; +import lombok.Cleanup; import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.authentication.AuthenticationDataSource; import org.apache.pulsar.broker.authentication.AuthenticationProvider; @@ -35,6 +36,7 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.AuthenticationDataProvider; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; @@ -217,9 +219,14 @@ public void testIncorrectRoles() throws Exception { providers.add(BasicAuthenticationProvider.class.getName()); proxyConfig.setAuthenticationProviders(providers); + @Cleanup + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + try (ProxyService proxyService = new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))) { + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)) { proxyService.start(); 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 759eabdb5a20e..e12415b0d91b4 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 @@ -20,16 +20,22 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import java.io.IOException; import java.net.URI; import java.nio.ByteBuffer; import java.util.Base64; +import java.util.Map; import java.util.Optional; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.Future; +import java.util.function.Consumer; import lombok.Cleanup; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.client.api.Authentication; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.common.util.ObjectMapperFactory; import org.apache.pulsar.websocket.data.ProducerMessage; import org.eclipse.jetty.client.HttpClient; @@ -155,4 +161,89 @@ public String getResponse() throws InterruptedException { } } + @Test + public void testProxyClientAuthentication() throws Exception { + final Consumer initConfig = (proxyConfig) -> { + proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl()); + proxyConfig.setBrokerWebServiceURL(pulsar.getWebServiceAddress()); + proxyConfig.setWebServicePort(Optional.of(0)); + proxyConfig.setServicePort(Optional.of(0)); + proxyConfig.setWebSocketServiceEnabled(true); + proxyConfig.setBrokerProxyAllowedTargetPorts("*"); + proxyConfig.setClusterName(configClusterName); + }; + + + + ProxyServiceStarter serviceStarter = new ProxyServiceStarter(ARGS, null, true); + initConfig.accept(serviceStarter.getConfig()); + // ProxyServiceStarter will throw an exception when Authentication#start is failed + serviceStarter.getConfig().setBrokerClientAuthenticationPlugin(ExceptionAuthentication1.class.getName()); + try { + serviceStarter.start(); + fail("ProxyServiceStarter should throw an exception when Authentication#start is failed"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("ExceptionAuthentication1#start")); + assertTrue(serviceStarter.getProxyClientAuthentication() instanceof ExceptionAuthentication1); + } + + serviceStarter = new ProxyServiceStarter(ARGS, 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()); + try { + serviceStarter.start(); + fail("ProxyServiceStarter should throw an exception when Authentication#start and Authentication#close are failed"); + } catch (Exception ex) { + assertTrue(ex.getMessage().contains("ExceptionAuthentication2#start")); + assertTrue(serviceStarter.getProxyClientAuthentication() instanceof ExceptionAuthentication2); + } + } + + public static class ExceptionAuthentication1 implements Authentication { + + @Override + public String getAuthMethodName() { + return "org.apache.pulsar.proxy.server.ProxyConfigurationTest.ExceptionAuthentication1"; + } + + @Override + public void configure(Map authParams) { + // no-op + } + + @Override + public void start() throws PulsarClientException { + throw new PulsarClientException("ExceptionAuthentication1#start"); + } + + @Override + public void close() throws IOException { + // no-op + } + } + + public static class ExceptionAuthentication2 implements Authentication { + + @Override + public String getAuthMethodName() { + return "org.apache.pulsar.proxy.server.ProxyConfigurationTest.ExceptionAuthentication2"; + } + + @Override + public void configure(Map authParams) { + // no-op + } + + @Override + public void start() throws PulsarClientException { + throw new PulsarClientException("ExceptionAuthentication2#start"); + } + + @Override + public void close() throws IOException { + throw new IOException("ExceptionAuthentication2#close"); + } + } + } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStatsTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStatsTest.java index 140af88aae71b..5769571d1871b 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStatsTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStatsTest.java @@ -38,6 +38,8 @@ import org.apache.pulsar.broker.ServiceConfiguration; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageRoutingMode; @@ -61,6 +63,7 @@ public class ProxyStatsTest extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private WebServer proxyWebServer; private final ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeClass @@ -75,8 +78,12 @@ protected void setup() throws Exception { // enable full parsing feature proxyConfig.setProxyLogLevel(Optional.of(2)); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, - new AuthenticationService(PulsarConfigurationLoader.convertFrom(proxyConfig)))); + new AuthenticationService(PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(new ZKMetadataStore(mockZooKeeper)).when(proxyService).createLocalMetadataStore(); doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(proxyService).createConfigurationMetadataStore(); @@ -88,7 +95,7 @@ protected void setup() throws Exception { PulsarConfigurationLoader.convertFrom(proxyConfig)); proxyWebServer = new WebServer(proxyConfig, authService); - ProxyServiceStarter.addWebServerHandlers(proxyWebServer, proxyConfig, proxyService, null); + ProxyServiceStarter.addWebServerHandlers(proxyWebServer, proxyConfig, proxyService, null, proxyClientAuthentication); proxyWebServer.start(); } @@ -106,6 +113,9 @@ protected ServiceConfiguration getDefaultConf() { protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } /** diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStuckConnectionTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStuckConnectionTest.java index 373926d11c10d..edc5e4eb1462d 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStuckConnectionTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyStuckConnectionTest.java @@ -28,6 +28,8 @@ import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.KeySharedPolicy; import org.apache.pulsar.client.api.Message; @@ -56,6 +58,7 @@ public class ProxyStuckConnectionTest extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private ProxyConfiguration proxyConfig; + private Authentication proxyClientAuthentication; private SocatContainer socatContainer; private String brokerServiceUriSocat; @@ -80,6 +83,10 @@ protected void setup() throws Exception { proxyConfig.setBrokerProxyAllowedTargetPorts("*"); proxyConfig.setBrokerServiceURL(pulsar.getBrokerServiceUrl()); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + startProxyService(); // use the same port for subsequent restarts proxyConfig.setServicePort(proxyService.getListenPort()); @@ -87,7 +94,7 @@ protected void setup() throws Exception { private void startProxyService() throws Exception { proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig))) { + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication) { @Override protected LookupProxyHandler newLookupProxyHandler(ProxyConnection proxyConnection) { return new TestLookupProxyHandler(this, proxyConnection); @@ -105,6 +112,9 @@ protected void cleanup() throws Exception { if (proxyService != null) { proxyService.close(); } + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } if (socatContainer != null) { socatContainer.close(); } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java index e799e2e948a4a..6fab7d6af21e7 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTest.java @@ -38,6 +38,8 @@ import org.apache.pulsar.PulsarVersion; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageRoutingMode; @@ -73,6 +75,7 @@ public class ProxyTest extends MockedPulsarServiceBaseTest { protected ProxyService proxyService; protected ProxyConfiguration proxyConfig = new ProxyConfiguration(); + protected Authentication proxyClientAuthentication; @Data @ToString @@ -90,25 +93,38 @@ public static class Foo { protected void setup() throws Exception { internalSetup(); - proxyConfig.setServicePort(Optional.ofNullable(0)); - proxyConfig.setBrokerProxyAllowedTargetPorts("*"); - proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); - proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + initializeProxyConfig(); proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(new ZKMetadataStore(mockZooKeeper)).when(proxyService).createLocalMetadataStore(); - doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(proxyService).createConfigurationMetadataStore(); + doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(proxyService) + .createConfigurationMetadataStore(); proxyService.start(); } + protected void initializeProxyConfig() throws Exception { + proxyConfig.setServicePort(Optional.ofNullable(0)); + proxyConfig.setBrokerProxyAllowedTargetPorts("*"); + proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); + proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyConfig.setClusterName(configClusterName); + + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + } + @Override @AfterClass(alwaysRun = true) protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTest.java index 64b0cd6b1a610..79a55dea67471 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTest.java @@ -27,6 +27,8 @@ import lombok.Cleanup; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageRoutingMode; @@ -45,6 +47,7 @@ public class ProxyTlsTest extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @Override @BeforeClass @@ -62,8 +65,12 @@ protected void setup() throws Exception { proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(new ZKMetadataStore(mockZooKeeper)).when(proxyService).createLocalMetadataStore(); doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(proxyService).createConfigurationMetadataStore(); @@ -76,6 +83,9 @@ protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTestWithAuth.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTestWithAuth.java index 0f1fa74a20916..2a68d92c9e1df 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTestWithAuth.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyTlsTestWithAuth.java @@ -27,6 +27,8 @@ import org.apache.pulsar.broker.auth.MockOIDCIdentityProvider; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; +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.metadata.impl.ZKMetadataStore; import org.mockito.Mockito; @@ -38,6 +40,7 @@ public class ProxyTlsTestWithAuth extends MockedPulsarServiceBaseTest { private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; private MockOIDCIdentityProvider server; @@ -74,8 +77,12 @@ protected void setup() throws Exception { " \"audience\": \"an-audience\"," + " \"privateKey\":\"file://" + tempFile.getAbsolutePath() + "\"}"); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(new ZKMetadataStore(mockZooKeeper)).when(proxyService).createLocalMetadataStore(); doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(proxyService).createConfigurationMetadataStore(); @@ -87,6 +94,9 @@ protected void setup() throws Exception { protected void cleanup() throws Exception { internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } server.stop(); } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java index 2d97a4b06a856..b10fb22160061 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationNegTest.java @@ -34,6 +34,7 @@ import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Producer; @@ -72,6 +73,7 @@ public class ProxyWithAuthorizationNegTest extends ProducerConsumerBase { private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @BeforeMethod @Override @@ -136,7 +138,10 @@ protected void setup() throws Exception { AuthenticationService authenticationService = new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig)); - proxyService = Mockito.spy(new ProxyService(proxyConfig, authenticationService)); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, authenticationService, proxyClientAuthentication)); proxyService.start(); } @@ -146,6 +151,9 @@ protected void setup() throws Exception { protected void cleanup() throws Exception { super.internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } /** diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java index 31757cc036720..9087e8f477e2a 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithAuthorizationTest.java @@ -38,6 +38,7 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.ClientBuilder; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -86,6 +87,7 @@ public class ProxyWithAuthorizationTest extends ProducerConsumerBase { private ProxyService proxyService; private WebServer webServer; private final ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @DataProvider(name = "hostnameVerification") public Object[][] hostnameVerificationCodecProvider() { @@ -228,7 +230,10 @@ protected void setup() throws Exception { AuthenticationService authService = new AuthenticationService(PulsarConfigurationLoader.convertFrom(proxyConfig)); - proxyService = Mockito.spy(new ProxyService(proxyConfig, authService)); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, authService, proxyClientAuthentication)); webServer = new WebServer(proxyConfig, authService); } @@ -238,11 +243,14 @@ protected void cleanup() throws Exception { super.internalCleanup(); proxyService.close(); webServer.stop(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } private void startProxy() throws Exception { proxyService.start(); - ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, proxyService, null); + ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, proxyService, null, proxyClientAuthentication); webServer.start(); } @@ -415,7 +423,7 @@ public void testTlsHostVerificationProxyToBroker(boolean hostnameVerificationEna * This test verifies whether the Client and Proxy honor the protocols and ciphers specified. Details description of * test cases can be found in protocolsCiphersProviderCodecProvider */ - @Test(dataProvider = "protocolsCiphersProvider", timeOut = 5000) + @Test(dataProvider = "protocolsCiphersProvider", timeOut = 10000) public void tlsCiphersAndProtocols(Set tlsCiphers, Set tlsProtocols, boolean expectFailure) throws Exception { log.info("-- Starting {} test --", methodName); @@ -455,9 +463,14 @@ public void tlsCiphersAndProtocols(Set tlsCiphers, Set tlsProtoc proxyConfig.setTlsProtocols(tlsProtocols); proxyConfig.setTlsCiphers(tlsCiphers); + final Authentication proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + + @Cleanup ProxyService proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); try { proxyService.start(); } catch (Exception ex) { diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java index 88ecfe8a3187b..6b1890e23269b 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithJwtAuthorizationTest.java @@ -39,7 +39,16 @@ import org.apache.pulsar.broker.authentication.utils.AuthTokenUtils; import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.client.api.*; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; +import org.apache.pulsar.client.api.ClientBuilder; +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.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.auth.AuthenticationToken; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.policies.data.AuthAction; @@ -74,6 +83,7 @@ public class ProxyWithJwtAuthorizationTest extends ProducerConsumerBase { private ProxyService proxyService; private WebServer webServer; private final ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @BeforeMethod @Override @@ -120,7 +130,10 @@ protected void setup() throws Exception { AuthenticationService authService = new AuthenticationService(PulsarConfigurationLoader.convertFrom(proxyConfig)); - proxyService = Mockito.spy(new ProxyService(proxyConfig, authService)); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, authService, proxyClientAuthentication)); webServer = new WebServer(proxyConfig, authService); } @@ -130,11 +143,14 @@ protected void cleanup() throws Exception { super.internalCleanup(); proxyService.close(); webServer.stop(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } private void startProxy() throws Exception { proxyService.start(); - ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, proxyService, null); + ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, proxyService, null, proxyClientAuthentication); webServer.start(); } @@ -415,7 +431,7 @@ void testGetStatus() throws Exception { PulsarConfigurationLoader.convertFrom(proxyConfig)); final WebServer webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, proxyService, - new BrokerDiscoveryProvider(proxyConfig, resource)); + new BrokerDiscoveryProvider(proxyConfig, resource), proxyClientAuthentication); webServer.start(); @Cleanup final Client client = javax.ws.rs.client.ClientBuilder @@ -440,7 +456,7 @@ void testGetMetrics() throws Exception { proxyConfig.setAuthenticateMetricsEndpoint(false); WebServer webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, proxyService, - new BrokerDiscoveryProvider(proxyConfig, resource)); + new BrokerDiscoveryProvider(proxyConfig, resource), proxyClientAuthentication); webServer.start(); @Cleanup Client client = javax.ws.rs.client.ClientBuilder.newClient(new ClientConfig().register(LoggingFeature.class)); @@ -453,7 +469,7 @@ void testGetMetrics() throws Exception { proxyConfig.setAuthenticateMetricsEndpoint(true); webServer = new WebServer(proxyConfig, authService); ProxyServiceStarter.addWebServerHandlers(webServer, proxyConfig, proxyService, - new BrokerDiscoveryProvider(proxyConfig, resource)); + new BrokerDiscoveryProvider(proxyConfig, resource), proxyClientAuthentication); webServer.start(); try { Response r = client.target(webServer.getServiceUri()).path("/metrics").request().get(); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithoutServiceDiscoveryTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithoutServiceDiscoveryTest.java index 9c8e2ba33c9e8..3f6ec8b4ee68c 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithoutServiceDiscoveryTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/ProxyWithoutServiceDiscoveryTest.java @@ -33,6 +33,7 @@ import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Producer; @@ -56,6 +57,7 @@ public class ProxyWithoutServiceDiscoveryTest extends ProducerConsumerBase { private static final Logger log = LoggerFactory.getLogger(ProxyWithoutServiceDiscoveryTest.class); private ProxyService proxyService; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; @BeforeMethod @Override @@ -119,9 +121,13 @@ protected void setup() throws Exception { proxyConfig.setAuthenticationProviders(providers); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); proxyService.start(); } @@ -131,6 +137,9 @@ protected void setup() throws Exception { protected void cleanup() throws Exception { super.internalCleanup(); proxyService.close(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } /** diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/SuperUserAuthedAdminProxyHandlerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/SuperUserAuthedAdminProxyHandlerTest.java index d3291c8fb910d..0d5102811345f 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/SuperUserAuthedAdminProxyHandlerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/SuperUserAuthedAdminProxyHandlerTest.java @@ -20,18 +20,17 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; - import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; - import java.util.Optional; - import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationProviderTls; import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.impl.auth.AuthenticationTls; import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; import org.apache.pulsar.common.policies.data.ClusterData; @@ -47,6 +46,7 @@ public class SuperUserAuthedAdminProxyHandlerTest extends MockedPulsarServiceBaseTest { private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; private WebServer webServer; private BrokerDiscoveryProvider discoveryProvider; private PulsarResources resource; @@ -95,13 +95,17 @@ protected void setup() throws Exception { resource = new PulsarResources(new ZKMetadataStore(mockZooKeeper), new ZKMetadataStore(mockZooKeeperGlobal)); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + webServer = new WebServer(proxyConfig, new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig))); discoveryProvider = spy(new BrokerDiscoveryProvider(proxyConfig, resource)); LoadManagerReport report = new LoadReport(brokerUrl.toString(), brokerUrlTls.toString(), null, null); doReturn(report).when(discoveryProvider).nextBroker(); - ServletHolder servletHolder = new ServletHolder(new AdminProxyHandler(proxyConfig, discoveryProvider)); + ServletHolder servletHolder = new ServletHolder(new AdminProxyHandler(proxyConfig, discoveryProvider, proxyClientAuthentication)); webServer.addServlet("/admin", servletHolder); webServer.addServlet("/lookup", servletHolder); @@ -113,6 +117,9 @@ protected void setup() throws Exception { @Override protected void cleanup() throws Exception { webServer.stop(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } super.internalCleanup(); } diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/URLRegexLookupProxyHandlerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/URLRegexLookupProxyHandlerTest.java index 96c63f5cae9d0..a1f8b9506b6b5 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/URLRegexLookupProxyHandlerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/URLRegexLookupProxyHandlerTest.java @@ -31,6 +31,8 @@ import lombok.Cleanup; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.authentication.AuthenticationService; +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationFactory; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageRoutingMode; @@ -46,6 +48,8 @@ public class URLRegexLookupProxyHandlerTest extends MockedPulsarServiceBaseTest { + private Authentication proxyClientAuthentication; + protected ProxyService proxyService; protected ProxyConfiguration proxyConfig = new ProxyConfiguration(); @@ -59,8 +63,12 @@ protected void setup() throws Exception { proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + proxyService = Mockito.spy(new ProxyService(proxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(proxyConfig)))); + PulsarConfigurationLoader.convertFrom(proxyConfig)), proxyClientAuthentication)); doReturn(new ZKMetadataStore(mockZooKeeper)).when(proxyService).createLocalMetadataStore(); doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(proxyService).createConfigurationMetadataStore(); @@ -88,7 +96,7 @@ void testMatchingRegex() throws Exception { @Cleanup ProxyService redirectProxyService = Mockito.spy(new ProxyService(redirectProxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(redirectProxyConfig)))); + PulsarConfigurationLoader.convertFrom(redirectProxyConfig)), proxyClientAuthentication)); doReturn(new ZKMetadataStore(mockZooKeeper)).when(redirectProxyService).createLocalMetadataStore(); doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(redirectProxyService).createConfigurationMetadataStore(); @@ -144,7 +152,7 @@ void testNotMatchingRegex() throws Exception { @Cleanup ProxyService redirectProxyService = Mockito.spy(new ProxyService(redirectProxyConfig, new AuthenticationService( - PulsarConfigurationLoader.convertFrom(redirectProxyConfig)))); + PulsarConfigurationLoader.convertFrom(redirectProxyConfig)), proxyClientAuthentication)); doReturn(new ZKMetadataStore(mockZooKeeper)).when(redirectProxyService).createLocalMetadataStore(); doReturn(new ZKMetadataStore(mockZooKeeperGlobal)).when(redirectProxyService).createConfigurationMetadataStore(); diff --git a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/UnauthedAdminProxyHandlerTest.java b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/UnauthedAdminProxyHandlerTest.java index aa4aeaa2ea887..78ac1e46a15f1 100644 --- a/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/UnauthedAdminProxyHandlerTest.java +++ b/pulsar-proxy/src/test/java/org/apache/pulsar/proxy/server/UnauthedAdminProxyHandlerTest.java @@ -35,6 +35,8 @@ import org.apache.pulsar.broker.authentication.AuthenticationService; import org.apache.pulsar.broker.resources.PulsarResources; import org.apache.pulsar.client.admin.PulsarAdmin; +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.configuration.VipStatus; import org.apache.pulsar.metadata.impl.ZKMetadataStore; @@ -49,6 +51,7 @@ public class UnauthedAdminProxyHandlerTest extends MockedPulsarServiceBaseTest { private final String STATUS_FILE_PATH = "./src/test/resources/vip_status.html"; private ProxyConfiguration proxyConfig = new ProxyConfiguration(); + private Authentication proxyClientAuthentication; private WebServer webServer; private BrokerDiscoveryProvider discoveryProvider; private AdminProxyWrapper adminProxyHandler; @@ -76,13 +79,17 @@ protected void setup() throws Exception { proxyConfig.setMetadataStoreUrl(DUMMY_VALUE); proxyConfig.setConfigurationMetadataStoreUrl(GLOBAL_DUMMY_VALUE); + proxyClientAuthentication = AuthenticationFactory.create(proxyConfig.getBrokerClientAuthenticationPlugin(), + proxyConfig.getBrokerClientAuthenticationParameters()); + proxyClientAuthentication.start(); + webServer = new WebServer(proxyConfig, new AuthenticationService( PulsarConfigurationLoader.convertFrom(proxyConfig))); resource = new PulsarResources(new ZKMetadataStore(mockZooKeeper), new ZKMetadataStore(mockZooKeeperGlobal)); discoveryProvider = spy(new BrokerDiscoveryProvider(proxyConfig, resource)); - adminProxyHandler = new AdminProxyWrapper(proxyConfig, discoveryProvider); + adminProxyHandler = new AdminProxyWrapper(proxyConfig, discoveryProvider, proxyClientAuthentication); ServletHolder servletHolder = new ServletHolder(adminProxyHandler); webServer.addServlet("/admin", servletHolder); webServer.addServlet("/lookup", servletHolder); @@ -100,6 +107,9 @@ protected void setup() throws Exception { protected void cleanup() throws Exception { internalCleanup(); webServer.stop(); + if (proxyClientAuthentication != null) { + proxyClientAuthentication.close(); + } } @Test @@ -126,8 +136,8 @@ public void testVipStatus() throws Exception { static class AdminProxyWrapper extends AdminProxyHandler { String rewrittenUrl; - AdminProxyWrapper(ProxyConfiguration config, BrokerDiscoveryProvider discoveryProvider) { - super(config, discoveryProvider); + AdminProxyWrapper(ProxyConfiguration config, BrokerDiscoveryProvider discoveryProvider, Authentication proxyClientAuthentication) { + super(config, discoveryProvider, proxyClientAuthentication); } @Override From bdfc5912cf058e718e13cb81155050664b65346d Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Wed, 14 Aug 2024 10:26:47 +0800 Subject: [PATCH 667/699] [fix][client] Fix for early hit `beforeConsume` for MultiTopicConsumer (#23141) (cherry picked from commit c07b158f003c5a5623296189f0932d7058d2e75a) (cherry picked from commit 69a6232765ead57cbe29686fe6214346f29ae89b) --- .../pulsar/client/api/InterceptorsTest.java | 44 +++++++++----- .../client/impl/MultiTopicsConsumerImpl.java | 58 +++++++++++++++++-- 2 files changed, 83 insertions(+), 19 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 f23d82b32cd43..afb17a186477c 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 @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.api; +import com.google.common.collect.Sets; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -29,8 +30,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; - -import com.google.common.collect.Sets; import lombok.Cleanup; import org.apache.commons.lang3.RandomStringUtils; import org.apache.commons.lang3.RandomUtils; @@ -79,6 +78,12 @@ public Object[][] getTopicPartition() { return new Object[][] {{ 0 }, { 3 }}; } + @DataProvider(name = "topics") + public Object[][] getTopics() { + return new Object[][] {{ List.of("persistent://my-property/my-ns/my-topic") }, + { List.of("persistent://my-property/my-ns/my-topic", "persistent://my-property/my-ns/my-topic1") }}; + } + @Test public void testProducerInterceptor() throws Exception { Map> ackCallback = new HashMap<>(); @@ -403,9 +408,9 @@ public void close() { @Override public Message beforeConsume(Consumer consumer, Message message) { - MessageImpl msg = (MessageImpl) message; + MessageImpl msg = ((MessageImpl) ((TopicMessageImpl) message).getMessage()); msg.getMessageBuilder().addProperty().setKey("beforeConsumer").setValue("1"); - return msg; + return message; } @Override @@ -449,13 +454,19 @@ public void onAckTimeoutSend(Consumer consumer, Set messageId int keyCount = 0; for (int i = 0; i < 2; i++) { - Message received = consumer.receive(); + Message received; + if (i % 2 == 0) { + received = consumer.receive(); + } else { + received = consumer.receiveAsync().join(); + } MessageImpl msg = (MessageImpl) ((TopicMessageImpl) received).getMessage(); for (KeyValue keyValue : msg.getMessageBuilder().getPropertiesList()) { if ("beforeConsumer".equals(keyValue.getKey())) { keyCount++; } } + Assert.assertEquals(keyCount, i + 1); consumer.acknowledge(received); } Assert.assertEquals(2, keyCount); @@ -475,9 +486,9 @@ public void close() { @Override public Message beforeConsume(Consumer consumer, Message message) { - MessageImpl msg = (MessageImpl) message; + MessageImpl msg = ((MessageImpl) ((TopicMessageImpl) message).getMessage()); msg.getMessageBuilder().addProperty().setKey("beforeConsumer").setValue("1"); - return msg; + return message; } @Override @@ -612,8 +623,8 @@ public void onAckTimeoutSend(Consumer consumer, Set messageId consumer.close(); } - @Test - public void testConsumerInterceptorForNegativeAcksSend() throws PulsarClientException, InterruptedException { + @Test(dataProvider = "topics") + public void testConsumerInterceptorForNegativeAcksSend(List topics) throws PulsarClientException, InterruptedException { final int totalNumOfMessages = 100; CountDownLatch latch = new CountDownLatch(totalNumOfMessages / 2); @@ -640,6 +651,7 @@ public void onAcknowledgeCumulative(Consumer consumer, MessageId message @Override public void onNegativeAcksSend(Consumer consumer, Set messageIds) { + Assert.assertTrue(latch.getCount() > 0); messageIds.forEach(messageId -> latch.countDown()); } @@ -650,7 +662,7 @@ public void onAckTimeoutSend(Consumer consumer, Set messageId }; Consumer consumer = pulsarClient.newConsumer(Schema.STRING) - .topic("persistent://my-property/my-ns/my-topic") + .topics(topics) .subscriptionType(SubscriptionType.Failover) .intercept(interceptor) .negativeAckRedeliveryDelay(100, TimeUnit.MILLISECONDS) @@ -658,7 +670,7 @@ public void onAckTimeoutSend(Consumer consumer, Set messageId .subscribe(); Producer producer = pulsarClient.newProducer(Schema.STRING) - .topic("persistent://my-property/my-ns/my-topic") + .topic(topics.get(0)) .create(); for (int i = 0; i < totalNumOfMessages; i++) { @@ -682,8 +694,9 @@ public void onAckTimeoutSend(Consumer consumer, Set messageId consumer.close(); } - @Test - public void testConsumerInterceptorForAckTimeoutSend() throws PulsarClientException, InterruptedException { + @Test(dataProvider = "topics") + public void testConsumerInterceptorForAckTimeoutSend(List topics) throws PulsarClientException, + InterruptedException { final int totalNumOfMessages = 100; CountDownLatch latch = new CountDownLatch(totalNumOfMessages / 2); @@ -714,16 +727,17 @@ public void onNegativeAcksSend(Consumer consumer, Set message @Override public void onAckTimeoutSend(Consumer consumer, Set messageIds) { + Assert.assertTrue(latch.getCount() > 0); messageIds.forEach(messageId -> latch.countDown()); } }; Producer producer = pulsarClient.newProducer(Schema.STRING) - .topic("persistent://my-property/my-ns/my-topic") + .topic(topics.get(0)) .create(); Consumer consumer = pulsarClient.newConsumer(Schema.STRING) - .topic("persistent://my-property/my-ns/my-topic") + .topics(topics) .subscriptionName("foo") .intercept(interceptor) .ackTimeout(2, TimeUnit.SECONDS) 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 303857eb187c2..158ac0691b844 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 @@ -26,6 +26,7 @@ import com.google.common.collect.Lists; import io.netty.util.Timeout; import io.netty.util.TimerTask; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -108,6 +109,7 @@ public class MultiTopicsConsumerImpl extends ConsumerBase { private final MessageIdAdv startMessageId; private volatile boolean duringSeek = false; private final long startMessageRollbackDurationInSec; + private final ConsumerInterceptors internalConsumerInterceptors; MultiTopicsConsumerImpl(PulsarClientImpl client, ConsumerConfigurationData conf, ExecutorProvider executorProvider, CompletableFuture> subscribeFuture, Schema schema, ConsumerInterceptors interceptors, boolean createTopicIfDoesNotExist) { @@ -137,6 +139,11 @@ public class MultiTopicsConsumerImpl extends ConsumerBase { long startMessageRollbackDurationInSec) { super(client, singleTopic, conf, Math.max(2, conf.getReceiverQueueSize()), executorProvider, subscribeFuture, schema, interceptors); + if (interceptors != null) { + this.internalConsumerInterceptors = getInternalConsumerInterceptors(interceptors); + } else { + this.internalConsumerInterceptors = null; + } checkArgument(conf.getReceiverQueueSize() > 0, "Receiver queue size needs to be greater than 0 for Topics Consumer"); @@ -316,7 +323,8 @@ private void messageReceived(ConsumerImpl consumer, Message message) { CompletableFuture> receivedFuture = nextPendingReceive(); if (receivedFuture != null) { unAckedMessageTracker.add(topicMessage.getMessageId(), topicMessage.getRedeliveryCount()); - completePendingReceive(receivedFuture, topicMessage); + final Message interceptMessage = beforeConsume(topicMessage); + completePendingReceive(receivedFuture, interceptMessage); } else if (enqueueMessageAndCheckBatchReceive(topicMessage) && hasPendingBatchReceive()) { notifyPendingBatchReceivedCallBack(); } @@ -369,7 +377,7 @@ protected Message internalReceive() throws PulsarClientException { checkState(message instanceof TopicMessageImpl); unAckedMessageTracker.add(message.getMessageId(), message.getRedeliveryCount()); resumeReceivingFromPausedConsumersIfNeeded(); - return message; + return beforeConsume(message); } catch (Exception e) { ExceptionHandler.handleInterruptedException(e); throw PulsarClientException.unwrap(e); @@ -388,6 +396,7 @@ protected Message internalReceive(long timeout, TimeUnit unit) throws PulsarC decreaseIncomingMessageSize(message); checkArgument(message instanceof TopicMessageImpl); trackUnAckedMsgIfNoListener(message.getMessageId(), message.getRedeliveryCount()); + message = beforeConsume(message); } resumeReceivingFromPausedConsumersIfNeeded(); return message; @@ -447,7 +456,7 @@ protected CompletableFuture> internalReceiveAsync() { checkState(message instanceof TopicMessageImpl); unAckedMessageTracker.add(message.getMessageId(), message.getRedeliveryCount()); resumeReceivingFromPausedConsumersIfNeeded(); - result.complete(message); + result.complete(beforeConsume(message)); } }); return result; @@ -1185,7 +1194,7 @@ private ConsumerImpl createInternalConsumer(ConsumerConfigurationData conf return ConsumerImpl.newConsumerImpl(client, partitionName, configurationData, client.externalExecutorProvider(), partitionIndex, true, listener != null, subFuture, - startMessageId, schema, interceptors, + startMessageId, schema, this.internalConsumerInterceptors, createIfDoesNotExist, startMessageRollbackDurationInSec); } @@ -1595,4 +1604,45 @@ private CompletableFuture> getExistsPartitions(String topic) { return list; }); } + + private ConsumerInterceptors getInternalConsumerInterceptors(ConsumerInterceptors multiTopicInterceptors) { + return new ConsumerInterceptors(new ArrayList<>()) { + + @Override + public Message beforeConsume(Consumer consumer, Message message) { + return message; + } + + @Override + public void onAcknowledge(Consumer consumer, MessageId messageId, Throwable exception) { + multiTopicInterceptors.onAcknowledge(consumer, messageId, exception); + } + + @Override + public void onAcknowledgeCumulative(Consumer consumer, + MessageId messageId, Throwable exception) { + multiTopicInterceptors.onAcknowledgeCumulative(consumer, messageId, exception); + } + + @Override + public void onNegativeAcksSend(Consumer consumer, Set set) { + multiTopicInterceptors.onNegativeAcksSend(consumer, set); + } + + @Override + public void onAckTimeoutSend(Consumer consumer, Set set) { + multiTopicInterceptors.onAckTimeoutSend(consumer, set); + } + + @Override + public void onPartitionsChange(String topicName, int partitions) { + multiTopicInterceptors.onPartitionsChange(topicName, partitions); + } + + @Override + public void close() throws IOException { + multiTopicInterceptors.close(); + } + }; + } } From b8aa90595a1655ceee221651898f578ecc39d17c Mon Sep 17 00:00:00 2001 From: Omar Yasin Date: Wed, 14 Aug 2024 01:00:23 -0700 Subject: [PATCH 668/699] [fix][client] Create the retry producer async (#23157) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Ómar Yasin (cherry picked from commit a0259380e1eb86dbe4e80d27c585188671b25135) (cherry picked from commit 507d40295301df56189df8ffb77bf76b3f47d39c) --- .../pulsar/client/impl/ConsumerImpl.java | 92 ++++++++++--------- 1 file changed, 49 insertions(+), 43 deletions(-) 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 844f7f54a62a8..65df1154af6ad 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 @@ -197,7 +197,7 @@ public class ConsumerImpl extends ConsumerBase implements ConnectionHandle private volatile CompletableFuture> deadLetterProducer; - private volatile Producer retryLetterProducer; + private volatile CompletableFuture> retryLetterProducer; private final ReadWriteLock createProducerLock = new ReentrantReadWriteLock(); protected volatile boolean paused; @@ -599,6 +599,7 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a Map customProperties, long delayTime, TimeUnit unit) { + MessageId messageId = message.getMessageId(); if (messageId == null) { return FutureUtil.failedFuture(new PulsarClientException @@ -615,28 +616,8 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a } return FutureUtil.failedFuture(exception); } - if (delayTime < 0) { - delayTime = 0; - } - if (retryLetterProducer == null) { - 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) - .create(); - } - } catch (Exception e) { - log.error("Create retry letter producer exception with topic: {}", - deadLetterPolicy.getRetryLetterTopic(), e); - return FutureUtil.failedFuture(e); - } finally { - createProducerLock.writeLock().unlock(); - } - } + + initRetryLetterProducerIfNeeded(); CompletableFuture result = new CompletableFuture<>(); if (retryLetterProducer != null) { try { @@ -656,7 +637,7 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a } propertiesMap.put(RetryMessageUtil.SYSTEM_PROPERTY_RECONSUMETIMES, String.valueOf(reconsumeTimes)); propertiesMap.put(RetryMessageUtil.SYSTEM_PROPERTY_DELAY_TIME, - String.valueOf(unit.toMillis(delayTime))); + String.valueOf(unit.toMillis(delayTime < 0 ? 0 : delayTime))); MessageId finalMessageId = messageId; if (reconsumeTimes > this.deadLetterPolicy.getMaxRedeliverCount() @@ -685,23 +666,29 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a }); } else { assert retryMessage != null; - TypedMessageBuilder typedMessageBuilderNew = retryLetterProducer - .newMessage(Schema.AUTO_PRODUCE_BYTES(message.getReaderSchema().get())) - .value(retryMessage.getData()) - .properties(propertiesMap); - if (delayTime > 0) { - typedMessageBuilderNew.deliverAfter(delayTime, unit); - } - if (message.hasKey()) { - typedMessageBuilderNew.key(message.getKey()); - } - typedMessageBuilderNew.sendAsync() - .thenCompose(__ -> doAcknowledge(finalMessageId, ackType, Collections.emptyMap(), null)) - .thenAccept(v -> result.complete(null)) - .exceptionally(ex -> { - result.completeExceptionally(ex); - return 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); + } + if (message.hasKey()) { + typedMessageBuilderNew.key(message.getKey()); + } + 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; + }); } } catch (Exception e) { result.completeExceptionally(e); @@ -710,7 +697,7 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a MessageId finalMessageId = messageId; result.exceptionally(ex -> { log.error("Send to retry letter topic exception with topic: {}, messageId: {}", - retryLetterProducer.getTopic(), finalMessageId, ex); + this.deadLetterPolicy.getRetryLetterTopic(), finalMessageId, ex); Set messageIds = Collections.singleton(finalMessageId); unAckedMessageTracker.remove(finalMessageId); redeliverUnacknowledgedMessages(messageIds); @@ -1084,7 +1071,7 @@ public synchronized CompletableFuture closeAsync() { ArrayList> closeFutures = new ArrayList<>(4); closeFutures.add(closeFuture); if (retryLetterProducer != null) { - closeFutures.add(retryLetterProducer.closeAsync().whenComplete((ignore, ex) -> { + closeFutures.add(retryLetterProducer.thenCompose(p -> p.closeAsync()).whenComplete((ignore, ex) -> { if (ex != null) { log.warn("Exception ignored in closing retryLetterProducer of consumer", ex); } @@ -2200,6 +2187,25 @@ private void initDeadLetterProducerIfNeeded() { } } + private void initRetryLetterProducerIfNeeded() { + if (retryLetterProducer == null) { + 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(); + } + } finally { + createProducerLock.writeLock().unlock(); + } + } + } + @Override public void seek(MessageId messageId) throws PulsarClientException { try { From c228b5d6953a53235f0f233d5a939e8a607a48d7 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 14 Aug 2024 16:39:55 +0800 Subject: [PATCH 669/699] [improve] [broker] Avoid subscription fenced error with consumer.seek whenever possible (#23163) (cherry picked from commit d5ce1cee35363ba2372375c2e8740be6d87488d8) (cherry picked from commit bbe67c857d24c79c6cc9e853ffde18c58af570cc) --- .../persistent/PersistentSubscription.java | 32 +++++++--- .../broker/service/SubscriptionSeekTest.java | 64 +++++++++++++++++++ 2 files changed, 86 insertions(+), 10 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 97fa497dda9c8..7e5cef8c4b755 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 @@ -132,6 +132,7 @@ public class PersistentSubscription extends AbstractSubscription implements Subs private final PendingAckHandle pendingAckHandle; private volatile Map subscriptionProperties; private volatile CompletableFuture fenceFuture; + private volatile CompletableFuture inProgressResetCursorFuture; static Map getBaseCursorProperties(boolean isReplicated) { return isReplicated ? REPLICATED_SUBSCRIPTION_CURSOR_PROPERTIES : NON_REPLICATED_SUBSCRIPTION_CURSOR_PROPERTIES; @@ -220,6 +221,16 @@ public boolean setReplicated(boolean replicated) { @Override public CompletableFuture addConsumer(Consumer consumer) { + CompletableFuture inProgressResetCursorFuture = this.inProgressResetCursorFuture; + if (inProgressResetCursorFuture != null) { + return inProgressResetCursorFuture.handle((ignore, ignoreEx) -> null) + .thenCompose(ignore -> addConsumerInternal(consumer)); + } else { + return addConsumerInternal(consumer); + } + } + + private CompletableFuture addConsumerInternal(Consumer consumer) { return pendingAckHandle.pendingAckHandleFuture().thenCompose(future -> { synchronized (PersistentSubscription.this) { cursor.updateLastActive(); @@ -747,7 +758,8 @@ public void findEntryComplete(Position position, Object ctx) { } else { finalPosition = position.getNext(); } - resetCursor(finalPosition, future); + CompletableFuture resetCursorFuture = resetCursor(finalPosition); + FutureUtil.completeAfter(future, resetCursorFuture); } @Override @@ -766,18 +778,13 @@ public void findEntryFailed(ManagedLedgerException exception, } @Override - public CompletableFuture resetCursor(Position position) { - CompletableFuture future = new CompletableFuture<>(); - resetCursor(position, future); - return future; - } - - private void resetCursor(Position finalPosition, CompletableFuture future) { + public CompletableFuture resetCursor(Position finalPosition) { if (!IS_FENCED_UPDATER.compareAndSet(PersistentSubscription.this, FALSE, TRUE)) { - future.completeExceptionally(new SubscriptionBusyException("Failed to fence subscription")); - return; + return CompletableFuture.failedFuture(new SubscriptionBusyException("Failed to fence subscription")); } + final CompletableFuture future = new CompletableFuture<>(); + inProgressResetCursorFuture = future; final CompletableFuture disconnectFuture; // Lock the Subscription object before locking the Dispatcher object to avoid deadlocks @@ -797,6 +804,7 @@ private void resetCursor(Position finalPosition, CompletableFuture future) if (throwable != null) { log.error("[{}][{}] Failed to disconnect consumer from subscription", topicName, subName, throwable); IS_FENCED_UPDATER.set(PersistentSubscription.this, FALSE); + inProgressResetCursorFuture = null; future.completeExceptionally( new SubscriptionBusyException("Failed to disconnect consumers from subscription")); return; @@ -835,6 +843,7 @@ public void resetComplete(Object ctx) { dispatcher.cursorIsReset(); } IS_FENCED_UPDATER.set(PersistentSubscription.this, FALSE); + inProgressResetCursorFuture = null; future.complete(null); } @@ -843,6 +852,7 @@ public void resetFailed(ManagedLedgerException exception, Object ctx) { log.error("[{}][{}] Failed to reset subscription to position {}", topicName, subName, finalPosition, exception); IS_FENCED_UPDATER.set(PersistentSubscription.this, FALSE); + inProgressResetCursorFuture = null; // todo - retry on InvalidCursorPositionException // or should we just ask user to retry one more time? if (exception instanceof InvalidCursorPositionException) { @@ -857,10 +867,12 @@ public void resetFailed(ManagedLedgerException exception, Object ctx) { }).exceptionally((e) -> { log.error("[{}][{}] Error while resetting cursor", topicName, subName, e); IS_FENCED_UPDATER.set(PersistentSubscription.this, FALSE); + inProgressResetCursorFuture = null; future.completeExceptionally(new BrokerServiceException(e)); return null; }); }); + return future; } @Override 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 fd08f284bbf99..12d34dc5d240c 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 @@ -34,12 +34,14 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; import lombok.Cleanup; import lombok.extern.slf4j.Slf4j; 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.client.api.InjectedClientCnxClientBuilder; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageRoutingMode; @@ -50,8 +52,12 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.BatchMessageIdImpl; +import org.apache.pulsar.client.impl.ClientBuilderImpl; +import org.apache.pulsar.client.impl.ClientCnx; import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.common.api.proto.CommandError; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.util.RelativeTimeUtil; import org.awaitility.Awaitility; import org.testng.annotations.AfterClass; @@ -781,6 +787,64 @@ public void testSeekByFunctionAndMultiTopic() throws Exception { assertEquals(count, (msgInTopic1Partition0 + msgInTopic1Partition1 + msgInTopic1Partition2) * 2); } + @Test + public void testSeekWillNotEncounteredFencedError() throws Exception { + String topicName = "persistent://prop/ns-abc/my-topic2"; + admin.topics().createNonPartitionedTopic(topicName); + admin.topicPolicies().setRetention(topicName, new RetentionPolicies(3600, 0)); + // Create a pulsar client with a subscription fenced counter. + ClientBuilderImpl clientBuilder = (ClientBuilderImpl) PulsarClient.builder().serviceUrl(lookupUrl.toString()); + AtomicInteger receivedFencedErrorCounter = new AtomicInteger(); + PulsarClient client = InjectedClientCnxClientBuilder.create(clientBuilder, (conf, eventLoopGroup) -> + new ClientCnx(conf, eventLoopGroup) { + protected void handleError(CommandError error) { + if (error.getMessage() != null && error.getMessage().contains("Subscription is fenced")) { + receivedFencedErrorCounter.incrementAndGet(); + } + super.handleError(error); + } + }); + + // publish some messages. + org.apache.pulsar.client.api.Consumer consumer = client.newConsumer(Schema.STRING) + .topic(topicName) + .subscriptionName("s1") + .subscribe(); + Producer producer = client.newProducer(Schema.STRING) + .topic(topicName).create(); + MessageIdImpl msgId1 = (MessageIdImpl) producer.send("0"); + for (int i = 1; i < 11; i++) { + admin.topics().unload(topicName); + producer.send(i + ""); + } + + // Inject a delay for reset-cursor. + mockZooKeeper.delay(3000, (op, path) -> { + if (path.equals("/managed-ledgers/prop/ns-abc/persistent/my-topic2/s1")) { + return op.toString().equalsIgnoreCase("SET"); + } + return false; + }); + + // Verify: consumer will not receive "subscription fenced" error after a seek. + for (int i = 1; i < 11; i++) { + Message msg = consumer.receive(2, TimeUnit.SECONDS); + assertNotNull(msg); + consumer.acknowledge(msg); + } + consumer.seek(msgId1); + Awaitility.await().untilAsserted(() -> { + assertTrue(consumer.isConnected()); + }); + assertEquals(receivedFencedErrorCounter.get(), 0); + + // cleanup. + producer.close(); + consumer.close(); + client.close(); + admin.topics().delete(topicName); + } + @Test public void testExceptionBySeekFunction() throws Exception { final String topicName = "persistent://prop/use/ns-abc/test" + UUID.randomUUID(); From bb6c14f403146e1c66772eee45fa88f4674987ca Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Thu, 15 Aug 2024 11:57:52 +0800 Subject: [PATCH 670/699] [fix][broker] Fix shadow topics cannot be consumed when the entry is not cached (#23147) For shadow topics, a `ReadOnlyLedgerHandle` is created to read messages from the source topic when the entry is not cached. However, it leverages the `readAsync` API that validates the `lastAddConfirmed` field (LAC). In `ReadOnlyLedgerHandle`, this field could never be updated, so `readAsync` could fail immediately. See `LedgerHandle#readAsync`: ```java if (lastEntry > lastAddConfirmed) { LOG.error("ReadAsync exception on ledgerId:{} firstEntry:{} lastEntry:{} lastAddConfirmed:{}", ledgerId, firstEntry, lastEntry, lastAddConfirmed); return FutureUtils.exception(new BKReadException()); } ``` This bug is not exposed because: 1. `PulsarMockReadHandle` does not maintain a LAC field. 2. The case for cache miss is never tested. Replace `readAsync` with `readUnconfirmedAsync` and compare the entry range with the `ManagedLedger#getLastConfirmedEntry`. The managed ledger already maintains a `lastConfirmedEntry` to limit the last entry. See `ManagedLedgerImpl#internalReadFromLedger`: ```java Position lastPosition = lastConfirmedEntry; if (ledger.getId() == lastPosition.getLedgerId()) { lastEntryInLedger = lastPosition.getEntryId(); ``` Add `ShadowTopicRealBkTest` to cover two code changes `RangeEntryCacheImpl#readFromStorage` and `EntryCache#asyncReadEntry`. Exceptionally, compare the entry range with the LAC of a ledger handle when it does not exist in the managed ledger. It's because `ReadOnlyManagedLedgerImpl` could read a ledger in another managed ledger. - [ ] `doc` - [ ] `doc-required` - [x] `doc-not-needed` - [ ] `doc-complete` PR in forked repository: https://github.com/BewareMyPower/pulsar/pull/33 (cherry picked from commit 15b88d250818bada5c1a94f5c54ef7806f88a500) (cherry picked from commit 14b3672c08838d006e32008bec8ed52d889b4859) --- .../mledger/impl/ManagedLedgerImpl.java | 2 + .../impl/cache/EntryCacheDisabled.java | 4 +- .../impl/cache/RangeEntryCacheImpl.java | 4 +- .../mledger/impl/cache/ReadEntryUtils.java | 54 +++++ .../mledger/impl/EntryCacheManagerTest.java | 7 +- .../mledger/impl/EntryCacheTest.java | 187 +++++++----------- .../mledger/impl/OffloadPrefixReadTest.java | 2 +- .../persistent/ShadowTopicRealBkTest.java | 109 ++++++++++ 8 files changed, 252 insertions(+), 117 deletions(-) create mode 100644 managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/ReadEntryUtils.java create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ShadowTopicRealBkTest.java 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 1ffa275c2ce64..a540828000e1c 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 @@ -4100,6 +4100,8 @@ public static ManagedLedgerException createManagedLedgerException(int bkErrorCod public static ManagedLedgerException createManagedLedgerException(Throwable t) { if (t instanceof org.apache.bookkeeper.client.api.BKException) { return createManagedLedgerException(((org.apache.bookkeeper.client.api.BKException) t).getCode()); + } else if (t instanceof ManagedLedgerException) { + return (ManagedLedgerException) t; } else if (t instanceof CompletionException && !(t.getCause() instanceof CompletionException) /* check to avoid stackoverlflow */) { return createManagedLedgerException(t.getCause()); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDisabled.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDisabled.java index d1050e0062826..64595dfe47e66 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDisabled.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/EntryCacheDisabled.java @@ -79,7 +79,7 @@ public void invalidateEntriesBeforeTimestamp(long timestamp) { @Override public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, boolean isSlowestReader, final AsyncCallbacks.ReadEntriesCallback callback, Object ctx) { - lh.readAsync(firstEntry, lastEntry).thenAcceptAsync( + ReadEntryUtils.readAsync(ml, lh, firstEntry, lastEntry).thenAcceptAsync( ledgerEntries -> { List entries = new ArrayList<>(); long totalSize = 0; @@ -107,7 +107,7 @@ public void asyncReadEntry(ReadHandle lh, long firstEntry, long lastEntry, boole @Override public void asyncReadEntry(ReadHandle lh, PositionImpl position, AsyncCallbacks.ReadEntryCallback callback, Object ctx) { - lh.readAsync(position.getEntryId(), position.getEntryId()).whenCompleteAsync( + ReadEntryUtils.readAsync(ml, lh, position.getEntryId(), position.getEntryId()).whenCompleteAsync( (ledgerEntries, exception) -> { if (exception != null) { ml.invalidateLedgerHandle(lh); 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 27aec6f178e39..21eb62e5a8caa 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 @@ -248,7 +248,7 @@ private void asyncReadEntry0(ReadHandle lh, PositionImpl position, final ReadEnt manager.mlFactoryMBean.recordCacheHit(cachedEntry.getLength()); callback.readEntryComplete(cachedEntry, ctx); } else { - lh.readAsync(position.getEntryId(), position.getEntryId()).thenAcceptAsync( + ReadEntryUtils.readAsync(ml, lh, position.getEntryId(), position.getEntryId()).thenAcceptAsync( ledgerEntries -> { try { Iterator iterator = ledgerEntries.iterator(); @@ -428,7 +428,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { CompletableFuture> readFromStorage(ReadHandle lh, long firstEntry, long lastEntry, boolean shouldCacheEntry) { final int entriesToRead = (int) (lastEntry - firstEntry) + 1; - CompletableFuture> readResult = lh.readAsync(firstEntry, lastEntry) + CompletableFuture> readResult = ReadEntryUtils.readAsync(ml, lh, firstEntry, lastEntry) .thenApply( ledgerEntries -> { requireNonNull(ml.getName()); diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/ReadEntryUtils.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/ReadEntryUtils.java new file mode 100644 index 0000000000000..5cf5f053f0ce7 --- /dev/null +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/cache/ReadEntryUtils.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.bookkeeper.mledger.impl.cache; + +import java.util.concurrent.CompletableFuture; +import org.apache.bookkeeper.client.api.LedgerEntries; +import org.apache.bookkeeper.client.api.ReadHandle; +import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerException; + +class ReadEntryUtils { + + static CompletableFuture readAsync(ManagedLedger ml, ReadHandle handle, long firstEntry, + long lastEntry) { + if (ml.getOptionalLedgerInfo(handle.getId()).isEmpty()) { + // The read handle comes from another managed ledger, in this case, we can only compare the entry range with + // the LAC of that read handle. Specifically, it happens when this method is called by a + // ReadOnlyManagedLedgerImpl object. + return handle.readAsync(firstEntry, lastEntry); + } + // Compare the entry range with the lastConfirmedEntry maintained by the managed ledger because the entry cache + // of `ShadowManagedLedgerImpl` reads entries via `ReadOnlyLedgerHandle`, which never updates `lastAddConfirmed` + final var lastConfirmedEntry = ml.getLastConfirmedEntry(); + if (lastConfirmedEntry == null) { + return CompletableFuture.failedFuture(new ManagedLedgerException( + "LastConfirmedEntry is null when reading ledger " + handle.getId())); + } + if (handle.getId() > lastConfirmedEntry.getLedgerId()) { + return CompletableFuture.failedFuture(new ManagedLedgerException("LastConfirmedEntry is " + + lastConfirmedEntry + " when reading ledger " + handle.getId())); + } + if (handle.getId() == lastConfirmedEntry.getLedgerId() && lastEntry > lastConfirmedEntry.getEntryId()) { + return CompletableFuture.failedFuture(new ManagedLedgerException("LastConfirmedEntry is " + + lastConfirmedEntry + " when reading entry " + lastEntry)); + } + return handle.readUnconfirmedAsync(firstEntry, lastEntry); + } +} diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java index 1ab3198498ac3..de8c6f5d7d075 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheManagerTest.java @@ -28,6 +28,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.Optional; import java.util.Random; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -43,6 +44,7 @@ import org.apache.bookkeeper.mledger.impl.cache.EntryCache; import org.apache.bookkeeper.mledger.impl.cache.EntryCacheDisabled; import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager; +import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; import org.testng.Assert; import org.testng.annotations.Test; @@ -390,6 +392,9 @@ void entryCacheDisabledAsyncReadEntry() throws Exception { EntryCache entryCache = cacheManager.getEntryCache(ml1); final CountDownLatch counter = new CountDownLatch(1); + when(ml1.getLastConfirmedEntry()).thenReturn(new PositionImpl(1L, 1L)); + when(ml1.getOptionalLedgerInfo(lh.getId())).thenReturn(Optional.of(mock( + MLDataFormats.ManagedLedgerInfo.LedgerInfo.class))); entryCache.asyncReadEntry(lh, new PositionImpl(1L,1L), new AsyncCallbacks.ReadEntryCallback() { public void readEntryComplete(Entry entry, Object ctx) { Assert.assertNotEquals(entry, null); @@ -404,7 +409,7 @@ public void readEntryFailed(ManagedLedgerException exception, Object ctx) { }, null); counter.await(); - verify(lh).readAsync(anyLong(), anyLong()); + verify(lh).readUnconfirmedAsync(anyLong(), anyLong()); } } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheTest.java index c8338798f271b..f6f395f777310 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/EntryCacheTest.java @@ -25,14 +25,15 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; - +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; import io.netty.buffer.Unpooled; - import java.util.ArrayList; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CountDownLatch; - +import java.util.concurrent.ExecutionException; +import java.util.function.Consumer; import lombok.Cleanup; import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException; import org.apache.bookkeeper.client.api.LedgerEntries; @@ -45,8 +46,8 @@ import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.impl.cache.EntryCache; import org.apache.bookkeeper.mledger.impl.cache.EntryCacheManager; +import org.apache.bookkeeper.mledger.proto.MLDataFormats; import org.apache.bookkeeper.test.MockedBookKeeperTestCase; -import org.testng.Assert; import org.testng.annotations.Test; public class EntryCacheTest extends MockedBookKeeperTestCase { @@ -60,6 +61,8 @@ protected void setUpTestCase() throws Exception { when(ml.getExecutor()).thenReturn(executor); when(ml.getMbean()).thenReturn(new ManagedLedgerMBeanImpl(ml)); when(ml.getConfig()).thenReturn(new ManagedLedgerConfig()); + when(ml.getOptionalLedgerInfo(0L)).thenReturn(Optional.of(mock( + MLDataFormats.ManagedLedgerInfo.LedgerInfo.class))); } @Test(timeOut = 5000) @@ -76,22 +79,13 @@ public void testRead() throws Exception { entryCache.insert(EntryImpl.create(0, i, data)); } - final CountDownLatch counter = new CountDownLatch(1); - - entryCache.asyncReadEntry(lh, 0, 9, false, new ReadEntriesCallback() { - public void readEntriesComplete(List entries, Object ctx) { - assertEquals(entries.size(), 10); - entries.forEach(Entry::release); - counter.countDown(); - } - - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - Assert.fail("should not have failed"); - } - }, null); - counter.await(); + when(ml.getLastConfirmedEntry()).thenReturn(new PositionImpl(0, 9)); + final var entries = readEntry(entryCache, lh, 0, 9, false, null); + assertEquals(entries.size(), 10); + entries.forEach(Entry::release); // Verify no entries were read from bookkeeper + verify(lh, never()).readUnconfirmedAsync(anyLong(), anyLong()); verify(lh, never()).readAsync(anyLong(), anyLong()); } @@ -109,19 +103,9 @@ public void testReadMissingBefore() throws Exception { entryCache.insert(EntryImpl.create(0, i, data)); } - final CountDownLatch counter = new CountDownLatch(1); - - entryCache.asyncReadEntry(lh, 0, 9, false, new ReadEntriesCallback() { - public void readEntriesComplete(List entries, Object ctx) { - assertEquals(entries.size(), 10); - counter.countDown(); - } - - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - Assert.fail("should not have failed"); - } - }, null); - counter.await(); + when(ml.getLastConfirmedEntry()).thenReturn(new PositionImpl(0, 9)); + final var entries = readEntry(entryCache, lh, 0, 9, false, null); + assertEquals(entries.size(), 10); } @Test(timeOut = 5000) @@ -138,19 +122,9 @@ public void testReadMissingAfter() throws Exception { entryCache.insert(EntryImpl.create(0, i, data)); } - final CountDownLatch counter = new CountDownLatch(1); - - entryCache.asyncReadEntry(lh, 0, 9, false, new ReadEntriesCallback() { - public void readEntriesComplete(List entries, Object ctx) { - assertEquals(entries.size(), 10); - counter.countDown(); - } - - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - Assert.fail("should not have failed"); - } - }, null); - counter.await(); + when(ml.getLastConfirmedEntry()).thenReturn(new PositionImpl(0, 9)); + final var entries = readEntry(entryCache, lh, 0, 9, false, null); + assertEquals(entries.size(), 10); } @Test(timeOut = 5000) @@ -168,19 +142,9 @@ public void testReadMissingMiddle() throws Exception { entryCache.insert(EntryImpl.create(0, 8, data)); entryCache.insert(EntryImpl.create(0, 9, data)); - final CountDownLatch counter = new CountDownLatch(1); - - entryCache.asyncReadEntry(lh, 0, 9, false, new ReadEntriesCallback() { - public void readEntriesComplete(List entries, Object ctx) { - assertEquals(entries.size(), 10); - counter.countDown(); - } - - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - Assert.fail("should not have failed"); - } - }, null); - counter.await(); + when(ml.getLastConfirmedEntry()).thenReturn(new PositionImpl(0, 9)); + final var entries = readEntry(entryCache, lh, 0, 9, false, null); + assertEquals(entries.size(), 10); } @Test(timeOut = 5000) @@ -198,19 +162,9 @@ public void testReadMissingMultiple() throws Exception { entryCache.insert(EntryImpl.create(0, 5, data)); entryCache.insert(EntryImpl.create(0, 8, data)); - final CountDownLatch counter = new CountDownLatch(1); - - entryCache.asyncReadEntry(lh, 0, 9, false, new ReadEntriesCallback() { - public void readEntriesComplete(List entries, Object ctx) { - assertEquals(entries.size(), 10); - counter.countDown(); - } - - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - Assert.fail("should not have failed"); - } - }, null); - counter.await(); + when(ml.getLastConfirmedEntry()).thenReturn(new PositionImpl(0, 9)); + final var entries = readEntry(entryCache, lh, 0, 9, false, null); + assertEquals(entries.size(), 10); } @Test @@ -222,19 +176,25 @@ public void testCachedReadReturnsDifferentByteBuffer() throws Exception { @Cleanup(value = "clear") EntryCache entryCache = cacheManager.getEntryCache(ml); - CompletableFuture> cacheMissFutureEntries = new CompletableFuture<>(); - - entryCache.asyncReadEntry(lh, 0, 1, true, new ReadEntriesCallback() { - public void readEntriesComplete(List entries, Object ctx) { - cacheMissFutureEntries.complete(entries); - } - - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - cacheMissFutureEntries.completeExceptionally(exception); - } - }, null); - - List cacheMissEntries = cacheMissFutureEntries.get(); + readEntry(entryCache, lh, 0, 1, true, e -> { + assertTrue(e instanceof ManagedLedgerException); + assertTrue(e.getMessage().contains("LastConfirmedEntry is null when reading ledger 0")); + }); + + when(ml.getLastConfirmedEntry()).thenReturn(new PositionImpl(-1, -1)); + readEntry(entryCache, lh, 0, 1, true, e -> { + assertTrue(e instanceof ManagedLedgerException); + assertTrue(e.getMessage().contains("LastConfirmedEntry is -1:-1 when reading ledger 0")); + }); + + when(ml.getLastConfirmedEntry()).thenReturn(new PositionImpl(0, 0)); + readEntry(entryCache, lh, 0, 1, true, e -> { + assertTrue(e instanceof ManagedLedgerException); + assertTrue(e.getMessage().contains("LastConfirmedEntry is 0:0 when reading entry 1")); + }); + + when(ml.getLastConfirmedEntry()).thenReturn(new PositionImpl(0, 1)); + List cacheMissEntries = readEntry(entryCache, lh, 0, 1, true, null); // Ensure first entry is 0 and assertEquals(cacheMissEntries.size(), 2); assertEquals(cacheMissEntries.get(0).getEntryId(), 0); @@ -243,19 +203,7 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { // Move the reader index to simulate consumption cacheMissEntries.get(0).getDataBuffer().readerIndex(10); - CompletableFuture> cacheHitFutureEntries = new CompletableFuture<>(); - - entryCache.asyncReadEntry(lh, 0, 1, true, new ReadEntriesCallback() { - public void readEntriesComplete(List entries, Object ctx) { - cacheHitFutureEntries.complete(entries); - } - - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - cacheHitFutureEntries.completeExceptionally(exception); - } - }, null); - - List cacheHitEntries = cacheHitFutureEntries.get(); + List cacheHitEntries = readEntry(entryCache, lh, 0, 1, true, null); assertEquals(cacheHitEntries.get(0).getEntryId(), 0); assertEquals(cacheHitEntries.get(0).getDataBuffer().readerIndex(), 0); } @@ -269,7 +217,7 @@ public void testReadWithError() throws Exception { CompletableFuture future = new CompletableFuture<>(); future.completeExceptionally(new BKNoSuchLedgerExistsException()); return future; - }).when(lh).readAsync(anyLong(), anyLong()); + }).when(lh).readUnconfirmedAsync(anyLong(), anyLong()); EntryCacheManager cacheManager = factory.getEntryCacheManager(); @Cleanup(value = "clear") @@ -278,18 +226,9 @@ public void testReadWithError() throws Exception { byte[] data = new byte[10]; entryCache.insert(EntryImpl.create(0, 2, data)); - final CountDownLatch counter = new CountDownLatch(1); - - entryCache.asyncReadEntry(lh, 0, 9, false, new ReadEntriesCallback() { - public void readEntriesComplete(List entries, Object ctx) { - Assert.fail("should not complete"); - } - - public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { - counter.countDown(); - } - }, null); - counter.await(); + when(ml.getLastConfirmedEntry()).thenReturn(new PositionImpl(0, 9)); + readEntry(entryCache, lh, 0, 9, false, e -> + assertTrue(e instanceof ManagedLedgerException.LedgerNotExistException)); } static ReadHandle getLedgerHandle() { @@ -306,9 +245,35 @@ static ReadHandle getLedgerHandle() { LedgerEntries ledgerEntries = mock(LedgerEntries.class); doAnswer((invocation2) -> entries.iterator()).when(ledgerEntries).iterator(); return CompletableFuture.completedFuture(ledgerEntries); - }).when(lh).readAsync(anyLong(), anyLong()); + }).when(lh).readUnconfirmedAsync(anyLong(), anyLong()); return lh; } + private List readEntry(EntryCache entryCache, ReadHandle lh, long firstEntry, long lastEntry, + boolean shouldCacheEntry, Consumer assertion) + throws InterruptedException { + final var future = new CompletableFuture>(); + entryCache.asyncReadEntry(lh, firstEntry, lastEntry, shouldCacheEntry, new ReadEntriesCallback() { + @Override + public void readEntriesComplete(List entries, Object ctx) { + future.complete(entries); + } + + @Override + public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { + future.completeExceptionally(exception); + } + }, null); + try { + final var entries = future.get(); + assertNull(assertion); + return entries; + } catch (ExecutionException e) { + if (assertion != null) { + assertion.accept(e.getCause()); + } + return List.of(); + } + } } diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java index 00c50249b4ac2..59e815fb1b4a5 100644 --- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java +++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/OffloadPrefixReadTest.java @@ -322,7 +322,7 @@ public CompletableFuture readAsync(long firstEntry, long lastEntr @Override public CompletableFuture readUnconfirmedAsync(long firstEntry, long lastEntry) { - return unsupported(); + return readAsync(firstEntry, lastEntry); } @Override diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ShadowTopicRealBkTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ShadowTopicRealBkTest.java new file mode 100644 index 0000000000000..9d810b06a7c7b --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/ShadowTopicRealBkTest.java @@ -0,0 +1,109 @@ +/* + * 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.collect.Lists; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import org.apache.bookkeeper.mledger.impl.ShadowManagedLedgerImpl; +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.SubscriptionInitialPosition; +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.PortManager; +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; + +public class ShadowTopicRealBkTest { + + private static final String cluster = "test"; + private final int zkPort = PortManager.nextLockedFreePort(); + private final LocalBookkeeperEnsemble bk = new LocalBookkeeperEnsemble(2, zkPort, PortManager::nextLockedFreePort); + private PulsarService pulsar; + private PulsarAdmin admin; + + @BeforeClass + public void setup() throws Exception { + bk.start(); + final var config = new ServiceConfiguration(); + config.setClusterName(cluster); + config.setAdvertisedAddress("localhost"); + config.setBrokerServicePort(Optional.of(0)); + config.setWebServicePort(Optional.of(0)); + config.setMetadataStoreUrl("zk:localhost:" + zkPort); + pulsar = new PulsarService(config); + pulsar.start(); + admin = pulsar.getAdminClient(); + admin.clusters().createCluster("test", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()) + .brokerServiceUrl(pulsar.getBrokerServiceUrl()).build()); + admin.tenants().createTenant("public", TenantInfo.builder().allowedClusters(Set.of(cluster)).build()); + admin.namespaces().createNamespace("public/default"); + } + + @AfterClass(alwaysRun = true) + public void cleanup() throws Exception { + if (pulsar != null) { + pulsar.close(); + } + bk.stop(); + } + + @Test + public void testReadFromStorage() throws Exception { + final var sourceTopic = TopicName.get("test-read-from-source").toString(); + final var shadowTopic = sourceTopic + "-shadow"; + + admin.topics().createNonPartitionedTopic(sourceTopic); + admin.topics().createShadowTopic(shadowTopic, sourceTopic); + admin.topics().setShadowTopics(sourceTopic, Lists.newArrayList(shadowTopic)); + + Awaitility.await().atMost(5, TimeUnit.SECONDS).untilAsserted(()->{ + final var sourcePersistentTopic = (PersistentTopic) pulsar.getBrokerService() + .getTopicIfExists(sourceTopic).get().orElseThrow(); + final var replicator = (ShadowReplicator) sourcePersistentTopic.getShadowReplicators().get(shadowTopic); + Assert.assertNotNull(replicator); + Assert.assertEquals(String.valueOf(replicator.getState()), "Started"); + }); + + final var client = pulsar.getClient(); + // When the message was sent, there is no cursor, so it will read from the cache + final var producer = client.newProducer().topic(sourceTopic).create(); + producer.send("message".getBytes()); + // 1. Verify RangeEntryCacheImpl#readFromStorage + final var consumer = client.newConsumer().topic(shadowTopic).subscriptionName("sub") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest).subscribe(); + final var msg = consumer.receive(5, TimeUnit.SECONDS); + Assert.assertNotNull(msg); + Assert.assertEquals(msg.getValue(), "message".getBytes()); + + // 2. Verify EntryCache#asyncReadEntry + final var shadowManagedLedger = ((PersistentTopic) pulsar.getBrokerService().getTopicIfExists(shadowTopic).get() + .orElseThrow()).getManagedLedger(); + Assert.assertTrue(shadowManagedLedger instanceof ShadowManagedLedgerImpl); + shadowManagedLedger.getEarliestMessagePublishTimeInBacklog().get(3, TimeUnit.SECONDS); + } +} From ef00bb2f55406f905b92e813006afa351c3fb0bf Mon Sep 17 00:00:00 2001 From: Andrey Yegorov <8622884+dlg99@users.noreply.github.com> Date: Thu, 15 Aug 2024 10:33:56 -0700 Subject: [PATCH 671/699] [fix] DLQ to handle bytes key properly (#23172) (cherry picked from commit 46c25ac73427312db7f38e150cd797a8cee23f28) (cherry picked from commit 79cae0a5c678c5cb599b0572399039039877ca91) --- .../client/api/DeadLetterTopicTest.java | 60 +++++++++++++++++++ .../pulsar/client/impl/ConsumerImpl.java | 56 +++++++++-------- 2 files changed, 92 insertions(+), 24 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 2a0cb3187d208..143b463fd3b3e 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 @@ -137,6 +137,66 @@ public void testDeadLetterTopicWithMessageKey() throws Exception { consumer.close(); } + @Test + public void testDeadLetterTopicWithBinaryMessageKey() throws Exception { + final String topic = "persistent://my-property/my-ns/dead-letter-topic"; + + final int maxRedeliveryCount = 1; + + final int sendMessages = 100; + + Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) + .topic(topic) + .subscriptionName("my-subscription") + .subscriptionType(SubscriptionType.Shared) + .ackTimeout(1, TimeUnit.SECONDS) + .deadLetterPolicy(DeadLetterPolicy.builder().maxRedeliverCount(maxRedeliveryCount).build()) + .receiverQueueSize(100) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + + @Cleanup + PulsarClient newPulsarClient = newPulsarClient(lookupUrl.toString(), 0);// Creates new client connection + Consumer deadLetterConsumer = newPulsarClient.newConsumer(Schema.BYTES) + .topic("persistent://my-property/my-ns/dead-letter-topic-my-subscription-DLQ") + .subscriptionName("my-subscription") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + + Producer producer = pulsarClient.newProducer(Schema.BYTES) + .topic(topic) + .create(); + + byte[] key = new byte[]{1, 2, 3, 4}; + for (int i = 0; i < sendMessages; i++) { + producer.newMessage() + .keyBytes(key) + .value(String.format("Hello Pulsar [%d]", i).getBytes()) + .send(); + } + + producer.close(); + + int totalReceived = 0; + do { + Message message = consumer.receive(); + log.info("consumer received message : {} {}", message.getMessageId(), new String(message.getData())); + totalReceived++; + } while (totalReceived < sendMessages * (maxRedeliveryCount + 1)); + + int totalInDeadLetter = 0; + do { + Message message = deadLetterConsumer.receive(); + assertEquals(message.getKeyBytes(), key); + log.info("dead letter consumer received message : {} {}", message.getMessageId(), new String(message.getData())); + deadLetterConsumer.acknowledge(message); + totalInDeadLetter++; + } while (totalInDeadLetter < sendMessages); + + deadLetterConsumer.close(); + 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/ConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerImpl.java index 65df1154af6ad..83c0747228c23 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 @@ -220,6 +220,7 @@ public class ConsumerImpl extends ConsumerBase implements ConnectionHandle private final AtomicReference clientCnxUsedForConsumerRegistration = new AtomicReference<>(); private final List previousExceptions = new CopyOnWriteArrayList(); private volatile boolean hasSoughtByTimestamp = false; + static ConsumerImpl newConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurationData conf, @@ -263,10 +264,12 @@ static ConsumerImpl newConsumerImpl(PulsarClientImpl client, } protected ConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurationData conf, - ExecutorProvider executorProvider, int partitionIndex, boolean hasParentConsumer, - boolean parentConsumerHasListener, CompletableFuture> subscribeFuture, MessageId startMessageId, - long startMessageRollbackDurationInSec, Schema schema, ConsumerInterceptors interceptors, - boolean createTopicIfDoesNotExist) { + ExecutorProvider executorProvider, int partitionIndex, boolean hasParentConsumer, + boolean parentConsumerHasListener, CompletableFuture> subscribeFuture, + MessageId startMessageId, + long startMessageRollbackDurationInSec, Schema schema, + ConsumerInterceptors interceptors, + boolean createTopicIfDoesNotExist) { super(client, topic, conf, conf.getReceiverQueueSize(), executorProvider, subscribeFuture, schema, interceptors); this.consumerId = client.newConsumerId(); @@ -338,21 +341,21 @@ protected ConsumerImpl(PulsarClientImpl client, String topic, ConsumerConfigurat } this.connectionHandler = new ConnectionHandler(this, - new BackoffBuilder() - .setInitialTime(client.getConfiguration().getInitialBackoffIntervalNanos(), - TimeUnit.NANOSECONDS) - .setMax(client.getConfiguration().getMaxBackoffIntervalNanos(), TimeUnit.NANOSECONDS) - .setMandatoryStop(0, TimeUnit.MILLISECONDS) - .create(), + new BackoffBuilder() + .setInitialTime(client.getConfiguration().getInitialBackoffIntervalNanos(), + TimeUnit.NANOSECONDS) + .setMax(client.getConfiguration().getMaxBackoffIntervalNanos(), TimeUnit.NANOSECONDS) + .setMandatoryStop(0, TimeUnit.MILLISECONDS) + .create(), this); this.topicName = TopicName.get(topic); if (this.topicName.isPersistent()) { this.acknowledgmentsGroupingTracker = - new PersistentAcknowledgmentsGroupingTracker(this, conf, client.eventLoopGroup()); + new PersistentAcknowledgmentsGroupingTracker(this, conf, client.eventLoopGroup()); } else { this.acknowledgmentsGroupingTracker = - NonPersistentAcknowledgmentGroupingTracker.of(); + NonPersistentAcknowledgmentGroupingTracker.of(); } if (conf.getDeadLetterPolicy() != null) { @@ -430,16 +433,16 @@ public CompletableFuture unsubscribeAsync() { log.error("[{}][{}] Failed to unsubscribe: {}", topic, subscription, e.getCause().getMessage()); setState(State.Ready); unsubscribeFuture.completeExceptionally( - PulsarClientException.wrap(e.getCause(), - String.format("Failed to unsubscribe the subscription %s of topic %s", - subscription, topicName.toString()))); + PulsarClientException.wrap(e.getCause(), + String.format("Failed to unsubscribe the subscription %s of topic %s", + subscription, topicName.toString()))); return null; }); } else { unsubscribeFuture.completeExceptionally( - new PulsarClientException( - String.format("The client is not connected to the broker when unsubscribing the " - + "subscription %s of the topic %s", subscription, topicName.toString()))); + new PulsarClientException( + String.format("The client is not connected to the broker when unsubscribing the " + + "subscription %s of the topic %s", subscription, topicName.toString()))); } return unsubscribeFuture; } @@ -592,6 +595,15 @@ protected CompletableFuture doAcknowledge(List messageIdList, A } } + private static void copyMessageKeyIfNeeded(Message message, TypedMessageBuilder typedMessageBuilderNew) { + if (message.hasKey()) { + if (message.hasBase64EncodedKey()) { + typedMessageBuilderNew.keyBytes(message.getKeyBytes()); + } else { + typedMessageBuilderNew.key(message.getKey()); + } + } + } @SuppressWarnings("unchecked") @Override @@ -674,9 +686,7 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a if (delayTime > 0) { typedMessageBuilderNew.deliverAfter(delayTime, unit); } - if (message.hasKey()) { - typedMessageBuilderNew.key(message.getKey()); - } + copyMessageKeyIfNeeded(message, typedMessageBuilderNew); typedMessageBuilderNew.sendAsync() .thenCompose(__ -> doAcknowledge(finalMessageId, ackType, Collections.emptyMap(), null)) .thenAccept(v -> result.complete(null)) @@ -2125,9 +2135,7 @@ private CompletableFuture processPossibleToDLQ(MessageIdAdv messageId) producerDLQ.newMessage(Schema.AUTO_PRODUCE_BYTES(message.getReaderSchema().get())) .value(message.getData()) .properties(getPropertiesMap(message, originMessageIdStr, originTopicNameStr)); - if (message.hasKey()) { - typedMessageBuilderNew.key(message.getKey()); - } + copyMessageKeyIfNeeded(message, typedMessageBuilderNew); typedMessageBuilderNew.sendAsync() .thenAccept(messageIdInDLQ -> { possibleSendToDeadLetterTopicMessages.remove(messageId); From 42e9157a7e033b49b121a46a7fc7e99668ba6ae6 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 16 Aug 2024 06:33:18 +0300 Subject: [PATCH 672/699] [fix][client] Copy orderingKey to retry letter topic and DLQ messages and fix bug in copying (#23182) Fixes #23173 Fixes #23181 See #23173 and #23181 - copy ordering key to messages sent to retry letter topic and DLQ topic (cherry picked from commit 67fc5b9f5342bd35d3fdacf37cf172a629ee15f9) (cherry picked from commit c83428fa328b327c9e7e1ae48878e8ddcd99a5db) --- .../client/api/DeadLetterTopicTest.java | 60 +++++++++++++++++++ .../pulsar/client/api/RetryTopicTest.java | 17 +++++- .../pulsar/client/impl/ConsumerImpl.java | 10 +++- 3 files changed, 83 insertions(+), 4 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 143b463fd3b3e..dd36d4fdc4db0 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 @@ -197,6 +197,66 @@ public void testDeadLetterTopicWithBinaryMessageKey() throws Exception { consumer.close(); } + @Test + public void testDeadLetterTopicMessagesWithOrderingKey() throws Exception { + final String topic = "persistent://my-property/my-ns/dead-letter-topic"; + + final int maxRedeliveryCount = 1; + + final int sendMessages = 100; + + Consumer consumer = pulsarClient.newConsumer(Schema.BYTES) + .topic(topic) + .subscriptionName("my-subscription") + .subscriptionType(SubscriptionType.Shared) + .ackTimeout(1, TimeUnit.SECONDS) + .deadLetterPolicy(DeadLetterPolicy.builder().maxRedeliverCount(maxRedeliveryCount).build()) + .receiverQueueSize(100) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + + @Cleanup + PulsarClient newPulsarClient = newPulsarClient(lookupUrl.toString(), 0);// Creates new client connection + Consumer deadLetterConsumer = newPulsarClient.newConsumer(Schema.BYTES) + .topic("persistent://my-property/my-ns/dead-letter-topic-my-subscription-DLQ") + .subscriptionName("my-subscription") + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + + Producer producer = pulsarClient.newProducer(Schema.BYTES) + .topic(topic) + .create(); + + byte[] key = new byte[]{1, 2, 3, 4}; + for (int i = 0; i < sendMessages; i++) { + producer.newMessage() + .orderingKey(key) + .value(String.format("Hello Pulsar [%d]", i).getBytes()) + .send(); + } + + producer.close(); + + int totalReceived = 0; + do { + Message message = consumer.receive(); + log.info("consumer received message : {} {}", message.getMessageId(), new String(message.getData())); + totalReceived++; + } while (totalReceived < sendMessages * (maxRedeliveryCount + 1)); + + int totalInDeadLetter = 0; + do { + Message message = deadLetterConsumer.receive(); + assertEquals(message.getOrderingKey(), key); + log.info("dead letter consumer received message : {} {}", message.getMessageId(), new String(message.getData())); + deadLetterConsumer.acknowledge(message); + totalInDeadLetter++; + } while (totalInDeadLetter < sendMessages); + + deadLetterConsumer.close(); + consumer.close(); + } + @DataProvider(name = "produceLargeMessages") public Object[][] produceLargeMessages() { return new Object[][] { { false }, { true } }; 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 2ccae72143443..9cb82fde04118 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 @@ -257,6 +257,9 @@ public void testAutoConsumeSchemaRetryLetter() throws Exception { public void testRetryTopicProperties() throws Exception { final String topic = "persistent://my-property/my-ns/retry-topic"; + byte[] key = "key".getBytes(); + byte[] orderingKey = "orderingKey".getBytes(); + final int maxRedeliveryCount = 3; final int sendMessages = 10; @@ -285,7 +288,11 @@ public void testRetryTopicProperties() throws Exception { Set originMessageIds = new HashSet<>(); for (int i = 0; i < sendMessages; i++) { - MessageId msgId = producer.send(String.format("Hello Pulsar [%d]", i).getBytes()); + MessageId msgId = producer.newMessage() + .value(String.format("Hello Pulsar [%d]", i).getBytes()) + .keyBytes(key) + .orderingKey(orderingKey) + .send(); originMessageIds.add(msgId.toString()); } @@ -298,6 +305,10 @@ public void testRetryTopicProperties() throws Exception { if (message.hasProperty(RetryMessageUtil.SYSTEM_PROPERTY_RECONSUMETIMES)) { // check the REAL_TOPIC property assertEquals(message.getProperty(RetryMessageUtil.SYSTEM_PROPERTY_REAL_TOPIC), topic); + assertTrue(message.hasKey()); + assertEquals(message.getKeyBytes(), key); + assertTrue(message.hasOrderingKey()); + assertEquals(message.getOrderingKey(), orderingKey); retryMessageIds.add(message.getProperty(RetryMessageUtil.SYSTEM_PROPERTY_ORIGIN_MESSAGE_ID)); } consumer.reconsumeLater(message, 1, TimeUnit.SECONDS); @@ -317,6 +328,10 @@ public void testRetryTopicProperties() throws Exception { if (message.hasProperty(RetryMessageUtil.SYSTEM_PROPERTY_RECONSUMETIMES)) { // check the REAL_TOPIC property assertEquals(message.getProperty(RetryMessageUtil.SYSTEM_PROPERTY_REAL_TOPIC), topic); + assertTrue(message.hasKey()); + assertEquals(message.getKeyBytes(), key); + assertTrue(message.hasOrderingKey()); + assertEquals(message.getOrderingKey(), orderingKey); deadLetterMessageIds.add(message.getProperty(RetryMessageUtil.SYSTEM_PROPERTY_ORIGIN_MESSAGE_ID)); } deadLetterConsumer.acknowledge(message); 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 83c0747228c23..cf74e7ed5f08d 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 @@ -595,7 +595,7 @@ protected CompletableFuture doAcknowledge(List messageIdList, A } } - private static void copyMessageKeyIfNeeded(Message message, TypedMessageBuilder typedMessageBuilderNew) { + private static void copyMessageKeysIfNeeded(Message message, TypedMessageBuilder typedMessageBuilderNew) { if (message.hasKey()) { if (message.hasBase64EncodedKey()) { typedMessageBuilderNew.keyBytes(message.getKeyBytes()); @@ -603,6 +603,9 @@ private static void copyMessageKeyIfNeeded(Message message, TypedMessageBuild typedMessageBuilderNew.key(message.getKey()); } } + if (message.hasOrderingKey()) { + typedMessageBuilderNew.orderingKey(message.getOrderingKey()); + } } @SuppressWarnings("unchecked") @@ -660,6 +663,7 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a dlqProducer.newMessage(Schema.AUTO_PRODUCE_BYTES(retryMessage.getReaderSchema().get())) .value(retryMessage.getData()) .properties(propertiesMap); + copyMessageKeysIfNeeded(message, typedMessageBuilderNew); typedMessageBuilderNew.sendAsync().thenAccept(msgId -> { doAcknowledge(finalMessageId, ackType, Collections.emptyMap(), null).thenAccept(v -> { result.complete(null); @@ -686,7 +690,7 @@ protected CompletableFuture doReconsumeLater(Message message, AckType a if (delayTime > 0) { typedMessageBuilderNew.deliverAfter(delayTime, unit); } - copyMessageKeyIfNeeded(message, typedMessageBuilderNew); + copyMessageKeysIfNeeded(message, typedMessageBuilderNew); typedMessageBuilderNew.sendAsync() .thenCompose(__ -> doAcknowledge(finalMessageId, ackType, Collections.emptyMap(), null)) .thenAccept(v -> result.complete(null)) @@ -2135,7 +2139,7 @@ private CompletableFuture processPossibleToDLQ(MessageIdAdv messageId) producerDLQ.newMessage(Schema.AUTO_PRODUCE_BYTES(message.getReaderSchema().get())) .value(message.getData()) .properties(getPropertiesMap(message, originMessageIdStr, originTopicNameStr)); - copyMessageKeyIfNeeded(message, typedMessageBuilderNew); + copyMessageKeysIfNeeded(message, typedMessageBuilderNew); typedMessageBuilderNew.sendAsync() .thenAccept(messageIdInDLQ -> { possibleSendToDeadLetterTopicMessages.remove(messageId); From e3b9cdcd984d668b5cfb29c7b586ef0eae3d7a7b Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 16 Aug 2024 07:48:18 +0300 Subject: [PATCH 673/699] [feat] Add scripts for updating BK RocksDB ini files (#23178) (cherry picked from commit 3ada56635a6bf54eccdbaa572b6a023baa6f9bfa) (cherry picked from commit ca4512ce1cd0d37ca78c0c614f905c03b25d7a3d) --- docker/pulsar/scripts/update-ini-from-env.py | 70 +++++++++++++++ .../scripts/update-rocksdb-conf-from-env.py | 86 +++++++++++++++++++ 2 files changed, 156 insertions(+) create mode 100755 docker/pulsar/scripts/update-ini-from-env.py create mode 100755 docker/pulsar/scripts/update-rocksdb-conf-from-env.py diff --git a/docker/pulsar/scripts/update-ini-from-env.py b/docker/pulsar/scripts/update-ini-from-env.py new file mode 100755 index 0000000000000..6b0d7a795c3f8 --- /dev/null +++ b/docker/pulsar/scripts/update-ini-from-env.py @@ -0,0 +1,70 @@ +#!/usr/bin/env python3 +# +# 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. +# + +import os +import sys +import configparser +import re + +def get_first_word(section_name): + # Split the section name by any non-word character and return the first word + return re.split(r'\W+', section_name)[0] + +def update_ini_file(ini_file_path, env_prefix): + # Read the existing INI file + config = configparser.ConfigParser() + config.read(ini_file_path) + + # Flag to track if any updates were made + updated = False + + # Iterate over environment variables + for key, value in os.environ.items(): + if env_prefix and not key.startswith(env_prefix): + continue + + stripped_key = key[len(env_prefix):] if env_prefix else key + + # Iterate through sections + for section in config.sections(): + first_word = get_first_word(section) + prefix = first_word + '_' + if stripped_key.startswith(prefix): + config.set(section, stripped_key[len(prefix):], value) + updated = True + break + elif config.has_option(section, stripped_key): + config.set(section, stripped_key, value) + updated = True + break + + # Write the updated INI file only if there were updates + if updated: + with open(ini_file_path, 'w') as configfile: + config.write(configfile) + +if __name__ == "__main__": + if len(sys.argv) != 3: + print("Usage: python3 update-ini-from-env.py ") + sys.exit(1) + + ini_file_path = sys.argv[1] + env_prefix = sys.argv[2] + update_ini_file(ini_file_path, env_prefix) \ No newline at end of file diff --git a/docker/pulsar/scripts/update-rocksdb-conf-from-env.py b/docker/pulsar/scripts/update-rocksdb-conf-from-env.py new file mode 100755 index 0000000000000..2e55b455de3b7 --- /dev/null +++ b/docker/pulsar/scripts/update-rocksdb-conf-from-env.py @@ -0,0 +1,86 @@ +#!/usr/bin/env python3 +# +# 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. +# + +# allows tuning of RocksDB configuration via environment variables which were effective +# before Pulsar 2.11 / BookKeeper 4.15 / https://github.com/apache/bookkeeper/pull/3056 +# the script should be applied to the `conf/entry_location_rocksdb.conf` file + +import os +import sys +import configparser + +# Constants for section keys +DB_OPTIONS = "DBOptions" +CF_OPTIONS = "CFOptions \"default\"" +TABLE_OPTIONS = "TableOptions/BlockBasedTable \"default\"" + +def update_ini_file(ini_file_path): + config = configparser.ConfigParser() + config.read(ini_file_path) + updated = False + + # Mapping of environment variables to INI sections and keys + env_to_ini_mapping = { + "dbStorage_rocksDB_logPath": (DB_OPTIONS, "log_path"), + "dbStorage_rocksDB_logLevel": (DB_OPTIONS, "info_log_level"), + "dbStorage_rocksDB_lz4CompressionEnabled": (CF_OPTIONS, "compression"), + "dbStorage_rocksDB_writeBufferSizeMB": (CF_OPTIONS, "write_buffer_size"), + "dbStorage_rocksDB_sstSizeInMB": (CF_OPTIONS, "target_file_size_base"), + "dbStorage_rocksDB_blockSize": (TABLE_OPTIONS, "block_size"), + "dbStorage_rocksDB_bloomFilterBitsPerKey": (TABLE_OPTIONS, "filter_policy"), + "dbStorage_rocksDB_blockCacheSize": (TABLE_OPTIONS, "block_cache"), + "dbStorage_rocksDB_numLevels": (CF_OPTIONS, "num_levels"), + "dbStorage_rocksDB_numFilesInLevel0": (CF_OPTIONS, "level0_file_num_compaction_trigger"), + "dbStorage_rocksDB_maxSizeInLevel1MB": (CF_OPTIONS, "max_bytes_for_level_base"), + "dbStorage_rocksDB_format_version": (TABLE_OPTIONS, "format_version") + } + + # Type conversion functions + def mb_to_bytes(mb): + return str(int(mb) * 1024 * 1024) + + def str_to_bool(value): + return True if value.lower() in ["true", "1", "yes"] else False + + # Iterate over environment variables + for key, value in os.environ.items(): + if key.startswith("PULSAR_PREFIX_"): + key = key[len("PULSAR_PREFIX_"):] + + if key in env_to_ini_mapping: + section, option = env_to_ini_mapping[key] + if key in ["dbStorage_rocksDB_writeBufferSizeMB", "dbStorage_rocksDB_sstSizeInMB", "dbStorage_rocksDB_maxSizeInLevel1MB"]: + value = mb_to_bytes(value) + elif key == "dbStorage_rocksDB_lz4CompressionEnabled": + value = "kLZ4Compression" if str_to_bool(value) else "kNoCompression" + elif key == "dbStorage_rocksDB_bloomFilterBitsPerKey": + value = "rocksdb.BloomFilter:{}:false".format(value) + if config.get(section, option, fallback=None) != value: + config.set(section, option, value) + updated = True + + # Write the updated INI file only if there were updates + if updated: + with open(ini_file_path, 'w') as configfile: + config.write(configfile) + +if __name__ == "__main__": + ini_file_path = sys.argv[1] if len(sys.argv) > 1 else "conf/entry_location_rocksdb.conf" + update_ini_file(ini_file_path) \ No newline at end of file From b463ba406293c553426918c8c7b8ed66acc40539 Mon Sep 17 00:00:00 2001 From: Lari Hotari Date: Fri, 16 Aug 2024 10:40:28 +0300 Subject: [PATCH 674/699] [fix][test] Fix flaky SubscriptionSeekTest.testSeekIsByReceive (#23170) (cherry picked from commit a1f3322ed358ab6841f0d3e43f2afcc54788b887) (cherry picked from commit 6434d57dc5afdc967aafcea129018ca4a3ab3254) --- .../broker/service/SubscriptionSeekTest.java | 37 +++++++++++++++---- 1 file changed, 30 insertions(+), 7 deletions(-) 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 12d34dc5d240c..52d631a86d11f 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 @@ -85,9 +85,11 @@ protected void cleanup() throws Exception { public void testSeek() throws Exception { final String topicName = "persistent://prop/use/ns-abc/testSeek"; + @Cleanup Producer producer = pulsarClient.newProducer().topic(topicName).create(); // Disable pre-fetch in consumer to track the messages received + @Cleanup org.apache.pulsar.client.api.Consumer consumer = pulsarClient.newConsumer().topic(topicName) .subscriptionName("my-subscription").receiverQueueSize(0).subscribe(); @@ -137,11 +139,13 @@ public void testSeek() throws Exception { @Test public void testSeekIsByReceive() throws PulsarClientException { - final String topicName = "persistent://prop/use/ns-abc/testSeek"; + final String topicName = "persistent://prop/use/ns-abc/testSeekIsByReceive"; + @Cleanup Producer producer = pulsarClient.newProducer().topic(topicName).create(); String subscriptionName = "my-subscription"; + @Cleanup org.apache.pulsar.client.api.Consumer consumer = pulsarClient.newConsumer().topic(topicName) .subscriptionName(subscriptionName) .subscribe(); @@ -163,6 +167,7 @@ public void testSeekForBatch() throws Exception { final String topicName = "persistent://prop/use/ns-abcd/testSeekForBatch"; String subscriptionName = "my-subscription-batch"; + @Cleanup Producer producer = pulsarClient.newProducer(Schema.STRING) .enableBatching(true) .batchingMaxMessages(3) @@ -189,6 +194,7 @@ public void testSeekForBatch() throws Exception { producer.close(); + @Cleanup org.apache.pulsar.client.api.Consumer consumer = pulsarClient.newConsumer(Schema.STRING) .topic(topicName) .subscriptionName(subscriptionName) @@ -219,6 +225,7 @@ public void testSeekForBatchMessageAndSpecifiedBatchIndex() throws Exception { final String topicName = "persistent://prop/use/ns-abcd/testSeekForBatchMessageAndSpecifiedBatchIndex"; String subscriptionName = "my-subscription-batch"; + @Cleanup Producer producer = pulsarClient.newProducer(Schema.STRING) .enableBatching(true) .batchingMaxMessages(3) @@ -263,6 +270,7 @@ public void testSeekForBatchMessageAndSpecifiedBatchIndex() throws Exception { .serviceUrl(lookupUrl.toString()) .build(); + @Cleanup org.apache.pulsar.client.api.Consumer consumer = newPulsarClient.newConsumer(Schema.STRING) .topic(topicName) .subscriptionName(subscriptionName) @@ -299,6 +307,7 @@ public void testSeekForBatchByAdmin() throws PulsarClientException, ExecutionExc final String topicName = "persistent://prop/use/ns-abcd/testSeekForBatchByAdmin-" + UUID.randomUUID().toString(); String subscriptionName = "my-subscription-batch"; + @Cleanup Producer producer = pulsarClient.newProducer(Schema.STRING) .enableBatching(true) .batchingMaxMessages(3) @@ -324,7 +333,7 @@ public void testSeekForBatchByAdmin() throws PulsarClientException, ExecutionExc producer.close(); - + @Cleanup org.apache.pulsar.client.api.Consumer consumer = pulsarClient.newConsumer(Schema.STRING) .topic(topicName) .subscriptionName(subscriptionName) @@ -380,6 +389,7 @@ public void testConcurrentResetCursor() throws Exception { final String topicName = "persistent://prop/use/ns-abc/testConcurrentReset_" + System.currentTimeMillis(); final String subscriptionName = "test-sub-name"; + @Cleanup Producer producer = pulsarClient.newProducer().topic(topicName).create(); admin.topics().createSubscription(topicName, subscriptionName, MessageId.earliest); @@ -429,6 +439,7 @@ public void testSeekOnPartitionedTopic() throws Exception { final String topicName = "persistent://prop/use/ns-abc/testSeekPartitions"; admin.topics().createPartitionedTopic(topicName, 2); + @Cleanup org.apache.pulsar.client.api.Consumer consumer = pulsarClient.newConsumer().topic(topicName) .subscriptionName("my-subscription").subscribe(); @@ -446,9 +457,11 @@ public void testSeekTime() throws Exception { long resetTimeInMillis = TimeUnit.SECONDS .toMillis(RelativeTimeUtil.parseRelativeTimeInSeconds(resetTimeStr)); + @Cleanup Producer producer = pulsarClient.newProducer().topic(topicName).create(); // Disable pre-fetch in consumer to track the messages received + @Cleanup org.apache.pulsar.client.api.Consumer consumer = pulsarClient.newConsumer().topic(topicName) .subscriptionName("my-subscription").receiverQueueSize(0).subscribe(); @@ -482,6 +495,7 @@ public void testSeekTimeByFunction() throws Exception { int msgNum = 20; admin.topics().createPartitionedTopic(topicName, partitionNum); creatProducerAndSendMsg(topicName, msgNum); + @Cleanup org.apache.pulsar.client.api.Consumer consumer = pulsarClient .newConsumer(Schema.STRING).startMessageIdInclusive() .topic(topicName).subscriptionName("my-sub").subscribe(); @@ -529,6 +543,7 @@ public void testSeekTimeOnPartitionedTopic() throws Exception { long resetTimeInMillis = TimeUnit.SECONDS .toMillis(RelativeTimeUtil.parseRelativeTimeInSeconds(resetTimeStr)); admin.topics().createPartitionedTopic(topicName, partitions); + @Cleanup Producer producer = pulsarClient.newProducer().topic(topicName).create(); // Disable pre-fetch in consumer to track the messages received org.apache.pulsar.client.api.Consumer consumer = pulsarClient.newConsumer().topic(topicName) @@ -582,12 +597,14 @@ public void testSeekTimeOnPartitionedTopic() throws Exception { public void testShouldCloseAllConsumersForMultipleConsumerDispatcherWhenSeek() throws Exception { final String topicName = "persistent://prop/use/ns-abc/testShouldCloseAllConsumersForMultipleConsumerDispatcherWhenSeek"; // Disable pre-fetch in consumer to track the messages received + @Cleanup org.apache.pulsar.client.api.Consumer consumer1 = pulsarClient.newConsumer() .topic(topicName) .subscriptionType(SubscriptionType.Shared) .subscriptionName("my-subscription") .subscribe(); + @Cleanup org.apache.pulsar.client.api.Consumer consumer2 = pulsarClient.newConsumer() .topic(topicName) .subscriptionType(SubscriptionType.Shared) @@ -614,20 +631,20 @@ public void testShouldCloseAllConsumersForMultipleConsumerDispatcherWhenSeek() t for (Consumer consumer : consumers) { assertFalse(connectedSinceSet.contains(consumer.getStats().getConnectedSince())); } - consumer1.close(); - consumer2.close(); } @Test public void testOnlyCloseActiveConsumerForSingleActiveConsumerDispatcherWhenSeek() throws Exception { final String topicName = "persistent://prop/use/ns-abc/testOnlyCloseActiveConsumerForSingleActiveConsumerDispatcherWhenSeek"; // Disable pre-fetch in consumer to track the messages received + @Cleanup org.apache.pulsar.client.api.Consumer consumer1 = pulsarClient.newConsumer() .topic(topicName) .subscriptionType(SubscriptionType.Failover) .subscriptionName("my-subscription") .subscribe(); + @Cleanup org.apache.pulsar.client.api.Consumer consumer2 = pulsarClient.newConsumer() .topic(topicName) .subscriptionType(SubscriptionType.Failover) @@ -667,11 +684,13 @@ public void testSeekByFunction() throws Exception { int msgNum = 160; admin.topics().createPartitionedTopic(topicName, partitionNum); creatProducerAndSendMsg(topicName, msgNum); + @Cleanup org.apache.pulsar.client.api.Consumer consumer = pulsarClient .newConsumer(Schema.STRING).startMessageIdInclusive() .topic(topicName).subscriptionName("my-sub").subscribe(); TopicName partitionedTopic = TopicName.get(topicName); + @Cleanup Reader reader = pulsarClient.newReader(Schema.STRING) .startMessageId(MessageId.earliest) .topic(partitionedTopic.getPartition(0).toString()).create(); @@ -720,12 +739,11 @@ public void testSeekByFunction() throws Exception { for (MessageId messageId : msgNotIn) { assertFalse(received.contains(messageId)); } - reader.close(); - consumer.close(); } private List creatProducerAndSendMsg(String topic, int msgNum) throws Exception { List messageIds = new ArrayList<>(); + @Cleanup Producer producer = pulsarClient .newProducer(Schema.STRING) .enableBatching(false) @@ -734,7 +752,6 @@ private List creatProducerAndSendMsg(String topic, int msgNum) throws for (int i = 0; i < msgNum; i++) { messageIds.add(producer.send("msg" + i)); } - producer.close(); return messageIds; } @@ -755,6 +772,7 @@ public void testSeekByFunctionAndMultiTopic() throws Exception { MessageId msgIdInTopic2Partition0 = admin.topics().getLastMessageId(topic2.getPartition(0).toString()); MessageId msgIdInTopic2Partition2 = admin.topics().getLastMessageId(topic2.getPartition(2).toString()); + @Cleanup org.apache.pulsar.client.api.Consumer consumer = pulsarClient .newConsumer(Schema.STRING).startMessageIdInclusive() .topics(Arrays.asList(topicName, topicName2)).subscriptionName("my-sub").subscribe(); @@ -795,6 +813,7 @@ public void testSeekWillNotEncounteredFencedError() throws Exception { // Create a pulsar client with a subscription fenced counter. ClientBuilderImpl clientBuilder = (ClientBuilderImpl) PulsarClient.builder().serviceUrl(lookupUrl.toString()); AtomicInteger receivedFencedErrorCounter = new AtomicInteger(); + @Cleanup PulsarClient client = InjectedClientCnxClientBuilder.create(clientBuilder, (conf, eventLoopGroup) -> new ClientCnx(conf, eventLoopGroup) { protected void handleError(CommandError error) { @@ -806,10 +825,13 @@ protected void handleError(CommandError error) { }); // publish some messages. + @Cleanup org.apache.pulsar.client.api.Consumer consumer = client.newConsumer(Schema.STRING) .topic(topicName) .subscriptionName("s1") .subscribe(); + + @Cleanup Producer producer = client.newProducer(Schema.STRING) .topic(topicName).create(); MessageIdImpl msgId1 = (MessageIdImpl) producer.send("0"); @@ -849,6 +871,7 @@ protected void handleError(CommandError error) { public void testExceptionBySeekFunction() throws Exception { final String topicName = "persistent://prop/use/ns-abc/test" + UUID.randomUUID(); creatProducerAndSendMsg(topicName,10); + @Cleanup org.apache.pulsar.client.api.Consumer consumer = pulsarClient .newConsumer() .topic(topicName).subscriptionName("my-sub").subscribe(); From 69064c3bf092af38ab78e129e0195123dba0e06a Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Fri, 16 Aug 2024 23:30:34 +0800 Subject: [PATCH 675/699] [improve][broker] Support customized shadow managed ledger implementation (#23179) (cherry picked from commit 7f04364f13330b56cabeab48b9b5055a70a88119) (cherry picked from commit 4196195dc6ffd580b0c38ec9db5abc63df742392) --- .../pulsar/broker/service/persistent/PersistentTopic.java | 3 +-- .../pulsar/broker/namespace/NamespaceServiceTest.java | 2 ++ .../pulsar/broker/service/MessageCumulativeAckTest.java | 5 ++++- .../service/PersistentDispatcherFailoverConsumerTest.java | 1 + .../apache/pulsar/broker/service/PersistentTopicTest.java | 6 ++++++ .../org/apache/pulsar/broker/service/ServerCnxTest.java | 1 + .../systopic/NamespaceEventsSystemTopicServiceTest.java | 7 ++++++- .../apache/pulsar/broker/transaction/TransactionTest.java | 2 ++ .../mledger/offload/jcloud/impl/MockManagedLedger.java | 2 +- 9 files changed, 24 insertions(+), 5 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 033f8e1350427..dd31031899db4 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 @@ -81,7 +81,6 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; -import org.apache.bookkeeper.mledger.impl.ShadowManagedLedgerImpl; import org.apache.bookkeeper.mledger.util.ManagedLedgerImplUtils; import org.apache.bookkeeper.net.BookieId; import org.apache.commons.collections4.CollectionUtils; @@ -391,7 +390,7 @@ public PersistentTopic(String topic, ManagedLedger ledger, BrokerService brokerS this.transactionBuffer = new TransactionBufferDisable(this); } transactionBuffer.syncMaxReadPositionForNormalPublish((PositionImpl) ledger.getLastConfirmedEntry(), true); - if (ledger instanceof ShadowManagedLedgerImpl) { + if (ledger.getConfig().getShadowSource() != null) { shadowSourceTopic = TopicName.get(ledger.getConfig().getShadowSource()); } else { shadowSourceTopic = null; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java index e975fe3cfa926..2ed5ebaf4e2b3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/namespace/NamespaceServiceTest.java @@ -53,6 +53,7 @@ import java.util.concurrent.atomic.AtomicReference; import lombok.Cleanup; import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.commons.lang3.tuple.Pair; @@ -197,6 +198,7 @@ public void testSplitMapWithRefreshedStatMap() throws Exception { ManagedLedger ledger = mock(ManagedLedger.class); when(ledger.getCursors()).thenReturn(new ArrayList<>()); + when(ledger.getConfig()).thenReturn(new ManagedLedgerConfig()); doReturn(CompletableFuture.completedFuture(null)).when(MockOwnershipCache).disableOwnership(any(NamespaceBundle.class)); Field ownership = NamespaceService.class.getDeclaredField("ownershipCache"); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessageCumulativeAckTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessageCumulativeAckTest.java index f3fe26af4b968..cc4fe22962484 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessageCumulativeAckTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/MessageCumulativeAckTest.java @@ -37,6 +37,7 @@ import io.netty.channel.ChannelHandlerContext; import java.net.InetSocketAddress; import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.pulsar.broker.testcontext.PulsarTestContext; import org.apache.pulsar.broker.service.persistent.PersistentSubscription; @@ -74,7 +75,9 @@ public void setup() throws Exception { .when(serverCnx).getCommandSender(); String topicName = TopicName.get("MessageCumulativeAckTest").toString(); - PersistentTopic persistentTopic = new PersistentTopic(topicName, mock(ManagedLedger.class), pulsarTestContext.getBrokerService()); + var mockManagedLedger = mock(ManagedLedger.class); + when(mockManagedLedger.getConfig()).thenReturn(new ManagedLedgerConfig()); + var persistentTopic = new PersistentTopic(topicName, mockManagedLedger, pulsarTestContext.getBrokerService()); sub = spy(new PersistentSubscription(persistentTopic, "sub-1", mock(ManagedCursorImpl.class), false)); doNothing().when(sub).acknowledgeMessage(any(), any(), any()); 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 631b702dfbd08..2718323f1bba3 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 @@ -180,6 +180,7 @@ void setupMLAsyncCallbackMocks() { cursorMock = mock(ManagedCursorImpl.class); doReturn(new ArrayList<>()).when(ledgerMock).getCursors(); + doReturn(new ManagedLedgerConfig()).when(ledgerMock).getConfig(); doReturn("mockCursor").when(cursorMock).getName(); // call openLedgerComplete with ledgerMock on ML factory asyncOpen 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 123a7e5372dc3..ea79694fd7ffd 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 @@ -230,6 +230,7 @@ public void teardown() throws Exception { @Test public void testCreateTopic() { final ManagedLedger ledgerMock = mock(ManagedLedger.class); + doReturn(new ManagedLedgerConfig()).when(ledgerMock).getConfig(); doReturn(new ArrayList<>()).when(ledgerMock).getCursors(); final String topicName = "persistent://prop/use/ns-abc/topic1"; @@ -357,6 +358,7 @@ public void testPublishMessageMLFailure() throws Exception { final String successTopicName = "persistent://prop/use/ns-abc/successTopic"; final ManagedLedger ledgerMock = mock(ManagedLedger.class); + doReturn(new ManagedLedgerConfig()).when(ledgerMock).getConfig(); doReturn(new ArrayList<>()).when(ledgerMock).getCursors(); PersistentTopic topic = new PersistentTopic(successTopicName, ledgerMock, brokerService); @@ -1365,6 +1367,7 @@ void setupMLAsyncCallbackMocks() { final CompletableFuture closeFuture = new CompletableFuture<>(); doReturn(new ArrayList<>()).when(ledgerMock).getCursors(); + doReturn(new ManagedLedgerConfig()).when(ledgerMock).getConfig(); doReturn("mockCursor").when(cursorMock).getName(); doReturn(true).when(cursorMock).isDurable(); // doNothing().when(cursorMock).asyncClose(new CloseCallback() { @@ -1662,6 +1665,7 @@ public void testAtomicReplicationRemoval() throws Exception { String remoteCluster = "remote"; final ManagedLedger ledgerMock = mock(ManagedLedger.class); doNothing().when(ledgerMock).asyncDeleteCursor(any(), any(), any()); + doReturn(new ManagedLedgerConfig()).when(ledgerMock).getConfig(); doReturn(new ArrayList<>()).when(ledgerMock).getCursors(); PersistentTopic topic = new PersistentTopic(globalTopicName, ledgerMock, brokerService); @@ -1721,6 +1725,7 @@ public void testClosingReplicationProducerTwice() throws Exception { final ManagedLedger ledgerMock = mock(ManagedLedger.class); doNothing().when(ledgerMock).asyncDeleteCursor(any(), any(), any()); doReturn(new ArrayList<>()).when(ledgerMock).getCursors(); + doReturn(new ManagedLedgerConfig()).when(ledgerMock).getConfig(); PersistentTopic topic = new PersistentTopic(globalTopicName, ledgerMock, brokerService); @@ -2111,6 +2116,7 @@ public void testTopicCloseFencingTimeout() throws Exception { @Test public void testGetDurableSubscription() throws Exception { ManagedLedger mockLedger = mock(ManagedLedger.class); + doReturn(new ManagedLedgerConfig()).when(mockLedger).getConfig(); ManagedCursor mockCursor = mock(ManagedCursorImpl.class); Position mockPosition = mock(Position.class); doReturn("test").when(mockCursor).getName(); 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 3971cc5eda53c..776a6787faccf 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 @@ -2920,6 +2920,7 @@ private void setupMLAsyncCallbackMocks() { ledgerMock = mock(ManagedLedger.class); cursorMock = mock(ManagedCursor.class); doReturn(new ArrayList<>()).when(ledgerMock).getCursors(); + doReturn(new ManagedLedgerConfig()).when(ledgerMock).getConfig(); // call openLedgerComplete with ledgerMock on ML factory asyncOpen doAnswer((Answer) invocationOnMock -> { 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 44a4de5e8a923..e66140efb32bb 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 @@ -20,10 +20,13 @@ import static org.apache.pulsar.broker.service.SystemTopicBasedTopicPoliciesService.getEventKey; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + import com.google.common.collect.Sets; import java.util.HashSet; import lombok.Cleanup; import org.apache.bookkeeper.mledger.ManagedLedger; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.persistent.PersistentTopic; @@ -99,7 +102,9 @@ public void testSystemTopicSchemaCompatibility() throws Exception { TopicPoliciesSystemTopicClient systemTopicClientForNamespace1 = systemTopicFactory .createTopicPoliciesSystemTopicClient(NamespaceName.get(NAMESPACE1)); String topicName = systemTopicClientForNamespace1.getTopicName().toString(); - SystemTopic topic = new SystemTopic(topicName, mock(ManagedLedger.class), pulsar.getBrokerService()); + final var mockManagedLedger = mock(ManagedLedger.class); + when(mockManagedLedger.getConfig()).thenReturn(new ManagedLedgerConfig()); + SystemTopic topic = new SystemTopic(topicName, mockManagedLedger, pulsar.getBrokerService()); Assert.assertEquals(SchemaCompatibilityStrategy.ALWAYS_COMPATIBLE, topic.getSchemaCompatibilityStrategy()); } 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 80063f207002f..5850a100f0158 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 @@ -69,6 +69,7 @@ import org.apache.bookkeeper.common.util.Bytes; import org.apache.bookkeeper.mledger.AsyncCallbacks; import org.apache.bookkeeper.mledger.ManagedCursor; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.bookkeeper.mledger.ManagedLedgerException; import org.apache.bookkeeper.mledger.ManagedLedgerFactory; import org.apache.bookkeeper.mledger.impl.ManagedCursorContainer; @@ -1587,6 +1588,7 @@ public void testTBRecoverChangeStateError() throws InterruptedException, Timeout // Mock managedLedger. ManagedLedgerImpl managedLedger = mock(ManagedLedgerImpl.class); ManagedCursorContainer managedCursors = new ManagedCursorContainer(); + when(managedLedger.getConfig()).thenReturn(new ManagedLedgerConfig()); when(managedLedger.getCursors()).thenReturn(managedCursors); PositionImpl position = PositionImpl.EARLIEST; when(managedLedger.getLastConfirmedEntry()).thenReturn(position); diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/MockManagedLedger.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/MockManagedLedger.java index 66ace69d7cda2..8f52d20c5ee83 100644 --- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/MockManagedLedger.java +++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/MockManagedLedger.java @@ -274,7 +274,7 @@ public boolean isTerminated() { @Override public ManagedLedgerConfig getConfig() { - return null; + return new ManagedLedgerConfig(); } @Override From 6c6f6bf24f78762db2972122b15be54b452aa13f Mon Sep 17 00:00:00 2001 From: Kai Wang Date: Sat, 17 Aug 2024 00:17:55 +0800 Subject: [PATCH 676/699] [improve][broker] Should notify bundle ownership listener onLoad event when ServiceUnitState start (ExtensibleLoadManagerImpl only) (#23152) (cherry picked from commit 3053b647e0ca646b2df9f03815947104cd2e705f) (cherry picked from commit 9a090f7f3d3a1bfb80d3d93dc325aed7315b8e46) --- .../channel/ServiceUnitStateChannelImpl.java | 16 +++++- .../ExtensibleLoadManagerImplTest.java | 50 +++++++++++++++++++ 2 files changed, 64 insertions(+), 2 deletions(-) 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 e7804806d9c03..7e864ddac2ce5 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 @@ -332,7 +332,8 @@ public synchronized void start() throws PulsarServerException { "topicCompactionStrategyClassName", ServiceUnitStateCompactionStrategy.class.getName())) .create(); - tableview.listen((key, value) -> handle(key, value)); + tableview.listen(this::handleEvent); + tableview.forEach(this::handleExisting); var strategy = (ServiceUnitStateCompactionStrategy) TopicCompactionStrategy.getInstance(TABLE_VIEW_TAG); if (strategy == null) { String err = TABLE_VIEW_TAG + "tag TopicCompactionStrategy is null."; @@ -663,7 +664,7 @@ public CompletableFuture publishSplitEventAsync(Split split) { }).thenApply(__ -> null); } - private void handle(String serviceUnit, ServiceUnitStateData data) { + private void handleEvent(String serviceUnit, ServiceUnitStateData data) { long totalHandledRequests = getHandlerTotalCounter(data).incrementAndGet(); if (debug()) { log.info("{} received a handle request for serviceUnit:{}, data:{}. totalHandledRequests:{}", @@ -689,6 +690,17 @@ private void handle(String serviceUnit, ServiceUnitStateData data) { } } + private void handleExisting(String serviceUnit, ServiceUnitStateData data) { + if (debug()) { + log.info("Loaded the service unit state data. serviceUnit: {}, data: {}", serviceUnit, data); + } + ServiceUnitState state = state(data); + if (state.equals(Owned) && isTargetBroker(data.dstBroker())) { + pulsar.getNamespaceService() + .onNamespaceBundleOwned(LoadManagerShared.getNamespaceBundle(pulsar, serviceUnit)); + } + } + private static boolean isTransferCommand(ServiceUnitStateData data) { if (data == null) { return false; 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 b941293c35f04..f23246317071b 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 @@ -392,6 +392,56 @@ public boolean test(NamespaceBundle namespaceBundle) { } } + @Test(timeOut = 30 * 1000) + public void testNamespaceOwnershipListener() throws Exception { + Pair topicAndBundle = + getBundleIsNotOwnByChangeEventTopic("test-namespace-ownership-listener"); + TopicName topicName = topicAndBundle.getLeft(); + NamespaceBundle bundle = topicAndBundle.getRight(); + + String broker = admin.lookups().lookupTopic(topicName.toString()); + log.info("Assign the bundle {} to {}", bundle, broker); + + checkOwnershipState(broker, bundle); + + AtomicInteger onloadCount = new AtomicInteger(0); + AtomicInteger unloadCount = new AtomicInteger(0); + + NamespaceBundleOwnershipListener listener = new NamespaceBundleOwnershipListener() { + @Override + public void onLoad(NamespaceBundle bundle) { + onloadCount.incrementAndGet(); + } + + @Override + public void unLoad(NamespaceBundle bundle) { + unloadCount.incrementAndGet(); + } + + @Override + public boolean test(NamespaceBundle namespaceBundle) { + return namespaceBundle.equals(bundle); + } + }; + pulsar1.getNamespaceService().addNamespaceBundleOwnershipListener(listener); + pulsar2.getNamespaceService().addNamespaceBundleOwnershipListener(listener); + + // There are a service unit state channel already started, when add listener, it will trigger the onload event. + Awaitility.await().untilAsserted(() -> { + assertEquals(onloadCount.get(), 1); + assertEquals(unloadCount.get(), 0); + }); + + ServiceUnitStateChannelImpl channel = new ServiceUnitStateChannelImpl(pulsar1); + channel.start(); + Awaitility.await().untilAsserted(() -> { + assertEquals(onloadCount.get(), 2); + assertEquals(unloadCount.get(), 0); + }); + + channel.close(); + } + private void checkOwnershipState(String broker, NamespaceBundle bundle) throws ExecutionException, InterruptedException { var targetLoadManager = secondaryLoadManager; From ae7a71ccab7c114f44cd61c02c0c87e38bc9f3c0 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Sat, 17 Aug 2024 17:55:37 +0800 Subject: [PATCH 677/699] [improve] [broker] Optimize performance for checking max topics when the topic is a system topic (#23185) (cherry picked from commit 73433cd06e65ce5e194372a657c5a414e820138b) (cherry picked from commit 09bc87706e5e4f60564b19734954de5cd8055a6c) --- .../java/org/apache/pulsar/broker/service/BrokerService.java | 3 +++ 1 file changed, 3 insertions(+) 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 cbe16e1776212..e7eaef543a3b1 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 @@ -3713,6 +3713,9 @@ public CompletableFuture deleteSchema(TopicName topicName) { } private CompletableFuture checkMaxTopicsPerNamespace(TopicName topicName, int numPartitions) { + if (isSystemTopic(topicName)) { + return CompletableFuture.completedFuture(null); + } return pulsar.getPulsarResources().getNamespaceResources() .getPoliciesAsync(topicName.getNamespaceObject()) .thenCompose(optPolicies -> { From 8d7f75066c5f7f9c83803ff5a731096038fdf81e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=81=93=E5=90=9B?= Date: Sat, 17 Aug 2024 19:50:29 +0800 Subject: [PATCH 678/699] [fix][broker] Skip reading entries from closed cursor. (#22751) (cherry picked from commit aa8226f45e3b28a14377f9f949d5a34f61b27e9a) (cherry picked from commit 93e09ae989aad112d4e6424586d08aa5a2e90ce4) --- ...PersistentDispatcherMultipleConsumers.java | 26 +++- ...sistentDispatcherSingleActiveConsumer.java | 23 +++- ...istentDispatcherMultipleConsumersTest.java | 71 ++++++++++ ...entDispatcherSingleActiveConsumerTest.java | 127 ++++++++++++++++++ 4 files changed, 242 insertions(+), 5 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumerTest.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 a04b1fc3ad12b..7d0cebc0870f1 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 @@ -19,6 +19,7 @@ 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; @@ -286,6 +287,12 @@ public void readMoreEntriesAsync() { } 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 @@ -853,7 +860,14 @@ public synchronized void readEntriesFailed(ManagedLedgerException exception, Obj ReadType readType = (ReadType) ctx; long waitTimeMillis = readFailureBackoff.next(); - if (exception instanceof NoMoreEntriesToReadException) { + // 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 @@ -892,7 +906,14 @@ public synchronized void readEntriesFailed(ManagedLedgerException exception, Obj } 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 (PersistentDispatcherMultipleConsumers.this) { // If it's a replay read we need to retry even if there's already @@ -902,11 +923,10 @@ public synchronized void readEntriesFailed(ManagedLedgerException exception, Obj log.info("[{}] Retrying read operation", name); readMoreEntries(); } else { - log.info("[{}] Skipping read retry: havePendingRead {}", name, havePendingRead, exception); + log.info("[{}] Skipping read retry: havePendingRead {}", name, havePendingRead, e); } } }, waitTimeMillis, TimeUnit.MILLISECONDS); - } private boolean needTrimAckedMessages() { diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java index c8572a87b7d6a..77d580b0389ca 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java @@ -20,6 +20,7 @@ import static org.apache.pulsar.broker.service.persistent.PersistentTopic.MESSAGE_RATE_BACKOFF_MS; import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; +import com.google.common.annotations.VisibleForTesting; import io.netty.util.Recycler; import java.util.Iterator; import java.util.List; @@ -318,6 +319,12 @@ public void redeliverUnacknowledgedMessages(Consumer consumer, List { // Jump again into dispatcher dedicated thread @@ -546,8 +566,7 @@ private synchronized void internalReadEntriesFailed(ManagedLedgerException excep } } }); - }, waitTimeMillis, TimeUnit.MILLISECONDS); - + }, delay, TimeUnit.MILLISECONDS); } @Override 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 f24c5c5933e5b..a03ed92b81590 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 @@ -20,15 +20,24 @@ 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; @@ -98,4 +107,66 @@ public void testTopicDeleteIfConsumerSetMismatchConsumerList2() throws Exception 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. + PersistentDispatcherMultipleConsumers dispatcher = + Mockito.spy(new PersistentDispatcherMultipleConsumers(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 -> { + PersistentDispatcherMultipleConsumers 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/PersistentDispatcherSingleActiveConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumerTest.java new file mode 100644 index 0000000000000..a4c9e26ffb853 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumerTest.java @@ -0,0 +1,127 @@ +/* + * 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.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.Consumer; +import org.apache.pulsar.broker.service.Subscription; +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.common.api.proto.CommandSubscribe; +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 PersistentDispatcherSingleActiveConsumerTest 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 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. + PersistentDispatcherSingleActiveConsumer dispatcher = + Mockito.spy(new PersistentDispatcherSingleActiveConsumer(cursor, CommandSubscribe.SubType.Exclusive,0, topic, sub)); + + // Mock a consumer + Consumer consumer = Mockito.mock(Consumer.class); + consumer.getAvailablePermits(); + Mockito.doReturn(10).when(consumer).getAvailablePermits(); + Mockito.doReturn(10).when(consumer).getAvgMessagesPerEntry(); + Mockito.doReturn("test").when(consumer).consumerName(); + Mockito.doReturn(true).when(consumer).isWritable(); + Mockito.doReturn(false).when(consumer).readCompacted(); + + // Make the consumer as the active consumer. + Mockito.doReturn(consumer).when(dispatcher).getActiveConsumer(); + + // 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.eq(consumer), 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 -> { + PersistentDispatcherSingleActiveConsumer 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(consumer); + + // 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); + } +} From 7047c2cdce2d37dc0cbcfce1875e273f1ba0e36a Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Sun, 18 Aug 2024 19:02:23 +0800 Subject: [PATCH 679/699] [fix] [broker] Topic can never be loaded up due to broker maintains a failed topic creation future (#23184) (cherry picked from commit 9edaa8569deff9c0cbb41b261fee472603f3df4d) (cherry picked from commit 34c6fa0528665eeba407839343e00294de0f031f) --- .../pulsar/broker/service/BrokerService.java | 15 ++- .../client/api/OrphanPersistentTopicTest.java | 95 +++++++++++++++++++ 2 files changed, 106 insertions(+), 4 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 e7eaef543a3b1..a0fe59370521b 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 @@ -1683,6 +1683,7 @@ private void checkOwnershipAndCreatePersistentTopic(final String topic, boolean topicFuture.completeExceptionally(new ServiceUnitNotReadyException(msg)); } }).exceptionally(ex -> { + pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); topicFuture.completeExceptionally(ex); return null; }); @@ -1782,10 +1783,16 @@ public void openLedgerComplete(ManagedLedger ledger, Object ctx) { long topicLoadLatencyMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()) - topicCreateTimeMs; pulsarStats.recordTopicLoadTimeValue(topic, topicLoadLatencyMs); - if (topicFuture.isCompletedExceptionally()) { + if (!topicFuture.complete(Optional.of(persistentTopic))) { // Check create persistent topic timeout. - log.warn("{} future is already completed with failure {}, closing the" - + " topic", topic, FutureUtil.getException(topicFuture)); + if (topicFuture.isCompletedExceptionally()) { + log.warn("{} future is already completed with failure {}, closing" + + " the topic", topic, FutureUtil.getException(topicFuture)); + } else { + // It should not happen. + log.error("{} future is already completed by another thread, " + + "which is not expected. Closing the current one", topic); + } executor().submit(() -> { persistentTopic.close().whenComplete((ignore, ex) -> { topics.remove(topic, topicFuture); @@ -1797,7 +1804,6 @@ public void openLedgerComplete(ManagedLedger ledger, Object ctx) { }); } else { addTopicToStatsMaps(topicName, persistentTopic); - topicFuture.complete(Optional.of(persistentTopic)); } }) .exceptionally((ex) -> { @@ -1826,6 +1832,7 @@ public void openLedgerComplete(ManagedLedger ledger, Object ctx) { public void openLedgerFailed(ManagedLedgerException exception, Object ctx) { if (!createIfMissing && exception instanceof ManagedLedgerNotFoundException) { // We were just trying to load a topic and the topic doesn't exist + pulsar.getExecutor().execute(() -> topics.remove(topic, topicFuture)); loadFuture.completeExceptionally(exception); topicFuture.complete(Optional.empty()); } else { 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 54b9ff1f2ea27..8dabab968fb25 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 @@ -19,18 +19,23 @@ package org.apache.pulsar.client.api; import static org.apache.pulsar.broker.service.persistent.PersistentTopic.DEDUPLICATION_CURSOR_NAME; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.broker.service.BrokerService; import org.apache.pulsar.broker.service.Topic; import org.apache.pulsar.broker.service.TopicPoliciesService; @@ -44,6 +49,7 @@ import org.awaitility.reflect.WhiteboxImpl; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Slf4j @@ -172,4 +178,93 @@ public CompletableFuture closeAsync() { pulsar.getConfig().setBrokerDeduplicationEnabled(false); pulsar.getConfig().setTransactionCoordinatorEnabled(false); } + + @DataProvider(name = "whetherTimeoutOrNot") + public Object[][] whetherTimeoutOrNot() { + return new Object[][] { + {true}, + {false} + }; + } + + @Test(timeOut = 60 * 1000, dataProvider = "whetherTimeoutOrNot") + public void testCheckOwnerShipFails(boolean injectTimeout) throws Exception { + if (injectTimeout) { + pulsar.getConfig().setTopicLoadTimeoutSeconds(5); + } + String ns = "public" + "/" + UUID.randomUUID().toString().replaceAll("-", ""); + String tpName = BrokerTestUtil.newUniqueName("persistent://" + ns + "/tp"); + admin.namespaces().createNamespace(ns); + admin.topics().createNonPartitionedTopic(tpName); + admin.namespaces().unload(ns); + + // Inject an error when calling "NamespaceService.isServiceUnitActiveAsync". + AtomicInteger failedTimes = new AtomicInteger(); + NamespaceService namespaceService = pulsar.getNamespaceService(); + doAnswer(invocation -> { + TopicName paramTp = (TopicName) invocation.getArguments()[0]; + if (paramTp.toString().equalsIgnoreCase(tpName) && failedTimes.incrementAndGet() <= 2) { + if (injectTimeout) { + Thread.sleep(10 * 1000); + } + log.info("Failed {} times", failedTimes.get()); + return CompletableFuture.failedFuture(new RuntimeException("mocked error")); + } + return invocation.callRealMethod(); + }).when(namespaceService).isServiceUnitActiveAsync(any(TopicName.class)); + + // Verify: the consumer can create successfully eventually. + Consumer consumer = pulsarClient.newConsumer().topic(tpName).subscriptionName("s1").subscribe(); + + // cleanup. + if (injectTimeout) { + pulsar.getConfig().setTopicLoadTimeoutSeconds(60); + } + consumer.close(); + admin.topics().delete(tpName); + } + + @Test(timeOut = 60 * 1000, dataProvider = "whetherTimeoutOrNot") + public void testTopicLoadAndDeleteAtTheSameTime(boolean injectTimeout) throws Exception { + if (injectTimeout) { + pulsar.getConfig().setTopicLoadTimeoutSeconds(5); + } + String ns = "public" + "/" + UUID.randomUUID().toString().replaceAll("-", ""); + String tpName = BrokerTestUtil.newUniqueName("persistent://" + ns + "/tp"); + admin.namespaces().createNamespace(ns); + admin.topics().createNonPartitionedTopic(tpName); + admin.namespaces().unload(ns); + + // Inject a race condition: load topic and delete topic execute at the same time. + AtomicInteger mockRaceConditionCounter = new AtomicInteger(); + NamespaceService namespaceService = pulsar.getNamespaceService(); + doAnswer(invocation -> { + TopicName paramTp = (TopicName) invocation.getArguments()[0]; + if (paramTp.toString().equalsIgnoreCase(tpName) && mockRaceConditionCounter.incrementAndGet() <= 1) { + if (injectTimeout) { + Thread.sleep(10 * 1000); + } + log.info("Race condition occurs {} times", mockRaceConditionCounter.get()); + pulsar.getManagedLedgerFactory().delete(TopicName.get(tpName).getPersistenceNamingEncoding()); + } + return invocation.callRealMethod(); + }).when(namespaceService).isServiceUnitActiveAsync(any(TopicName.class)); + + // Verify: the consumer create failed due to pulsar does not allow to create topic automatically. + try { + pulsar.getBrokerService().getTopic(tpName, false, Collections.emptyMap()).join(); + } catch (Exception ex) { + log.warn("Expected error", ex); + } + + // Verify: the consumer create successfully after allowing to create topic automatically. + Consumer consumer = pulsarClient.newConsumer().topic(tpName).subscriptionName("s1").subscribe(); + + // cleanup. + if (injectTimeout) { + pulsar.getConfig().setTopicLoadTimeoutSeconds(60); + } + consumer.close(); + admin.topics().delete(tpName); + } } From c29f36ab5bcfadd65b8910e69c7dbb913eeb4fc2 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Tue, 20 Aug 2024 23:32:22 +0800 Subject: [PATCH 680/699] [improve][broker] Optimize high CPU usage when consuming from topics with ongoing txn (#23189) (cherry picked from commit 94e1341d1e299bc93b809fc3046b70390de43592) (cherry picked from commit b7ffa73f2d8a528e956f5ba685dd5685545c6e91) --- .../org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 025d40e1e89f3..efb8aff683440 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 @@ -942,7 +942,7 @@ public void asyncReadEntriesWithSkipOrWait(int maxEntries, long maxSizeBytes, Re int numberOfEntriesToRead = applyMaxSizeCap(maxEntries, maxSizeBytes); - if (hasMoreEntries()) { + if (hasMoreEntries() && maxPosition.compareTo(readPosition) >= 0) { // If we have available entries, we can read them immediately if (log.isDebugEnabled()) { log.debug("[{}] [{}] Read entries immediately", ledger.getName(), name); From 336d5d64eb0f4160f9f149f5113b880b89911bf4 Mon Sep 17 00:00:00 2001 From: ken <1647023764@qq.com> Date: Wed, 21 Aug 2024 11:06:35 +0800 Subject: [PATCH 681/699] [improve][broker] Improve pulsar_topic_load_failed metric to record correct failed time (#23199) Co-authored-by: fanjianye (cherry picked from commit b661ec82d20adb71c0fe00ad115ec1ab71327880) (cherry picked from commit b7f5026618d443d3cbe06ef3c8554e1ee4ddfacb) --- .../apache/pulsar/broker/service/BrokerService.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 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 a0fe59370521b..d63121e38783c 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 @@ -1596,6 +1596,11 @@ protected CompletableFuture> loadOrCreatePersistentTopic(final S Duration.ofSeconds(pulsar.getConfiguration().getTopicLoadTimeoutSeconds()), executor(), () -> FAILED_TO_LOAD_TOPIC_TIMEOUT_EXCEPTION); + topicFuture.exceptionally(t -> { + pulsarStats.recordTopicLoadFailed(); + return null; + }); + checkTopicNsOwnership(topic) .thenRun(() -> { final Semaphore topicLoadSemaphore = topicLoadRequestSemaphore.get(); @@ -1703,11 +1708,6 @@ private void createPersistentTopic(final String topic, boolean createIfMissing, TopicName topicName = TopicName.get(topic); final long topicCreateTimeMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()); - topicFuture.exceptionally(t -> { - pulsarStats.recordTopicLoadFailed(); - return null; - }); - if (isTransactionInternalName(topicName)) { String msg = String.format("Can not create transaction system topic %s", topic); log.warn(msg); From 959f80b9c6683c2cb81985fb15b2bc5253ffacf4 Mon Sep 17 00:00:00 2001 From: Yunze Xu Date: Wed, 21 Aug 2024 14:17:44 +0800 Subject: [PATCH 682/699] [improve][client] Don't print info logs for each schema loaded by client (#23206) (cherry picked from commit 66e1a06b247cd032872e5ab454ff8c8c6cd98550) (cherry picked from commit 06ebe2a801e93fde3cb579f8c74dd0bd3dd9a71c) --- .../client/impl/schema/reader/MultiVersionAvroReader.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/reader/MultiVersionAvroReader.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/reader/MultiVersionAvroReader.java index 0ca847917eeca..85d4d63a1b136 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/reader/MultiVersionAvroReader.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/reader/MultiVersionAvroReader.java @@ -44,9 +44,11 @@ public MultiVersionAvroReader(Schema readerSchema, ClassLoader pojoClassLoader, protected SchemaReader loadReader(BytesSchemaVersion schemaVersion) { SchemaInfo schemaInfo = getSchemaInfoByVersion(schemaVersion.get()); if (schemaInfo != null) { - LOG.info("Load schema reader for version({}), schema is : {}, schemaInfo: {}", - SchemaUtils.getStringSchemaVersion(schemaVersion.get()), - schemaInfo.getSchemaDefinition(), schemaInfo.toString()); + if (LOG.isDebugEnabled()) { + LOG.debug("Load schema reader for version({}), schema is : {}, schemaInfo: {}", + SchemaUtils.getStringSchemaVersion(schemaVersion.get()), + schemaInfo.getSchemaDefinition(), schemaInfo); + } boolean jsr310ConversionEnabled = getJsr310ConversionEnabledFromSchemaInfo(schemaInfo); return new AvroReader<>(parseAvroSchema(schemaInfo.getSchemaDefinition()), readerSchema, pojoClassLoader, jsr310ConversionEnabled); From 8ad5249c12d8771a87ef4ce1c6bbe117118edf4d Mon Sep 17 00:00:00 2001 From: Xiangying Meng <55571188+liangyepianzhou@users.noreply.github.com> Date: Sat, 3 Aug 2024 15:30:22 +0800 Subject: [PATCH 683/699] [fix][client] the nullValue in msgMetadata should be true by default (#22372) Co-authored-by: xiangying ### Motivation When a message is not set value, the `nullValue` message metadata should be true and change to false after the value is set. Otherwise, the message data will be set as a [] when the value is not set, that would cause the message data to be encoded and throw a `SchemaSerializationException` when calling `reconsumerLater`. ``` org.apache.pulsar.client.api.PulsarClientException: java.util.concurrent.ExecutionException: org.apache.pulsar.client.api.SchemaSerializationException: Size of data received by IntSchema is not 4 at org.apache.pulsar.client.api.PulsarClientException.unwrap(PulsarClientException.java:1131) at org.apache.pulsar.client.impl.ConsumerBase.reconsumeLater(ConsumerBase.java:467) at org.apache.pulsar.client.impl.ConsumerBase.reconsumeLater(ConsumerBase.java:452) at org.apache.pulsar.client.api.ConsumerRedeliveryTest.testRedeliverMessagesWithoutValue(ConsumerRedeliveryTest.java:445) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77) at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.base/java.lang.reflect.Method.invoke(Method.java:568) at org.testng.internal.invokers.MethodInvocationHelper.invokeMethod(MethodInvocationHelper.java:139) at org.testng.internal.invokers.TestInvoker.invokeMethod(TestInvoker.java:677) at org.testng.internal.invokers.TestInvoker.invokeTestMethod(TestInvoker.java:221) at org.testng.internal.invokers.MethodRunner.runInSequence(MethodRunner.java:50) at org.testng.internal.invokers.TestInvoker$MethodInvocationAgent.invoke(TestInvoker.java:969) at org.testng.internal.invokers.TestInvoker.invokeTestMethods(TestInvoker.java:194) at org.testng.internal.invokers.TestMethodWorker.invokeTestMethods(TestMethodWorker.java:148) at org.testng.internal.invokers.TestMethodWorker.run(TestMethodWorker.java:128) at java.base/java.util.ArrayList.forEach(ArrayList.java:1511) at org.testng.TestRunner.privateRun(TestRunner.java:829) at org.testng.TestRunner.run(TestRunner.java:602) at org.testng.SuiteRunner.runTest(SuiteRunner.java:437) at org.testng.SuiteRunner.runSequentially(SuiteRunner.java:431) at org.testng.SuiteRunner.privateRun(SuiteRunner.java:391) at org.testng.SuiteRunner.run(SuiteRunner.java:330) at org.testng.SuiteRunnerWorker.runSuite(SuiteRunnerWorker.java:52) at org.testng.SuiteRunnerWorker.run(SuiteRunnerWorker.java:95) at org.testng.TestNG.runSuitesSequentially(TestNG.java:1256) at org.testng.TestNG.runSuitesLocally(TestNG.java:1176) at org.testng.TestNG.runSuites(TestNG.java:1099) at org.testng.TestNG.run(TestNG.java:1067) at com.intellij.rt.testng.IDEARemoteTestNG.run(IDEARemoteTestNG.java:65) at com.intellij.rt.testng.RemoteTestNGStarter.main(RemoteTestNGStarter.java:105) Caused by: java.util.concurrent.ExecutionException: org.apache.pulsar.client.api.SchemaSerializationException: Size of data received by IntSchema is not 4 at java.base/java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:396) at java.base/java.util.concurrent.CompletableFuture.get(CompletableFuture.java:2073) at org.apache.pulsar.client.impl.ConsumerBase.reconsumeLater(ConsumerBase.java:462) ... 29 more Caused by: org.apache.pulsar.client.api.SchemaSerializationException: Size of data received by IntSchema is not 4 at org.apache.pulsar.client.impl.schema.IntSchema.validate(IntSchema.java:49) at org.apache.pulsar.client.impl.schema.AutoProduceBytesSchema.encode(AutoProduceBytesSchema.java:80) at org.apache.pulsar.client.impl.schema.AutoProduceBytesSchema.encode(AutoProduceBytesSchema.java:32) at org.apache.pulsar.client.impl.TypedMessageBuilderImpl.lambda$value$3(TypedMessageBuilderImpl.java:157) at java.base/java.util.Optional.orElseGet(Optional.java:364) at org.apache.pulsar.client.impl.TypedMessageBuilderImpl.value(TypedMessageBuilderImpl.java:156) at org.apache.pulsar.client.impl.ConsumerImpl.doReconsumeLater(ConsumerImpl.java:689) at org.apache.pulsar.client.impl.MultiTopicsConsumerImpl.doReconsumeLater(MultiTopicsConsumerImpl.java:550) at org.apache.pulsar.client.impl.ConsumerBase.reconsumeLaterAsync(ConsumerBase.java:574) ``` ### Modifications When a message is not set value, the `nullValue` message metadata should be true and change to false after the value is set. (cherry picked from commit f3c177e2243e26a7849feb91dbed9fec4c5723c0) (cherry picked from commit 054117603cb6e5624fa6fb5d1a2a50f33b9b9fae) --- .../client/api/ConsumerRedeliveryTest.java | 24 +++++++++++++ .../client/impl/MessageChecksumTest.java | 5 +++ .../pulsar/compaction/CompactionTest.java | 2 +- .../client/impl/TypedMessageBuilderImpl.java | 35 ++++++++++--------- .../impl/TypedMessageBuilderImplTest.java | 17 +++++++-- 5 files changed, 63 insertions(+), 20 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ConsumerRedeliveryTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ConsumerRedeliveryTest.java index 90114add25084..fcf1a638d5884 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ConsumerRedeliveryTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/ConsumerRedeliveryTest.java @@ -424,4 +424,28 @@ public void testAckNotSent(int numAcked, int batchSize, CommandAck.AckType ackTy assertTrue(values.isEmpty()); } } + + @Test + public void testRedeliverMessagesWithoutValue() throws Exception { + String topic = "persistent://my-property/my-ns/testRedeliverMessagesWithoutValue"; + @Cleanup Consumer consumer = pulsarClient.newConsumer(Schema.INT32) + .topic(topic) + .subscriptionName("sub") + .enableRetry(true) + .subscribe(); + @Cleanup Producer producer = pulsarClient.newProducer(Schema.INT32) + .topic(topic) + .enableBatching(true) + .create(); + for (int i = 0; i < 10; i++) { + producer.newMessage().key("messages without value").send(); + } + + Message message = consumer.receive(); + consumer.reconsumeLater(message, 2, TimeUnit.SECONDS); + for (int i = 0; i < 9; i++) { + assertNotNull(consumer.receive(5, TimeUnit.SECONDS)); + } + assertTrue(consumer.receive(5, TimeUnit.SECONDS).getTopicName().contains("sub-RETRY")); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java index 515b34db8509d..f84c840fc3be9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/MessageChecksumTest.java @@ -24,6 +24,8 @@ import static org.testng.Assert.fail; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; + +import java.lang.reflect.Method; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -224,6 +226,9 @@ public void testTamperingMessageIsDetected() throws Exception { .create(); TypedMessageBuilderImpl msgBuilder = (TypedMessageBuilderImpl) producer.newMessage() .value("a message".getBytes()); + Method method = TypedMessageBuilderImpl.class.getDeclaredMethod("beforeSend"); + method.setAccessible(true); + method.invoke(msgBuilder); MessageMetadata msgMetadata = msgBuilder.getMetadataBuilder() .setProducerName("test") .setSequenceId(1) 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 a07f2ca8391af..ea95626fbe741 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 @@ -1385,7 +1385,7 @@ public void testEmptyPayloadDeletesWhenEncrypted() throws Exception { Message message4 = consumer.receive(); Assert.assertEquals(message4.getKey(), "key2"); - Assert.assertEquals(new String(message4.getData()), ""); + assertNull(message4.getData()); Message message5 = consumer.receive(); Assert.assertEquals(message5.getKey(), "key4"); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java index 026f8a1e69e0b..d90c2e8828364 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TypedMessageBuilderImpl.java @@ -50,6 +50,7 @@ public class TypedMessageBuilderImpl implements TypedMessageBuilder { private final transient Schema schema; private transient ByteBuffer content; private final transient TransactionImpl txn; + private transient T value; public TypedMessageBuilderImpl(ProducerBase producer, Schema schema) { this(producer, schema, null); @@ -65,6 +66,22 @@ public TypedMessageBuilderImpl(ProducerBase producer, } private long beforeSend() { + if (value == null) { + msgMetadata.setNullValue(true); + } else { + getKeyValueSchema().map(keyValueSchema -> { + if (keyValueSchema.getKeyValueEncodingType() == KeyValueEncodingType.SEPARATED) { + setSeparateKeyValue(value, keyValueSchema); + return this; + } else { + return null; + } + }).orElseGet(() -> { + content = ByteBuffer.wrap(schema.encode(value)); + return this; + }); + } + if (txn == null) { return -1L; } @@ -140,22 +157,8 @@ public TypedMessageBuilder orderingKey(byte[] orderingKey) { @Override public TypedMessageBuilder value(T value) { - if (value == null) { - msgMetadata.setNullValue(true); - return this; - } - - return getKeyValueSchema().map(keyValueSchema -> { - if (keyValueSchema.getKeyValueEncodingType() == KeyValueEncodingType.SEPARATED) { - setSeparateKeyValue(value, keyValueSchema); - return this; - } else { - return null; - } - }).orElseGet(() -> { - content = ByteBuffer.wrap(schema.encode(value)); - return this; - }); + this.value = value; + return this; } @Override diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TypedMessageBuilderImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TypedMessageBuilderImplTest.java index 94c683e527177..05db4402a1586 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TypedMessageBuilderImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TypedMessageBuilderImplTest.java @@ -27,6 +27,8 @@ import org.mockito.Mock; import org.testng.annotations.Test; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.nio.ByteBuffer; import java.util.Base64; @@ -45,7 +47,7 @@ public class TypedMessageBuilderImplTest { protected ProducerBase producerBase; @Test - public void testDefaultValue() { + public void testDefaultValue() throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { producerBase = mock(ProducerBase.class); AvroSchema fooSchema = AvroSchema.of(SchemaDefinition.builder().withPojo(SchemaTestUtils.Foo.class).build()); @@ -63,6 +65,9 @@ public void testDefaultValue() { // Check kv.encoding.type default, not set value TypedMessageBuilderImpl typedMessageBuilder = (TypedMessageBuilderImpl)typedMessageBuilderImpl.value(keyValue); + Method method = TypedMessageBuilderImpl.class.getDeclaredMethod("beforeSend"); + method.setAccessible(true); + method.invoke(typedMessageBuilder); ByteBuffer content = typedMessageBuilder.getContent(); byte[] contentByte = new byte[content.remaining()]; content.get(contentByte); @@ -73,7 +78,7 @@ public void testDefaultValue() { } @Test - public void testInlineValue() { + public void testInlineValue() throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { producerBase = mock(ProducerBase.class); AvroSchema fooSchema = AvroSchema.of(SchemaDefinition.builder().withPojo(SchemaTestUtils.Foo.class).build()); @@ -91,6 +96,9 @@ public void testInlineValue() { // Check kv.encoding.type INLINE TypedMessageBuilderImpl typedMessageBuilder = (TypedMessageBuilderImpl)typedMessageBuilderImpl.value(keyValue); + Method method = TypedMessageBuilderImpl.class.getDeclaredMethod("beforeSend"); + method.setAccessible(true); + method.invoke(typedMessageBuilder); ByteBuffer content = typedMessageBuilder.getContent(); byte[] contentByte = new byte[content.remaining()]; content.get(contentByte); @@ -101,7 +109,7 @@ public void testInlineValue() { } @Test - public void testSeparatedValue() { + public void testSeparatedValue() throws Exception { producerBase = mock(ProducerBase.class); AvroSchema fooSchema = AvroSchema.of(SchemaDefinition.builder().withPojo(SchemaTestUtils.Foo.class).build()); @@ -119,6 +127,9 @@ public void testSeparatedValue() { // Check kv.encoding.type SEPARATED TypedMessageBuilderImpl typedMessageBuilder = (TypedMessageBuilderImpl)typedMessageBuilderImpl.value(keyValue); + Method method = TypedMessageBuilderImpl.class.getDeclaredMethod("beforeSend"); + method.setAccessible(true); + method.invoke(typedMessageBuilder); ByteBuffer content = typedMessageBuilder.getContent(); byte[] contentByte = new byte[content.remaining()]; content.get(contentByte); From 7b7d3354aee45fb42b3da876acb121a5a8a1d1fc Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 22 Aug 2024 11:13:55 +0800 Subject: [PATCH 684/699] [fix] [log] Do not print warn log when concurrently publishing and switching ledgers (#23209) (cherry picked from commit 0a5cb51a2f010d6771ae0ae0fd259d002cca20da) (cherry picked from commit 57b0ca403a66b805745aed2f9c1813ce937b433b) --- .../mledger/impl/ManagedLedgerImpl.java | 5 +- ...ProducerConsumerMLInitializeDelayTest.java | 70 +++++++++++++++++++ 2 files changed, 73 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 a540828000e1c..beccda60cc1b6 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 @@ -1682,8 +1682,9 @@ void createNewOpAddEntryForNewLedger() { if (existsOp.ledger != null) { existsOp = existsOp.duplicateAndClose(currentLedgerTimeoutTriggered); } else { - // This scenario should not happen. - log.warn("[{}] An OpAddEntry's ledger is empty.", name); + // It may happen when the following operations execute at the same time, so it is expected. + // - Adding entry. + // - Switching ledger. existsOp.setTimeoutTriggered(currentLedgerTimeoutTriggered); } existsOp.setLedger(currentLedger); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerMLInitializeDelayTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerMLInitializeDelayTest.java index ab4e063ae3d83..7c7665a5bd3e4 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerMLInitializeDelayTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/SimpleProducerConsumerMLInitializeDelayTest.java @@ -20,12 +20,19 @@ import com.carrotsearch.hppc.ObjectSet; import java.time.Duration; +import java.util.ArrayList; +import java.util.LinkedHashSet; import java.util.List; +import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.mledger.ManagedLedgerConfig; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.service.Dispatcher; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.impl.MessageIdImpl; import org.apache.pulsar.common.naming.TopicName; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; @@ -105,4 +112,67 @@ public void testConsumerListMatchesConsumerSet() throws Exception { // cleanup. client.close(); } + + @Test(timeOut = 30 * 1000) + public void testConcurrentlyOfPublishAndSwitchLedger() 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); + // Make ledger switches faster. + PersistentTopic persistentTopic = + (PersistentTopic) pulsar.getBrokerService().getTopic(topicName, false).join().get(); + ManagedLedgerConfig config = persistentTopic.getManagedLedger().getConfig(); + config.setMaxEntriesPerLedger(2); + config.setMinimumRolloverTime(0, TimeUnit.MILLISECONDS); + // Inject a delay for switching ledgers, so publishing requests will be push in to the pending queue. + AtomicInteger delayTimes = new AtomicInteger(); + mockZooKeeper.delay(10, (op, s) -> { + if (op.toString().equals("SET") && s.contains(TopicName.get(topicName).getPersistenceNamingEncoding())) { + return delayTimes.incrementAndGet() == 1; + } + return false; + }); + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topicName).enableBatching(false) + .create(); + List> sendRequests = new ArrayList<>(); + List msgsSent = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + String msg = i + ""; + sendRequests.add(producer.sendAsync(i + "")); + msgsSent.add(msg); + } + // Verify: + // - All messages were sent. + // - The order of messages are correct. + Set msgIds = new LinkedHashSet<>(); + MessageIdImpl previousMsgId = null; + for (CompletableFuture msgId : sendRequests) { + Assert.assertNotNull(msgId.join()); + MessageIdImpl messageIdImpl = (MessageIdImpl) msgId.join(); + if (previousMsgId != null) { + Assert.assertTrue(messageIdImpl.compareTo(previousMsgId) > 0); + } + msgIds.add(String.format("%s:%s", messageIdImpl.getLedgerId(), messageIdImpl.getEntryId())); + previousMsgId = messageIdImpl; + } + Assert.assertEquals(msgIds.size(), 100); + log.info("messages were sent: {}", msgIds.toString()); + List msgsReceived = new ArrayList<>(); + Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName) + .subscriptionName(subscription).subscribe(); + while (true) { + Message receivedMsg = consumer.receive(2, TimeUnit.SECONDS); + if (receivedMsg == null) { + break; + } + msgsReceived.add(receivedMsg.getValue()); + } + Assert.assertEquals(msgsReceived, msgsSent); + + // cleanup. + consumer.close(); + producer.close(); + admin.topics().delete(topicName); + } } From 751aea3ca574b6fefe8f9d7cd5ed681db9aa74fa Mon Sep 17 00:00:00 2001 From: Paul Gier Date: Mon, 26 Aug 2024 12:38:23 -0400 Subject: [PATCH 685/699] AS-4027: add string write function (#308) This function was recently changed causing a failure in broker metrics. --- .../pulsar/common/util/SimpleTextOutputStream.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java index 9bf6302f50f02..d3f319bd958ba 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/SimpleTextOutputStream.java @@ -56,6 +56,15 @@ public SimpleTextOutputStream write(char c) { return this; } + public SimpleTextOutputStream write(String s) { + if (s == null) { + return this; + } + + buffer.writeCharSequence(s, CharsetUtil.UTF_8); + return this; + } + public SimpleTextOutputStream write(CharSequence s) { if (s == null) { return this; From ada390bb0c2d43aa37aa73f2efee048b2e770abe Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 23 Aug 2024 08:31:49 +0800 Subject: [PATCH 686/699] [improve] [broker] Phase 1 of PIP-370 support disable create topics on remote cluster through replication (#23169) (cherry picked from commit 44f986014e4d314a4a52484856c7dfb2d89ea3c1) (cherry picked from commit 9553c33481952c7354f1ab44bcfd3592aa0c7455) --- conf/broker.conf | 10 + conf/standalone.conf | 10 + .../pulsar/broker/ServiceConfiguration.java | 5 + .../pulsar/broker/admin/AdminResource.java | 10 +- .../admin/impl/PersistentTopicsBase.java | 6 + .../broker/service/AbstractReplicator.java | 16 +- .../persistent/GeoPersistentReplicator.java | 29 +++ ...opicToRemoteClusterForReplicationTest.java | 208 ++++++++++++++++++ .../pulsar/broker/service/StandaloneTest.java | 1 + .../naming/ServiceConfigurationTest.java | 5 + .../configurations/pulsar_broker_test.conf | 1 + .../pulsar_broker_test_standalone.conf | 3 +- .../pulsar/client/impl/PulsarClientImpl.java | 2 +- 13 files changed, 297 insertions(+), 9 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DisabledCreateTopicToRemoteClusterForReplicationTest.java diff --git a/conf/broker.conf b/conf/broker.conf index 77020fd5c9994..d4d803530f570 100644 --- a/conf/broker.conf +++ b/conf/broker.conf @@ -1548,6 +1548,16 @@ replicatorPrefix=pulsar.repl # due to missing ZooKeeper watch (disable with value 0) replicationPolicyCheckDurationSeconds=600 +# Whether the internal replication of the local cluster will trigger topic auto-creation on the remote cluster. +# 1. After enabling namespace-level Geo-Replication: whether the local broker will create topics on the remote +# cluster automatically when calling `pulsar-admin topics create-partitioned-topic`. +# 2. When enabling topic-level Geo-Replication on a partitioned topic: whether the local broker will create topics on +# the remote cluster. +# 3. Whether the internal Geo-Replicator in the local cluster will trigger non-persistent topic auto-creation for +# remote clusters. +# It is not a dynamic config, the default value is "true" to preserve backward-compatible behavior. +createTopicToRemoteClusterForReplication=true + # Default message retention time. # 0 means retention is disabled. -1 means data is not removed by time quota. defaultRetentionTimeInMinutes=0 diff --git a/conf/standalone.conf b/conf/standalone.conf index d9b368d2952ca..773ec5497b781 100644 --- a/conf/standalone.conf +++ b/conf/standalone.conf @@ -953,6 +953,16 @@ replicationProducerQueueSize=1000 # due to missing ZooKeeper watch (disable with value 0) replicationPolicyCheckDurationSeconds=600 +# Whether the internal replication of the local cluster will trigger topic auto-creation on the remote cluster. +# 1. After enabling namespace-level Geo-Replication: whether the local broker will create topics on the remote +# cluster automatically when calling `pulsar-admin topics create-partitioned-topic`. +# 2. When enabling topic-level Geo-Replication on a partitioned topic: whether the local broker will create topics on +# the remote cluster. +# 3. Whether the internal Geo-Replicator in the local cluster will trigger non-persistent topic auto-creation for +# remote clusters. +# It is not a dynamic config, the default value is "true" to preserve backward-compatible behavior. +createTopicToRemoteClusterForReplication=true + # Default message retention time. 0 means retention is disabled. -1 means data is not removed by time quota defaultRetentionTimeInMinutes=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 8e71c1dec578d..5321a420fb4b4 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 @@ -2789,6 +2789,11 @@ The delayed message index time step(in seconds) in per bucket snapshot segment, + "inconsistency due to missing ZooKeeper watch (disable with value 0)" ) private int replicationPolicyCheckDurationSeconds = 600; + @FieldContext( + category = CATEGORY_REPLICATION, + doc = "Whether the internal replicator will trigger topic auto-creation on the remote cluster." + ) + private boolean createTopicToRemoteClusterForReplication = true; @Deprecated @FieldContext( category = CATEGORY_REPLICATION, 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 9183630bb8116..d61c6115dccb4 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 @@ -602,11 +602,15 @@ protected void internalCreatePartitionedTopic(AsyncResponse asyncResponse, int n .thenCompose(__ -> provisionPartitionedTopicPath(numPartitions, createLocalTopicOnly, properties)) .thenCompose(__ -> tryCreatePartitionsAsync(numPartitions)) .thenRun(() -> { - if (!createLocalTopicOnly && topicName.isGlobal()) { + if (!createLocalTopicOnly && topicName.isGlobal() + && pulsar().getConfig().isCreateTopicToRemoteClusterForReplication()) { internalCreatePartitionedTopicToReplicatedClustersInBackground(numPartitions); + log.info("[{}] Successfully created partitioned for topic {} for the remote clusters", + clientAppId()); + } else { + log.info("[{}] Skip creating partitioned for topic {} for the remote clusters", + clientAppId(), topicName); } - log.info("[{}] Successfully created partitions for topic {} in cluster {}", - clientAppId(), topicName, pulsar().getConfiguration().getClusterName()); asyncResponse.resume(Response.noContent().build()); }) .exceptionally(ex -> { 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 3b67808403f60..6a7c39c0917b7 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 @@ -3484,6 +3484,12 @@ protected CompletableFuture internalSetReplicationClusters(List cl } return FutureUtil.waitForAll(futures); }).thenCompose(__ -> { + if (!pulsar().getConfig().isCreateTopicToRemoteClusterForReplication()) { + log.info("[{}] Skip creating partitioned for topic {} for the remote clusters {}", + clientAppId(), topicName, replicationClusters.stream().filter(v -> + !pulsar().getConfig().getClusterName().equals(v)).collect(Collectors.toList())); + return CompletableFuture.completedFuture(null); + } // Sync to create partitioned topic on the remote cluster if needed. TopicName topicNameWithoutPartition = TopicName.get(topicName.getPartitionedTopicName()); return pulsar().getPulsarResources().getNamespaceResources().getPartitionedTopicResources() diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java index d7769d233bf60..919771832fdb0 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractReplicator.java @@ -141,6 +141,10 @@ public String getRemoteCluster() { return remoteCluster; } + protected CompletableFuture prepareCreateProducer() { + return CompletableFuture.completedFuture(null); + } + public void startProducer() { // Guarantee only one task call "producerBuilder.createAsync()". Pair setStartingRes = compareSetAndGetState(State.Disconnected, State.Starting); @@ -167,12 +171,15 @@ public void startProducer() { } log.info("[{}] Starting replicator", replicatorId); + // Force only replicate messages to a non-partitioned topic, to avoid auto-create a partitioned topic on // the remote cluster. - ProducerBuilderImpl builderImpl = (ProducerBuilderImpl) producerBuilder; - builderImpl.getConf().setNonPartitionedTopicExpected(true); - producerBuilder.createAsync().thenAccept(producer -> { - setProducerAndTriggerReadEntries(producer); + prepareCreateProducer().thenCompose(ignore -> { + ProducerBuilderImpl builderImpl = (ProducerBuilderImpl) producerBuilder; + builderImpl.getConf().setNonPartitionedTopicExpected(true); + return producerBuilder.createAsync().thenAccept(producer -> { + setProducerAndTriggerReadEntries(producer); + }); }).exceptionally(ex -> { Pair setDisconnectedRes = compareSetAndGetState(State.Starting, State.Disconnected); if (setDisconnectedRes.getLeft()) { @@ -197,6 +204,7 @@ public void startProducer() { } return null; }); + } /*** diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java index 082dfed10c664..3390c3a288526 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/GeoPersistentReplicator.java @@ -27,11 +27,13 @@ import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.pulsar.broker.PulsarServerException; import org.apache.pulsar.broker.service.BrokerService; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.transaction.TxnID; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.protocol.Markers; import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.util.FutureUtil; @Slf4j public class GeoPersistentReplicator extends PersistentReplicator { @@ -51,6 +53,33 @@ protected String getProducerName() { return getReplicatorName(replicatorPrefix, localCluster) + REPL_PRODUCER_NAME_DELIMITER + remoteCluster; } + @Override + protected CompletableFuture prepareCreateProducer() { + if (brokerService.getPulsar().getConfig().isCreateTopicToRemoteClusterForReplication()) { + return CompletableFuture.completedFuture(null); + } else { + CompletableFuture topicCheckFuture = new CompletableFuture<>(); + replicationClient.getPartitionedTopicMetadata(localTopic.getName(), false, false) + .whenComplete((metadata, ex) -> { + if (ex == null) { + if (metadata.partitions == 0) { + topicCheckFuture.complete(null); + } else { + String errorMsg = String.format("{} Can not create the replicator due to the partitions in the" + + " remote cluster is not 0, but is %s", + replicatorId, metadata.partitions); + log.error(errorMsg); + topicCheckFuture.completeExceptionally( + new PulsarClientException.NotAllowedException(errorMsg)); + } + } else { + topicCheckFuture.completeExceptionally(FutureUtil.unwrapCompletionException(ex)); + } + }); + return topicCheckFuture; + } + } + @Override protected boolean replicateEntries(List entries) { boolean atLeastOneMessageSentForReplication = false; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DisabledCreateTopicToRemoteClusterForReplicationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DisabledCreateTopicToRemoteClusterForReplicationTest.java new file mode 100644 index 0000000000000..0f8db4aaa7316 --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/DisabledCreateTopicToRemoteClusterForReplicationTest.java @@ -0,0 +1,208 @@ +/* + * 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 static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; +import java.util.Arrays; +import java.util.HashSet; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Consumer; +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.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; +import org.apache.pulsar.zookeeper.ZookeeperServerTest; +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 DisabledCreateTopicToRemoteClusterForReplicationTest extends OneWayReplicatorTestBase { + + @Override + @BeforeClass(alwaysRun = true, timeOut = 300000) + public void setup() throws Exception { + super.setup(); + admin1.namespaces().setRetention(replicatedNamespace, new RetentionPolicies(300, 1024)); + admin2.namespaces().setRetention(replicatedNamespace, new RetentionPolicies(300, 1024)); + admin1.namespaces().setRetention(nonReplicatedNamespace, new RetentionPolicies(300, 1024)); + admin2.namespaces().setRetention(nonReplicatedNamespace, new RetentionPolicies(300, 1024)); + } + + @Override + @AfterClass(alwaysRun = true, timeOut = 300000) + public void cleanup() throws Exception { + super.cleanup(); + } + + @Override + protected void setConfigDefaults(ServiceConfiguration config, String clusterName, + LocalBookkeeperEnsemble bookkeeperEnsemble, ZookeeperServerTest brokerConfigZk) { + super.setConfigDefaults(config, clusterName, bookkeeperEnsemble, brokerConfigZk); + config.setCreateTopicToRemoteClusterForReplication(false); + config.setReplicationStartAt("earliest"); + } + + @Test + public void testCreatePartitionedTopicWithNsReplication() throws Exception { + String ns = defaultTenant + "/" + UUID.randomUUID().toString().replace("-", ""); + admin1.namespaces().createNamespace(ns); + admin2.namespaces().createNamespace(ns); + admin1.namespaces().setRetention(ns, new RetentionPolicies(3600, -1)); + admin2.namespaces().setRetention(ns, new RetentionPolicies(3600, -1)); + + // Create non-partitioned topic. + // Enable replication. + final String tp = BrokerTestUtil.newUniqueName("persistent://" + ns + "/tp_"); + final String part1 = TopicName.get(tp).getPartition(0).toString(); + admin1.topics().createPartitionedTopic(tp, 1); + admin1.namespaces().setNamespaceReplicationClusters(ns, new HashSet<>(Arrays.asList(cluster1, cluster2))); + + // Trigger and wait for replicator starts. + String msgValue = "msg-1"; + Producer producer1 = client1.newProducer(Schema.STRING).topic(tp).create(); + producer1.send(msgValue); + producer1.close(); + Awaitility.await().untilAsserted(() -> { + PersistentTopic topicPart1 = (PersistentTopic) broker1.getTopic(part1, false).join().get(); + assertFalse(topicPart1.getReplicators().isEmpty()); + }); + + // Verify: there is no topic with the same name on the remote cluster. + try { + admin2.topics().getPartitionedTopicMetadata(tp); + fail("Expected a not found ex"); + } catch (PulsarAdminException.NotFoundException ex) { + // expected. + } + + // Verify: after creating the topic on the remote cluster, all things are fine. + admin2.topics().createPartitionedTopic(tp, 1); + Consumer consumer2 = client2.newConsumer(Schema.STRING).topic(tp).isAckReceiptEnabled(true) + .subscriptionName("s1").subscribe(); + assertEquals(consumer2.receive(10, TimeUnit.SECONDS).getValue(), msgValue); + consumer2.close(); + + // cleanup. + admin1.namespaces().setNamespaceReplicationClusters(ns, new HashSet<>(Arrays.asList(cluster1))); + Awaitility.await().untilAsserted(() -> { + PersistentTopic topicPart1 = (PersistentTopic) broker1.getTopic(part1, false).join().get(); + assertTrue(topicPart1.getReplicators().isEmpty()); + }); + admin1.topics().deletePartitionedTopic(tp, false); + admin2.topics().deletePartitionedTopic(tp, false); + admin1.namespaces().deleteNamespace(ns); + admin2.namespaces().deleteNamespace(ns); + } + + @Test + public void testEnableTopicReplication() throws Exception { + String ns = nonReplicatedNamespace; + + // Create non-partitioned topic. + // Enable replication. + final String tp = BrokerTestUtil.newUniqueName("persistent://" + ns + "/tp_"); + final String part1 = TopicName.get(tp).getPartition(0).toString(); + admin1.topics().createPartitionedTopic(tp, 1); + admin1.topics().setReplicationClusters(tp, Arrays.asList(cluster1, cluster2)); + + // Trigger and wait for replicator starts. + Producer p1 = client1.newProducer(Schema.STRING).topic(tp).create(); + p1.send("msg-1"); + p1.close(); + Awaitility.await().untilAsserted(() -> { + PersistentTopic topicPart1 = (PersistentTopic) broker1.getTopic(part1, false).join().get(); + assertFalse(topicPart1.getReplicators().isEmpty()); + }); + + // Verify: there is no topic with the same name on the remote cluster. + try { + admin2.topics().getPartitionedTopicMetadata(tp); + fail("Expected a not found ex"); + } catch (PulsarAdminException.NotFoundException ex) { + // expected. + } + + // Verify: after creating the topic on the remote cluster, all things are fine. + admin2.topics().createPartitionedTopic(tp, 1); + waitReplicatorStarted(part1); + + // cleanup. + admin1.topics().setReplicationClusters(tp, Arrays.asList(cluster1)); + Awaitility.await().untilAsserted(() -> { + PersistentTopic topicPart1 = (PersistentTopic) broker1.getTopic(part1, false).join().get(); + assertTrue(topicPart1.getReplicators().isEmpty()); + }); + admin1.topics().deletePartitionedTopic(tp, false); + admin2.topics().deletePartitionedTopic(tp, false); + } + + @Test + public void testNonPartitionedTopic() throws Exception { + String ns = nonReplicatedNamespace; + + // Create non-partitioned topic. + // Enable replication. + final String tp = BrokerTestUtil.newUniqueName("persistent://" + ns + "/tp_"); + admin1.topics().createNonPartitionedTopic(tp); + admin1.topics().setReplicationClusters(tp, Arrays.asList(cluster1, cluster2)); + + // Trigger and wait for replicator starts. + Producer p1 = client1.newProducer(Schema.STRING).topic(tp).create(); + p1.send("msg-1"); + p1.close(); + Awaitility.await().untilAsserted(() -> { + PersistentTopic topicPart1 = (PersistentTopic) broker1.getTopic(tp, false).join().get(); + assertFalse(topicPart1.getReplicators().isEmpty()); + }); + + // Verify: there is no topic with the same name on the remote cluster. + try { + admin2.topics().getPartitionedTopicMetadata(tp); + fail("Expected a not found ex"); + } catch (PulsarAdminException.NotFoundException ex) { + // expected. + } + + // Verify: after creating the topic on the remote cluster, all things are fine. + admin2.topics().createNonPartitionedTopic(tp); + waitReplicatorStarted(tp); + + // cleanup. + admin1.topics().setReplicationClusters(tp, Arrays.asList(cluster1)); + Awaitility.await().untilAsserted(() -> { + PersistentTopic topicPart1 = (PersistentTopic) broker1.getTopic(tp, false).join().get(); + assertTrue(topicPart1.getReplicators().isEmpty()); + }); + admin1.topics().delete(tp, false); + admin2.topics().delete(tp, false); + } +} diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java index 67d188efd2550..fecf103ddbe48 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/StandaloneTest.java @@ -58,5 +58,6 @@ public void testAdvertised() throws Exception { "internal:pulsar://192.168.1.11:6660,internal:pulsar+ssl://192.168.1.11:6651"); assertEquals(standalone.getConfig().getMaxSecondsToClearTopicNameCache(), 1); assertEquals(standalone.getConfig().getTopicNameCacheMaxCapacity(), 200); + assertEquals(standalone.getConfig().isCreateTopicToRemoteClusterForReplication(), true); } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java index ae13afb19344b..ed108c7d1625d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/common/naming/ServiceConfigurationTest.java @@ -75,6 +75,7 @@ public void testInit() throws Exception { assertEquals(config.getHttpMaxRequestHeaderSize(), 1234); assertEquals(config.getMaxSecondsToClearTopicNameCache(), 1); assertEquals(config.getTopicNameCacheMaxCapacity(), 200); + assertEquals(config.isCreateTopicToRemoteClusterForReplication(), false); OffloadPoliciesImpl offloadPolicies = OffloadPoliciesImpl.create(config.getProperties()); assertEquals(offloadPolicies.getManagedLedgerOffloadedReadPriority().getValue(), "bookkeeper-first"); } @@ -291,6 +292,7 @@ public void testTransactionBatchConfigurations() throws Exception{ assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxRecords(), 512); assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxSize(), 1024 * 1024 * 4); assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxDelayInMillis(), 1); + assertEquals(configuration.isCreateTopicToRemoteClusterForReplication(), true); } // pulsar_broker_test.conf. try (InputStream inputStream = this.getClass().getClassLoader().getResourceAsStream(fileName)) { @@ -303,6 +305,7 @@ public void testTransactionBatchConfigurations() throws Exception{ assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxRecords(), 44); assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxSize(), 55); assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxDelayInMillis(), 66); + assertEquals(configuration.isCreateTopicToRemoteClusterForReplication(), false); } // string input stream. StringBuilder stringBuilder = new StringBuilder(); @@ -314,6 +317,7 @@ public void testTransactionBatchConfigurations() throws Exception{ stringBuilder.append("transactionPendingAckBatchedWriteMaxRecords=521").append(System.lineSeparator()); stringBuilder.append("transactionPendingAckBatchedWriteMaxSize=1025").append(System.lineSeparator()); stringBuilder.append("transactionPendingAckBatchedWriteMaxDelayInMillis=20").append(System.lineSeparator()); + stringBuilder.append("createTopicToRemoteClusterForReplication=false").append(System.lineSeparator()); try(ByteArrayInputStream inputStream = new ByteArrayInputStream(stringBuilder.toString().getBytes(StandardCharsets.UTF_8))){ configuration = PulsarConfigurationLoader.create(inputStream, ServiceConfiguration.class); @@ -325,6 +329,7 @@ public void testTransactionBatchConfigurations() throws Exception{ assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxRecords(), 521); assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxSize(), 1025); assertEquals(configuration.getTransactionPendingAckBatchedWriteMaxDelayInMillis(), 20); + assertEquals(configuration.isCreateTopicToRemoteClusterForReplication(), false); } } diff --git a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf index 551a9c88757a4..0d249693285d6 100644 --- a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf +++ b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test.conf @@ -105,3 +105,4 @@ transactionPendingAckBatchedWriteMaxSize=55 transactionPendingAckBatchedWriteMaxDelayInMillis=66 topicNameCacheMaxCapacity=200 maxSecondsToClearTopicNameCache=1 +createTopicToRemoteClusterForReplication=false diff --git a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf index e9aeed1a34da9..062ead20a7c4d 100644 --- a/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf +++ b/pulsar-broker/src/test/resources/configurations/pulsar_broker_test_standalone.conf @@ -95,4 +95,5 @@ supportedNamespaceBundleSplitAlgorithms=[range_equally_divide] defaultNamespaceBundleSplitAlgorithm=topic_count_equally_divide maxMessagePublishBufferSizeInMB=-1 topicNameCacheMaxCapacity=200 -maxSecondsToClearTopicNameCache=1 \ No newline at end of file +maxSecondsToClearTopicNameCache=1 +createTopicToRemoteClusterForReplication=true 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 f28b81e8e5544..3bf1a57649bc3 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 @@ -382,7 +382,7 @@ private CompletableFuture checkPartitions(String topic, boolean forceNo getPartitionedTopicMetadata(topic, !forceNoPartitioned, true).thenAccept(metadata -> { if (forceNoPartitioned && metadata.partitions > 0) { String errorMsg = String.format("Can not create the producer[%s] for the topic[%s] that contains %s" - + " partitions, but the producer does not support for a partitioned topic.", + + " partitions b,ut the producer does not support for a partitioned topic.", producerNameForLog, topic, metadata.partitions); log.error(errorMsg); checkPartitions.completeExceptionally( From b0c3ffba78ba99f95bd416e8a5114d616a24050f Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 26 Aug 2024 10:05:21 +0800 Subject: [PATCH 687/699] [improve] [broker] Part 2 of PIP-370: add metrics "pulsar_replication_disconnected_count" (#23213) (cherry picked from commit 09a16c26974408de270bcaaf6162b0e2a9a6d203) (cherry picked from commit fbf526848a15153727d12ab1ae4956847ec3f50f) --- .../prometheus/AggregatedNamespaceStats.java | 1 + .../AggregatedReplicationStats.java | 3 + .../prometheus/NamespaceStatsAggregator.java | 8 +- .../broker/stats/prometheus/TopicStats.java | 2 + .../broker/service/OneWayReplicatorTest.java | 123 +++++++++++++++++- .../OneWayReplicatorUsingGlobalZKTest.java | 6 + .../AggregatedNamespaceStatsTest.java | 2 + 7 files changed, 143 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java index 3975cd89cfa6b..85ff15c915aa7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStats.java @@ -133,6 +133,7 @@ void updateStats(TopicStats stats) { replStats.replicationBacklog += as.replicationBacklog; replStats.msgRateExpired += as.msgRateExpired; replStats.connectedCount += as.connectedCount; + replStats.disconnectedCount += as.disconnectedCount; replStats.replicationDelayInSeconds += as.replicationDelayInSeconds; }); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedReplicationStats.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedReplicationStats.java index 78f33f874e998..82668de6c35f7 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedReplicationStats.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/AggregatedReplicationStats.java @@ -41,6 +41,9 @@ public class AggregatedReplicationStats { /** The count of replication-subscriber up and running to replicate to remote cluster. */ public long connectedCount; + /** The count of replication-subscriber that failed to start to replicate to remote cluster. */ + public long disconnectedCount; + /** Time in seconds from the time a message was produced to the time when it is about to be replicated. */ public long replicationDelayInSeconds; diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java index 3728c3edd1e8b..d25af8d289c27 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/stats/prometheus/NamespaceStatsAggregator.java @@ -301,7 +301,11 @@ private static void getTopicStats(Topic topic, TopicStats stats, boolean include aggReplStats.msgThroughputOut += replStats.msgThroughputOut; aggReplStats.replicationBacklog += replStats.replicationBacklog; aggReplStats.msgRateExpired += replStats.msgRateExpired; - aggReplStats.connectedCount += replStats.connected ? 1 : 0; + if (replStats.connected) { + aggReplStats.connectedCount += 1; + } else { + aggReplStats.disconnectedCount += 1; + } aggReplStats.replicationDelayInSeconds += replStats.replicationDelayInSeconds; }); @@ -497,6 +501,8 @@ private static void printNamespaceStats(PrometheusMetricStreams stream, Aggregat replStats -> replStats.replicationBacklog, cluster, namespace); writeReplicationStat(stream, "pulsar_replication_connected_count", stats, replStats -> replStats.connectedCount, cluster, namespace); + writeReplicationStat(stream, "pulsar_replication_disconnected_count", stats, + replStats -> replStats.disconnectedCount, cluster, namespace); writeReplicationStat(stream, "pulsar_replication_rate_expired", stats, replStats -> replStats.msgRateExpired, cluster, namespace); writeReplicationStat(stream, "pulsar_replication_delay_in_seconds", stats, 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 27288291d2969..e907760d9d939 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 @@ -389,6 +389,8 @@ public static void printTopicStats(PrometheusMetricStreams stream, TopicStats st cluster, namespace, topic, remoteCluster, splitTopicAndPartitionIndexLabel); writeMetric(stream, "pulsar_replication_connected_count", replStats.connectedCount, cluster, namespace, topic, remoteCluster, splitTopicAndPartitionIndexLabel); + writeMetric(stream, "pulsar_replication_disconnected_count", replStats.disconnectedCount, + cluster, namespace, topic, remoteCluster, splitTopicAndPartitionIndexLabel); writeMetric(stream, "pulsar_replication_rate_expired", replStats.msgRateExpired, cluster, namespace, topic, remoteCluster, splitTopicAndPartitionIndexLabel); writeMetric(stream, "pulsar_replication_delay_in_seconds", replStats.replicationDelayInSeconds, 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 90a67ead4d0f5..00d43b796570e 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 @@ -29,6 +29,7 @@ import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import io.netty.util.concurrent.FastThreadLocalThread; import java.lang.reflect.Field; @@ -64,6 +65,7 @@ import org.apache.pulsar.broker.service.persistent.GeoPersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentReplicator; import org.apache.pulsar.broker.service.persistent.PersistentTopic; +import org.apache.pulsar.broker.stats.prometheus.PrometheusMetricsClient; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; @@ -78,8 +80,8 @@ import org.apache.pulsar.client.impl.conf.ProducerConfigurationData; 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.AutoTopicCreationOverride; +import org.apache.pulsar.common.policies.data.ClusterData; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.common.policies.data.TopicStats; @@ -88,6 +90,8 @@ import org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; +import org.glassfish.jersey.client.JerseyClient; +import org.glassfish.jersey.client.JerseyClientBuilder; import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterClass; @@ -1128,4 +1132,121 @@ public void testDifferentTopicCreationRule(ReplicationMode replicationMode) thro admin1.namespaces().deleteNamespace(ns); admin2.namespaces().deleteNamespace(ns); } + + @Test + public void testReplicationCountMetrics() throws Exception { + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + nonReplicatedNamespace + "/tp_"); + // 1.Create topic, does not enable replication now. + admin1.topics().createNonPartitionedTopic(topicName); + PersistentTopic persistentTopic = + (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get(); + + // We inject an error to make the internal producer fail to connect. + final AtomicInteger createProducerCounter = new AtomicInteger(); + final AtomicBoolean failedCreateProducer = new AtomicBoolean(true); + Runnable taskToClearInjection = injectMockReplicatorProducerBuilder((producerCnf, originalProducer) -> { + if (topicName.equals(producerCnf.getTopicName())) { + // There is a switch to determine create producer successfully or not. + if (failedCreateProducer.get()) { + log.info("Retry create replicator.producer count: {}", createProducerCounter); + // Release producer and fail callback. + originalProducer.closeAsync(); + throw new RuntimeException("mock error"); + } + return originalProducer; + } + return originalProducer; + }); + + // 2.Enable replication. + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1, cluster2)); + + // Verify: metrics. + // Cluster level: + // - pulsar_replication_connected_count + // - pulsar_replication_disconnected_count + // Namespace level: + // - pulsar_replication_connected_count + // - pulsar_replication_disconnected_count + // Topic level: + // - pulsar_replication_connected_count + // - pulsar_replication_disconnected_count + JerseyClient httpClient = JerseyClientBuilder.createClient(); + Awaitility.await().untilAsserted(() -> { + int topicConnected = 0; + int topicDisconnected = 0; + + String response = httpClient.target(pulsar1.getWebServiceAddress()).path("/metrics/") + .request().get(String.class); + Multimap metricMap = PrometheusMetricsClient.parseMetrics(response); + if (!metricMap.containsKey("pulsar_replication_disconnected_count")) { + fail("Expected 1 disconnected replicator."); + } + for (PrometheusMetricsClient.Metric metric : metricMap.get("pulsar_replication_connected_count")) { + if (cluster1.equals(metric.tags.get("cluster")) + && nonReplicatedNamespace.equals(metric.tags.get("namespace")) + && topicName.equals(metric.tags.get("topic"))) { + topicConnected += Double.valueOf(metric.value).intValue(); + } + } + for (PrometheusMetricsClient.Metric metric : metricMap.get("pulsar_replication_disconnected_count")) { + if (cluster1.equals(metric.tags.get("cluster")) + && nonReplicatedNamespace.equals(metric.tags.get("namespace")) + && topicName.equals(metric.tags.get("topic"))) { + topicDisconnected += Double.valueOf(metric.value).intValue(); + } + } + log.info("{}, {},", topicConnected, topicDisconnected); + assertEquals(topicConnected, 0); + assertEquals(topicDisconnected, 1); + }); + + // Let replicator connect successfully. + failedCreateProducer.set(false); + // Verify: metrics. + // Cluster level: + // - pulsar_replication_connected_count + // - pulsar_replication_disconnected_count + // Namespace level: + // - pulsar_replication_connected_count + // - pulsar_replication_disconnected_count + // Topic level: + // - pulsar_replication_connected_count + // - pulsar_replication_disconnected_count + Awaitility.await().atMost(Duration.ofSeconds(130)).untilAsserted(() -> { + int topicConnected = 0; + int topicDisconnected = 0; + + String response = httpClient.target(pulsar1.getWebServiceAddress()).path("/metrics/") + .request().get(String.class); + Multimap metricMap = PrometheusMetricsClient.parseMetrics(response); + if (!metricMap.containsKey("pulsar_replication_disconnected_count")) { + fail("Expected 1 disconnected replicator."); + } + for (PrometheusMetricsClient.Metric metric : metricMap.get("pulsar_replication_connected_count")) { + if (cluster1.equals(metric.tags.get("cluster")) + && nonReplicatedNamespace.equals(metric.tags.get("namespace")) + && topicName.equals(metric.tags.get("topic"))) { + topicConnected += Double.valueOf(metric.value).intValue(); + } + } + for (PrometheusMetricsClient.Metric metric : metricMap.get("pulsar_replication_disconnected_count")) { + if (cluster1.equals(metric.tags.get("cluster")) + && nonReplicatedNamespace.equals(metric.tags.get("namespace")) + && topicName.equals(metric.tags.get("topic"))) { + topicDisconnected += Double.valueOf(metric.value).intValue(); + } + } + log.info("{}, {}", topicConnected, topicDisconnected); + assertEquals(topicConnected, 1); + assertEquals(topicDisconnected, 0); + }); + + // cleanup. + taskToClearInjection.run(); + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1)); + waitReplicatorStopped(topicName); + admin1.topics().delete(topicName, false); + admin2.topics().delete(topicName, false); + } } 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 34810bbe9057b..d99969fbaa7e5 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 @@ -167,4 +167,10 @@ public void testConfigReplicationStartAt() throws Exception { public void testDifferentTopicCreationRule(ReplicationMode replicationMode) throws Exception { super.testDifferentTopicCreationRule(replicationMode); } + + @Test(enabled = false) + @Override + public void testReplicationCountMetrics() throws Exception { + super.testReplicationCountMetrics(); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java index 0e12d75f74fa0..11358eb1e2c1c 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/stats/prometheus/AggregatedNamespaceStatsTest.java @@ -101,6 +101,7 @@ public void testSimpleAggregation() { replStats2.msgThroughputOut = 1536.0; replStats2.replicationBacklog = 99; replStats2.connectedCount = 1; + replStats2.disconnectedCount = 2; replStats2.msgRateExpired = 3.0; replStats2.replicationDelayInSeconds = 20; topicStats2.replicationStats.put(namespace, replStats2); @@ -148,6 +149,7 @@ public void testSimpleAggregation() { assertEquals(nsReplStats.msgThroughputOut, 1792.0); assertEquals(nsReplStats.replicationBacklog, 100); assertEquals(nsReplStats.connectedCount, 1); + assertEquals(nsReplStats.disconnectedCount, 2); assertEquals(nsReplStats.msgRateExpired, 6.0); assertEquals(nsReplStats.replicationDelayInSeconds, 40); From 4322edf63aede24b3a0fe98bbade038ad831001a Mon Sep 17 00:00:00 2001 From: nikhil-ctds Date: Wed, 28 Aug 2024 21:55:14 +0530 Subject: [PATCH 688/699] Release 3.1.4.5 --- 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-experimental/pom.xml | 2 +- docker/pulsar/pom.xml | 2 +- jclouds-shaded/pom.xml | 2 +- jetcd-core-shaded/pom.xml | 2 +- managed-ledger/pom.xml | 2 +- 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-client-1x-base/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-1x/pom.xml | 2 +- pulsar-client-1x-base/pulsar-client-2x-shaded/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/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/hdfs2/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-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-sql/pom.xml | 2 +- pulsar-sql/presto-distribution/pom.xml | 2 +- pulsar-sql/presto-pulsar-plugin/pom.xml | 2 +- pulsar-sql/presto-pulsar/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 +- 135 files changed, 137 insertions(+), 137 deletions(-) diff --git a/bouncy-castle/bc/pom.xml b/bouncy-castle/bc/pom.xml index c7af0a2d57aa7..2c03088df0883 100644 --- a/bouncy-castle/bc/pom.xml +++ b/bouncy-castle/bc/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.4.5-SNAPSHOT + 3.1.4.5 .. diff --git a/bouncy-castle/bcfips-include-test/pom.xml b/bouncy-castle/bcfips-include-test/pom.xml index fd620aa7c3c8d..263b1658ec7ba 100644 --- a/bouncy-castle/bcfips-include-test/pom.xml +++ b/bouncy-castle/bcfips-include-test/pom.xml @@ -24,7 +24,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.4.5-SNAPSHOT + 3.1.4.5 .. diff --git a/bouncy-castle/bcfips/pom.xml b/bouncy-castle/bcfips/pom.xml index 63b7b5c153d62..c5c9b0505c479 100644 --- a/bouncy-castle/bcfips/pom.xml +++ b/bouncy-castle/bcfips/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss bouncy-castle-parent - 3.1.4.5-SNAPSHOT + 3.1.4.5 .. diff --git a/bouncy-castle/pom.xml b/bouncy-castle/pom.xml index 5b83606d2d85d..bbfc970413529 100644 --- a/bouncy-castle/pom.xml +++ b/bouncy-castle/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.4.5-SNAPSHOT + 3.1.4.5 .. diff --git a/buildtools/pom.xml b/buildtools/pom.xml index ecac4cc6fedd6..f7bf0328619ed 100644 --- a/buildtools/pom.xml +++ b/buildtools/pom.xml @@ -31,12 +31,12 @@ com.datastax.oss buildtools - 3.1.4.5-SNAPSHOT + 3.1.4.5 jar Pulsar Build Tools - 2024-08-01T10:13:06Z + 2024-08-28T16:25:11Z 1.8 1.8 3.1.0 diff --git a/distribution/io/pom.xml b/distribution/io/pom.xml index 5f8982debdc86..18e7198bbbd1a 100644 --- a/distribution/io/pom.xml +++ b/distribution/io/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.4.5-SNAPSHOT + 3.1.4.5 .. diff --git a/distribution/offloaders/pom.xml b/distribution/offloaders/pom.xml index f5faf92c63482..6f1d76f89a143 100644 --- a/distribution/offloaders/pom.xml +++ b/distribution/offloaders/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.4.5-SNAPSHOT + 3.1.4.5 .. diff --git a/distribution/pom.xml b/distribution/pom.xml index 33aeab9e646aa..f4d1fa7bcf89a 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.4.5-SNAPSHOT + 3.1.4.5 .. diff --git a/distribution/server/pom.xml b/distribution/server/pom.xml index 10ead82462c55..358987acbc52e 100644 --- a/distribution/server/pom.xml +++ b/distribution/server/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.4.5-SNAPSHOT + 3.1.4.5 .. diff --git a/distribution/shell/pom.xml b/distribution/shell/pom.xml index 657541eac9c46..94941c737fd58 100644 --- a/distribution/shell/pom.xml +++ b/distribution/shell/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss distribution - 3.1.4.5-SNAPSHOT + 3.1.4.5 .. diff --git a/docker/pom.xml b/docker/pom.xml index d010cf3d8f56d..a57d8df657ad2 100644 --- a/docker/pom.xml +++ b/docker/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.4.5-SNAPSHOT + 3.1.4.5 docker-images Apache Pulsar :: Docker Images diff --git a/docker/pulsar-all/pom.xml b/docker/pulsar-all/pom.xml index f203d8efb9b21..d9cdb89ec0d04 100644 --- a/docker/pulsar-all/pom.xml +++ b/docker/pulsar-all/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.4.5-SNAPSHOT + 3.1.4.5 4.0.0 pulsar-all-docker-image diff --git a/docker/pulsar-experimental/pom.xml b/docker/pulsar-experimental/pom.xml index 6ec181fdf8d3c..bde30bb8a1f04 100644 --- a/docker/pulsar-experimental/pom.xml +++ b/docker/pulsar-experimental/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.4.5-SNAPSHOT + 3.1.4.5 4.0.0 pulsar-experimental-docker-image diff --git a/docker/pulsar/pom.xml b/docker/pulsar/pom.xml index 241eb20b13420..eebd9e3486dff 100644 --- a/docker/pulsar/pom.xml +++ b/docker/pulsar/pom.xml @@ -23,7 +23,7 @@ com.datastax.oss docker-images - 3.1.4.5-SNAPSHOT + 3.1.4.5 4.0.0 pulsar-docker-image diff --git a/jclouds-shaded/pom.xml b/jclouds-shaded/pom.xml index 3e2539f23869e..11e0c30e04531 100644 --- a/jclouds-shaded/pom.xml +++ b/jclouds-shaded/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.4.5-SNAPSHOT + 3.1.4.5 .. diff --git a/jetcd-core-shaded/pom.xml b/jetcd-core-shaded/pom.xml index 4ffd2edae465b..42770034e0889 100644 --- a/jetcd-core-shaded/pom.xml +++ b/jetcd-core-shaded/pom.xml @@ -26,7 +26,7 @@ com.datastax.oss pulsar - 3.1.4.5-SNAPSHOT + 3.1.4.5 jetcd-core-shaded diff --git a/managed-ledger/pom.xml b/managed-ledger/pom.xml index 89b21881bf36d..bf3a9c711fda5 100644 --- a/managed-ledger/pom.xml +++ b/managed-ledger/pom.xml @@ -25,7 +25,7 @@ com.datastax.oss pulsar - 3.1.4.5-SNAPSHOT + 3.1.4.5 .. diff --git a/pom.xml b/pom.xml index 143200dc1bd88..0a60101d680e7 100644 --- a/pom.xml +++ b/pom.xml @@ -32,7 +32,7 @@ com.datastax.oss pulsar - 3.1.4.5-SNAPSHOT + 3.1.4.5 Pulsar Pulsar is a distributed pub-sub messaging platform with a very @@ -95,7 +95,7 @@ flexible messaging model and an intuitive client API. UTF-8 UTF-8 - 2024-08-01T10:13:06Z + 2024-08-28T16:25:11Z true ### Motivation In our scenario, there is a centralized message proxy service, this service will use the same PulsarClient instance to create a lot of subscription groups to consume many topics and cache messages locally.Then the business will pull messages from the cache of the proxy service. It seems that there is no problem, but during use, we found that when the message processing time of several consumer groups (listener mode) is very high, it almost affects all consumer groups responsible for the proxy service, causing a large number of message delays. By analyzing the source code, we found that by default, all consumer instances created from the same PulsarClient will share a thread pool to process message listeners, and sometimes there are multiple consumer message listeners bound to the same thread. Obviously, when a consumer processes messages and causes long-term blocking, it will cause the messages of other consumers bound to the thread to fail to be processed in time, resulting in message delays. Therefore, for this scenario, it may be necessary to support specific a message listener thread pool with consumer latitudes to avoid mutual influence between different consumers. ### Modifications Support custom message listener thread pool for specific subscription. (cherry picked from commit 10f4e0248f0f985b1dc7ad38970c906b7fe629be) (cherry picked from commit c5846bbbecdbd4dd197679f78874eb398da38c23) --- .../api/MessageListenerExecutorTest.java | 193 ++++++++++++++++++ .../pulsar/client/api/ConsumerBuilder.java | 15 ++ .../client/api/MessageListenerExecutor.java | 43 ++++ .../pulsar/client/impl/ConsumerBase.java | 28 ++- .../client/impl/ConsumerBuilderImpl.java | 8 + .../impl/conf/ConsumerConfigurationData.java | 3 + 6 files changed, 280 insertions(+), 10 deletions(-) create mode 100644 pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageListenerExecutorTest.java create mode 100644 pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessageListenerExecutor.java diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageListenerExecutorTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageListenerExecutorTest.java new file mode 100644 index 0000000000000..9e148beb3045d --- /dev/null +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/MessageListenerExecutorTest.java @@ -0,0 +1,193 @@ +/* + * 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.testng.Assert.assertTrue; +import com.google.common.util.concurrent.Uninterruptibles; +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicLong; +import lombok.Cleanup; +import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.common.naming.TopicName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +@Test(groups = "broker-api") +public class MessageListenerExecutorTest extends ProducerConsumerBase { + private static final Logger log = LoggerFactory.getLogger(MessageListenerExecutorTest.class); + + @BeforeClass(alwaysRun = true) + @Override + protected void setup() throws Exception { + super.internalSetup(); + super.producerBaseSetup(); + } + + @AfterClass(alwaysRun = true) + @Override + protected void cleanup() throws Exception { + super.internalCleanup(); + } + + @Override + protected void customizeNewPulsarClientBuilder(ClientBuilder clientBuilder) { + // Set listenerThreads to 1 to reproduce the pr more easily in #22861 + clientBuilder.listenerThreads(1); + } + + @Test + public void testConsumerMessageListenerExecutorIsolation() throws Exception { + log.info("-- Starting {} test --", methodName); + + @Cleanup("shutdownNow") + ExecutorService executor = Executors.newCachedThreadPool(); + List> maxConsumeDelayWithDisableIsolationFutures = new ArrayList<>(); + int loops = 5; + long consumeSleepTimeMs = 10000; + for (int i = 0; i < loops; i++) { + // The first consumer will consume messages with sleep block 1s, + // and the others will consume messages without sleep block. + // The maxConsumeDelayWithDisableIsolation of all consumers + // should be greater than sleepTimeMs cause by disable MessageListenerExecutor. + CompletableFuture maxConsumeDelayFuture = startConsumeAndComputeMaxConsumeDelay( + "persistent://my-property/my-ns/testConsumerMessageListenerDisableIsolation-" + i, + "my-sub-testConsumerMessageListenerDisableIsolation-" + i, + i == 0 ? Duration.ofMillis(consumeSleepTimeMs) : Duration.ofMillis(0), + false, + executor); + maxConsumeDelayWithDisableIsolationFutures.add(maxConsumeDelayFuture); + } + + // ensure all consumers consume messages delay more than consumeSleepTimeMs + boolean allDelayMoreThanConsumeSleepTimeMs = maxConsumeDelayWithDisableIsolationFutures.stream() + .map(CompletableFuture::join) + .allMatch(delay -> delay > consumeSleepTimeMs); + assertTrue(allDelayMoreThanConsumeSleepTimeMs); + + List> maxConsumeDelayWhitEnableIsolationFutures = new ArrayList<>(); + for (int i = 0; i < loops; i++) { + // The first consumer will consume messages with sleep block 1s, + // and the others will consume messages without sleep block. + // The maxConsumeDelayWhitEnableIsolation of the first consumer + // should be greater than sleepTimeMs, and the others should be + // less than sleepTimeMs, cause by enable MessageListenerExecutor. + CompletableFuture maxConsumeDelayFuture = startConsumeAndComputeMaxConsumeDelay( + "persistent://my-property/my-ns/testConsumerMessageListenerEnableIsolation-" + i, + "my-sub-testConsumerMessageListenerEnableIsolation-" + i, + i == 0 ? Duration.ofMillis(consumeSleepTimeMs) : Duration.ofMillis(0), + true, + executor); + maxConsumeDelayWhitEnableIsolationFutures.add(maxConsumeDelayFuture); + } + + assertTrue(maxConsumeDelayWhitEnableIsolationFutures.get(0).join() > consumeSleepTimeMs); + boolean remainingAlmostNoDelay = maxConsumeDelayWhitEnableIsolationFutures.stream() + .skip(1) + .map(CompletableFuture::join) + .allMatch(delay -> delay < 1000); + assertTrue(remainingAlmostNoDelay); + + log.info("-- Exiting {} test --", methodName); + } + + private CompletableFuture startConsumeAndComputeMaxConsumeDelay(String topic, String subscriptionName, + Duration consumeSleepTime, + boolean enableMessageListenerExecutorIsolation, + ExecutorService executorService) + throws Exception { + int numMessages = 2; + final CountDownLatch latch = new CountDownLatch(numMessages); + int numPartitions = 50; + TopicName nonIsolationTopicName = TopicName.get(topic); + admin.topics().createPartitionedTopic(nonIsolationTopicName.toString(), numPartitions); + + AtomicLong maxConsumeDelay = new AtomicLong(-1); + ConsumerBuilder consumerBuilder = + pulsarClient.newConsumer(Schema.INT64) + .topic(nonIsolationTopicName.toString()) + .subscriptionName(subscriptionName) + .messageListener((c1, msg) -> { + Assert.assertNotNull(msg, "Message cannot be null"); + log.debug("Received message [{}] in the listener", msg.getValue()); + c1.acknowledgeAsync(msg); + maxConsumeDelay.set(Math.max(maxConsumeDelay.get(), + System.currentTimeMillis() - msg.getValue())); + if (consumeSleepTime.toMillis() > 0) { + Uninterruptibles.sleepUninterruptibly(consumeSleepTime); + } + latch.countDown(); + }); + + ExecutorService executor = Executors.newSingleThreadExecutor( + new ExecutorProvider.ExtendedThreadFactory(subscriptionName + "listener-executor-", true)); + if (enableMessageListenerExecutorIsolation) { + consumerBuilder.messageListenerExecutor((message, runnable) -> executor.execute(runnable)); + } + + Consumer consumer = consumerBuilder.subscribe(); + ProducerBuilder producerBuilder = pulsarClient.newProducer(Schema.INT64) + .topic(nonIsolationTopicName.toString()); + + Producer producer = producerBuilder.create(); + List> futures = new ArrayList<>(); + + // Asynchronously produce messages + for (int i = 0; i < numMessages; i++) { + Future future = producer.sendAsync(System.currentTimeMillis()); + futures.add(future); + } + + log.info("Waiting for async publish to complete"); + for (Future future : futures) { + future.get(); + } + + CompletableFuture maxDelayFuture = new CompletableFuture<>(); + + CompletableFuture.runAsync(() -> { + try { + latch.await(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }, executorService).whenCompleteAsync((v, ex) -> { + maxDelayFuture.complete(maxConsumeDelay.get()); + try { + producer.close(); + consumer.close(); + executor.shutdownNow(); + } catch (PulsarClientException e) { + throw new RuntimeException(e); + } + }); + + return maxDelayFuture; + } +} 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 588a34d2069da..ec944c4967dcf 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 @@ -282,6 +282,21 @@ public interface ConsumerBuilder extends Cloneable { */ ConsumerBuilder messageListener(MessageListener messageListener); + /** + * Set the {@link MessageListenerExecutor} to be used for message listeners of current consumer. + * (default: use executor from PulsarClient, + * {@link org.apache.pulsar.client.impl.PulsarClientImpl#externalExecutorProvider}). + * + *

    The listener thread pool is exclusively owned by current consumer + * that are using a "listener" model to get messages. For a given internal consumer, + * the listener will always be invoked from the same thread, to ensure ordering. + * + *

    The caller need to shut down the thread pool after closing the consumer to avoid leaks. + * @param messageListenerExecutor the executor of the consumer message listener + * @return the consumer builder instance + */ + ConsumerBuilder messageListenerExecutor(MessageListenerExecutor messageListenerExecutor); + /** * Sets a {@link CryptoKeyReader}. * diff --git a/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessageListenerExecutor.java b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessageListenerExecutor.java new file mode 100644 index 0000000000000..53bb828c05aa8 --- /dev/null +++ b/pulsar-client-api/src/main/java/org/apache/pulsar/client/api/MessageListenerExecutor.java @@ -0,0 +1,43 @@ +/* + * 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; + +/** + * Interface for providing service to execute message listeners. + */ +public interface MessageListenerExecutor { + + /** + * select a thread by message to execute the runnable! + *

    + * Suggestions: + *

    + * 1. The message listener task will be submitted to this executor for execution, + * so the implementations of this interface should carefully consider execution + * order if sequential consumption is required. + *

    + *

    + * 2. The users should release resources(e.g. threads) of the executor after closing + * the consumer to avoid leaks. + *

    + * @param message the message + * @param runnable the runnable to execute, that is, the message listener task + */ + void execute(Message message, Runnable runnable); +} 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 3747dc23d9d10..2b4a3f334cd8a 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 @@ -50,6 +50,7 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.MessageIdAdv; import org.apache.pulsar.client.api.MessageListener; +import org.apache.pulsar.client.api.MessageListenerExecutor; import org.apache.pulsar.client.api.Messages; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; @@ -82,6 +83,7 @@ public abstract class ConsumerBase extends HandlerState implements Consumer listener; protected final ConsumerEventListener consumerEventListener; protected final ExecutorProvider executorProvider; + protected final MessageListenerExecutor messageListenerExecutor; protected final ExecutorService externalPinnedExecutor; protected final ExecutorService internalPinnedExecutor; protected UnAckedMessageTracker unAckedMessageTracker; @@ -139,6 +141,11 @@ protected ConsumerBase(PulsarClientImpl client, String topic, ConsumerConfigurat this.unAckedChunkedMessageIdSequenceMap = ConcurrentOpenHashMap.newBuilder().build(); this.executorProvider = executorProvider; + this.messageListenerExecutor = conf.getMessageListenerExecutor() == null + ? (conf.getSubscriptionType() == SubscriptionType.Key_Shared + ? this::executeKeySharedMessageListener + : this::executeMessageListener) + : conf.getMessageListenerExecutor(); this.externalPinnedExecutor = executorProvider.getExecutor(); this.internalPinnedExecutor = client.getInternalExecutorService(); this.pendingReceives = Queues.newConcurrentLinkedQueue(); @@ -1117,14 +1124,7 @@ private void triggerListener() { // internal pinned executor thread while the message processing happens final Message finalMsg = msg; MESSAGE_LISTENER_QUEUE_SIZE_UPDATER.incrementAndGet(this); - if (SubscriptionType.Key_Shared == conf.getSubscriptionType()) { - executorProvider.getExecutor(peekMessageKey(msg)).execute(() -> - callMessageListener(finalMsg)); - } else { - getExternalExecutor(msg).execute(() -> { - callMessageListener(finalMsg); - }); - } + messageListenerExecutor.execute(msg, () -> callMessageListener(finalMsg)); } else { if (log.isDebugEnabled()) { log.debug("[{}] [{}] Message has been cleared from the queue", topic, subscription); @@ -1137,6 +1137,14 @@ private void triggerListener() { }); } + private void executeMessageListener(Message message, Runnable runnable) { + getExternalExecutor(message).execute(runnable); + } + + private void executeKeySharedMessageListener(Message message, Runnable runnable) { + executorProvider.getExecutor(peekMessageKey(message)).execute(runnable); + } + protected void callMessageListener(Message msg) { try { if (log.isDebugEnabled()) { @@ -1166,7 +1174,7 @@ protected void callMessageListener(Message msg) { } static final byte[] NONE_KEY = "NONE_KEY".getBytes(StandardCharsets.UTF_8); - protected byte[] peekMessageKey(Message msg) { + protected byte[] peekMessageKey(Message msg) { byte[] key = NONE_KEY; if (msg.hasKey()) { key = msg.getKeyBytes(); @@ -1233,7 +1241,7 @@ public int getCurrentReceiverQueueSize() { protected abstract void completeOpBatchReceive(OpBatchReceive op); - private ExecutorService getExternalExecutor(Message msg) { + private ExecutorService getExternalExecutor(Message msg) { ConsumerImpl receivedConsumer = (msg instanceof TopicMessageImpl) ? ((TopicMessageImpl) msg).receivedByconsumer : null; ExecutorService executor = receivedConsumer != null && receivedConsumer.externalPinnedExecutor != null 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 2d31dc427d0c6..351025d426a39 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 @@ -44,6 +44,7 @@ import org.apache.pulsar.client.api.KeySharedPolicy; import org.apache.pulsar.client.api.MessageCrypto; import org.apache.pulsar.client.api.MessageListener; +import org.apache.pulsar.client.api.MessageListenerExecutor; import org.apache.pulsar.client.api.MessagePayloadProcessor; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.PulsarClientException.InvalidConfigurationException; @@ -299,6 +300,13 @@ public ConsumerBuilder messageListener(@NonNull MessageListener messageLis return this; } + @Override + public ConsumerBuilder messageListenerExecutor(MessageListenerExecutor messageListenerExecutor) { + checkArgument(messageListenerExecutor != null, "messageListenerExecutor needs to be not null"); + conf.setMessageListenerExecutor(messageListenerExecutor); + return this; + } + @Override public ConsumerBuilder consumerEventListener(@NonNull ConsumerEventListener consumerEventListener) { conf.setConsumerEventListener(consumerEventListener); 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 3ae0e977d13c4..2bb7ef79c647a 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 @@ -43,6 +43,7 @@ import org.apache.pulsar.client.api.KeySharedPolicy; import org.apache.pulsar.client.api.MessageCrypto; import org.apache.pulsar.client.api.MessageListener; +import org.apache.pulsar.client.api.MessageListenerExecutor; import org.apache.pulsar.client.api.MessagePayloadProcessor; import org.apache.pulsar.client.api.RedeliveryBackoff; import org.apache.pulsar.client.api.RegexSubscriptionMode; @@ -90,6 +91,8 @@ public class ConsumerConfigurationData implements Serializable, Cloneable { private SubscriptionMode subscriptionMode = SubscriptionMode.Durable; + @JsonIgnore + private transient MessageListenerExecutor messageListenerExecutor; @JsonIgnore private MessageListener messageListener; From fa412fa5cc2b947dd876921d5e61f78ad034abc7 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 2 Sep 2024 21:33:41 +0800 Subject: [PATCH 699/699] [fix][broker] Fix brokers still retry start replication after closed the topic (#23237) (cherry picked from commit aee2ee5070d07c683c54877bc1457a58e273440b) (cherry picked from commit 311b6af2629516ca1d7c520076d3d44374427516) --- .../service/persistent/PersistentTopic.java | 20 +++++++++ .../broker/service/OneWayReplicatorTest.java | 44 +++++++++++++++++++ 2 files changed, 64 insertions(+) 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 dd31031899db4..5d0d837bf9ac4 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 @@ -1689,6 +1689,23 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { return closeFuture; } + private boolean isClosed() { + if (closeFutures == null) { + return false; + } + if (closeFutures.notWaitDisconnectClients != null + && closeFutures.notWaitDisconnectClients.isDone() + && !closeFutures.notWaitDisconnectClients.isCompletedExceptionally()) { + return true; + } + if (closeFutures.waitDisconnectClients != null + && closeFutures.waitDisconnectClients.isDone() + && !closeFutures.waitDisconnectClients.isCompletedExceptionally()) { + return true; + } + return false; + } + private void disposeTopic(CompletableFuture closeFuture) { brokerService.removeTopicFromCache(PersistentTopic.this) .thenRun(() -> { @@ -1711,6 +1728,9 @@ private void disposeTopic(CompletableFuture closeFuture) { @VisibleForTesting CompletableFuture checkReplicationAndRetryOnFailure() { + if (isClosed()) { + return CompletableFuture.completedFuture(null); + } CompletableFuture result = new CompletableFuture(); checkReplication().thenAccept(res -> { result.complete(null); 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 00d43b796570e..79b856b1da07b 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 @@ -29,6 +29,7 @@ import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import com.github.benmanes.caffeine.cache.AsyncLoadingCache; import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import io.netty.util.concurrent.FastThreadLocalThread; @@ -43,6 +44,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -1249,4 +1251,46 @@ public void testReplicationCountMetrics() throws Exception { admin1.topics().delete(topicName, false); admin2.topics().delete(topicName, false); } + + /** + * This test used to confirm the "start replicator retry task" will be skipped after the topic is closed. + */ + @Test + public void testCloseTopicAfterStartReplicationFailed() throws Exception { + Field fieldTopicNameCache = TopicName.class.getDeclaredField("cache"); + fieldTopicNameCache.setAccessible(true); + ConcurrentHashMap topicNameCache = + (ConcurrentHashMap) fieldTopicNameCache.get(null); + final String topicName = BrokerTestUtil.newUniqueName("persistent://" + nonReplicatedNamespace + "/tp_"); + // 1.Create topic, does not enable replication now. + admin1.topics().createNonPartitionedTopic(topicName); + Producer producer1 = client1.newProducer().topic(topicName).create(); + PersistentTopic persistentTopic = + (PersistentTopic) pulsar1.getBrokerService().getTopic(topicName, false).join().get(); + + // We inject an error to make "start replicator" to fail. + AsyncLoadingCache existsCache = + WhiteboxImpl.getInternalState(pulsar1.getConfigurationMetadataStore(), "existsCache"); + String path = "/admin/partitioned-topics/" + TopicName.get(topicName).getPersistenceNamingEncoding(); + existsCache.put(path, CompletableFuture.completedFuture(true)); + + // 2.Enable replication and unload topic after failed to start replicator. + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1, cluster2)); + Thread.sleep(3000); + producer1.close(); + existsCache.synchronous().invalidate(path); + admin1.topics().unload(topicName); + // Verify: the "start replicator retry task" will be skipped after the topic is closed. + // - Retry delay is "PersistentTopic.POLICY_UPDATE_FAILURE_RETRY_TIME_SECONDS": 60s, so wait for 70s. + // - Since the topic should not be touched anymore, we use "TopicName" to confirm whether it be used by + // Replication again. + Thread.sleep(10 * 1000); + topicNameCache.remove(topicName); + Thread.sleep(60 * 1000); + assertTrue(!topicNameCache.containsKey(topicName)); + + // cleanup. + admin1.topics().setReplicationClusters(topicName, Arrays.asList(cluster1)); + admin1.topics().delete(topicName, false); + } }